From d998c96d801d97e06668b5f575047dd7fc74b5a9 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 2 Jul 2019 19:59:55 -0700 Subject: [PATCH 01/15] [SPARK-28238][SQL] Implement DESCRIBE TABLE for Data Source V2 Tables. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../catalyst/plans/DescribeTableSchemas.scala | 35 +++++++ .../plans/logical/basicLogicalOperators.scala | 10 ++ .../logical/sql/DescribeColumnStatement.scala | 35 +++++++ .../logical/sql/DescribeTableStatement.scala | 29 ++++++ .../spark/sql/execution/SparkSqlParser.scala | 9 +- .../spark/sql/execution/command/tables.scala | 11 +-- .../datasources/DataSourceResolution.scala | 28 +++++- .../datasources/v2/DataSourceV2Strategy.scala | 6 +- .../datasources/v2/DescribeTableExec.scala | 92 +++++++++++++++++++ .../sql/sources/v2/DataSourceV2SQLSuite.scala | 48 +++++++++- .../sources/v2/TestInMemoryTableCatalog.scala | 16 ++-- 12 files changed, 292 insertions(+), 29 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala 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 aff1415de4e8..85ffa97f9499 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 @@ -190,7 +190,7 @@ statement | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction | (DESC | DESCRIBE) database EXTENDED? db=errorCapturingIdentifier #describeDatabase | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? - tableIdentifier partitionSpec? describeColName? #describeTable + multipartIdentifier partitionSpec? describeColName? #describeTable | (DESC | DESCRIBE) QUERY? query #describeQuery | REFRESH TABLE tableIdentifier #refreshTable | REFRESH (STRING | .*?) #refreshResource diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala new file mode 100644 index 000000000000..d23888d9189a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans + +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.types.{MetadataBuilder, StringType, StructField, StructType} + +private[sql] object DescribeTableSchemas { + val DESCRIBE_TABLE_ATTRIBUTES = Seq( + AttributeReference("col_name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the column").build())(), + AttributeReference("data_type", StringType, nullable = false, + new MetadataBuilder().putString("comment", "data type of the column").build())(), + AttributeReference("comment", StringType, nullable = true, + new MetadataBuilder().putString("comment", "comment of the column").build())() + ) + + val DESCRIBE_TABLE_SCHEMA = StructType( + DESCRIBE_TABLE_ATTRIBUTES.map(attr => StructField(attr.name, attr.dataType, attr.nullable))) +} 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 93a558de584e..3de1552337fe 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 @@ -22,9 +22,11 @@ import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.AliasIdentifier import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction} import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.sql.DescribeTableStatement import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.types._ @@ -499,6 +501,14 @@ object OverwritePartitionsDynamic { } } +case class DescribeTable( + catalog: TableCatalog, + ident: Identifier, + isExtended: Boolean) extends Command { + override lazy val output = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES + override lazy val schema = DescribeTableSchemas.DESCRIBE_TABLE_SCHEMA +} + /** * Drop a table. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala new file mode 100644 index 000000000000..80ee262af672 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical.sql + +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.types.{MetadataBuilder, StringType} + +case class DescribeColumnStatement( + tableName: Seq[String], + colNameParts: Seq[String], + isExtended: Boolean) extends ParsedStatement { + override def output: Seq[Attribute] = { + Seq( + AttributeReference("info_name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the column info").build())(), + AttributeReference("info_value", StringType, nullable = false, + new MetadataBuilder().putString("comment", "value of the column info").build())() + ) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala new file mode 100644 index 000000000000..fc7ffef0a799 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.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.plans.logical.sql + +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.DescribeTableSchemas + +case class DescribeTableStatement( + tableName: Seq[String], + partitionSpec: TablePartitionSpec, + extended: Boolean) extends ParsedStatement { + override def output: Seq[Attribute] = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 12cd8abcad89..a1b08613096a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} @@ -336,8 +337,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { if (ctx.partitionSpec != null) { throw new ParseException("DESC TABLE COLUMN for a specific partition is not supported", ctx) } else { - DescribeColumnCommand( - visitTableIdentifier(ctx.tableIdentifier), + DescribeColumnStatement( + visitMultipartIdentifier(ctx.multipartIdentifier()), ctx.describeColName.nameParts.asScala.map(_.getText), isExtended) } @@ -352,8 +353,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } else { Map.empty[String, String] } - DescribeTableCommand( - visitTableIdentifier(ctx.tableIdentifier), + DescribeTableStatement( + visitMultipartIdentifier(ctx.multipartIdentifier()), partitionSpec, isExtended) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 03aca89bc642..14f46f8a66df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.DescribeTableSchemas import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} @@ -499,15 +500,7 @@ case class TruncateTableCommand( } abstract class DescribeCommandBase extends RunnableCommand { - override val output: Seq[Attribute] = Seq( - // Column names are based on Hive. - AttributeReference("col_name", StringType, nullable = false, - new MetadataBuilder().putString("comment", "name of the column").build())(), - AttributeReference("data_type", StringType, nullable = false, - new MetadataBuilder().putString("comment", "data type of the column").build())(), - AttributeReference("comment", StringType, nullable = true, - new MetadataBuilder().putString("comment", "comment of the column").build())() - ) + override val output = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES protected def describeSchema( schema: 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 26f7230c8fe8..36a8d2ac0734 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,10 +27,10 @@ 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} -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, DescribeTable, DropTable, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} 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.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2.TableProvider import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} @@ -85,6 +85,28 @@ case class DataSourceResolution( .asTableCatalog convertCTAS(catalog, identifier, create) + case DescribeColumnStatement( + AsTableIdentifier(tableName), colName, isExtended) => + DescribeColumnCommand(tableName, colName, isExtended) + + case DescribeColumnStatement( + CatalogObjectIdentifier(Some(catalog), ident), colName, isExtended) => + throw new AnalysisException("Describing columns is not supported for v2 tables.") + + case DescribeTableStatement( + AsTableIdentifier(tableName), partitionSpec, isExtended) => + DescribeTableCommand(tableName, partitionSpec, isExtended) + + case DescribeTableStatement( + CatalogObjectIdentifier(Some(catalog), ident), partitionSpec, isExtended) => + if (partitionSpec.nonEmpty) { + throw new AnalysisException("DESCRIBE TABLE... PARTITION is not supported for v2 tables.") + } + DescribeTable(catalog.asTableCatalog, ident, isExtended) + + case DropTableStatement(AsTableIdentifier(tableName), ifExists, purge) => + DropTableCommand(tableName, ifExists, isView = false, purge) + case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) => DropTable(catalog.asTableCatalog, ident, ifExists) 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 27d87960edb3..1da538be21e8 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 @@ -23,8 +23,9 @@ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} 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.{AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} import org.apache.spark.sql.sources @@ -199,6 +200,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { Nil } + case DescribeTable(catalog, ident, isExtended) => + DescribeTableExec(catalog, ident, isExtended) :: Nil + case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala new file mode 100644 index 000000000000..9d4d4ca13cc8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 scala.collection.mutable.ArrayBuffer + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRowWithSchema} +import org.apache.spark.sql.catalyst.plans.DescribeTableSchemas +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.sources.v2.Table + +case class DescribeTableExec( + catalog: TableCatalog, + ident: Identifier, + isExtended: Boolean) extends LeafExecNode { + + private val EMPTY_ROW = toCatalystRow("", "", "") + + override def output: Seq[AttributeReference] = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES + + override protected def doExecute(): RDD[InternalRow] = { + val rows = new ArrayBuffer[InternalRow]() + if (catalog.tableExists(ident)) { + val table = catalog.loadTable(ident) + addSchema(rows, table) + + if (isExtended) { + addPartitioning(rows, table) + addProperties(rows, table) + } + + } else { + rows += toCatalystRow(s"Table $ident does not exist.", "", "") + } + sparkContext.parallelize(rows) + } + + private def addSchema(rows: ArrayBuffer[InternalRow], table: Table): Unit = { + rows ++= table.schema.map{ column => + toCatalystRow( + column.name, column.dataType.simpleString, column.getComment().getOrElse("")) + } + } + + private def addPartitioning(rows: ArrayBuffer[InternalRow], table: Table): Unit = { + rows += EMPTY_ROW + rows += toCatalystRow(" Partitioning", "", "") + rows += toCatalystRow("--------------", "", "") + if (table.partitioning.isEmpty) { + rows += toCatalystRow("Not partitioned", "", "") + } else { + rows ++= table.partitioning.zipWithIndex.map { + case (transform, index) => toCatalystRow(s"Part $index", transform.describe(), "") + } + } + } + + private def addProperties(rows: ArrayBuffer[InternalRow], table: Table): Unit = { + rows += EMPTY_ROW + rows += toCatalystRow(" Table Property", " Value", "") + rows += toCatalystRow("----------------", "-------", "") + rows ++= table.properties.asScala.toList.sortBy(_._1).map { + case (key, value) => toCatalystRow(key, value, "") + } + } + + private def toCatalystRow(strs: String*): InternalRow = { + val encoder = RowEncoder(DescribeTableSchemas.DESCRIBE_TABLE_SCHEMA).resolveAndBind() + encoder.toRow( + new GenericRowWithSchema(strs.toArray, DescribeTableSchemas.DESCRIBE_TABLE_SCHEMA)) + } +} 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 96345e22dbd5..f5f156273f1e 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 @@ -21,12 +21,12 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalog.v2.Identifier import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{LongType, StringType, StructType} +import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { @@ -66,6 +66,50 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), Seq.empty) } + test("DescribeTable using v2 catalog") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string)" + + " USING foo" + + " PARTITIONED BY (id)") + val descriptionDf = spark.sql("DESCRIBE TABLE testcat.table_name") + assert(descriptionDf.schema === StructType( + Seq( + StructField("col_name", StringType, nullable = false), + StructField("data_type", StringType, nullable = false), + StructField("comment", StringType)))) + val description = descriptionDf.collect() + assert(description === Seq( + Row("id", "bigint", ""), + Row("data", "string", ""))) + } + + test("DescribeTable extended using v2 catalog") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string)" + + " USING foo" + + " PARTITIONED BY (id)" + + " TBLPROPERTIES ('bar'='baz')") + val descriptionDf = spark.sql("DESCRIBE TABLE EXTENDED testcat.table_name") + assert(descriptionDf.schema === StructType( + Seq( + StructField("col_name", StringType, nullable = false), + StructField("data_type", StringType, nullable = false), + StructField("comment", StringType)))) + assert(descriptionDf.collect() + .map(_.toSeq) + .map(_.toArray.map(_.toString.trim)) === Array( + Array("id", "bigint", ""), + Array("data", "string", ""), + Array("", "", ""), + Array("Partitioning", "", ""), + Array("--------------", "", ""), + Array("Part 0", "id", ""), + Array("", "", ""), + Array("Table Property", "Value", ""), + Array("----------------", "-------", ""), + Array("bar", "baz", ""), + Array("provider", "foo", ""))) + + } + test("CreateTable: use v2 plan because provider is v2") { spark.sql(s"CREATE TABLE table_name (id bigint, data string) USING $orc2") 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 4e9f961016de..35de96f7b142 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 @@ -71,12 +71,7 @@ class TestInMemoryTableCatalog extends TableCatalog { throw new TableAlreadyExistsException(ident) } - if (partitions.nonEmpty) { - throw new UnsupportedOperationException( - s"Catalog $name: Partitioned tables are not supported") - } - - val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties) + val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions) tables.put(ident, table) @@ -88,7 +83,8 @@ class TestInMemoryTableCatalog extends TableCatalog { case Some(table) => val properties = CatalogV2Util.applyPropertiesChanges(table.properties, changes) val schema = CatalogV2Util.applySchemaChanges(table.schema, changes) - val newTable = new InMemoryTable(table.name, schema, properties, table.data) + val newTable = new InMemoryTable( + table.name, schema, properties, table.partitioning, table.data) tables.put(ident, newTable) @@ -111,15 +107,17 @@ class TestInMemoryTableCatalog extends TableCatalog { private class InMemoryTable( val name: String, val schema: StructType, - override val properties: util.Map[String, String]) + override val properties: util.Map[String, String], + override val partitioning: Array[Transform]) extends Table with SupportsRead with SupportsWrite { def this( name: String, schema: StructType, properties: util.Map[String, String], + partitioning: Array[Transform], data: Array[BufferedRows]) = { - this(name, schema, properties) + this(name, schema, properties, partitioning) replaceData(data) } From 596832b52915e331c89a06528ca72c51500d60f1 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 3 Jul 2019 13:39:34 -0700 Subject: [PATCH 02/15] Fix ignoring rigorous comparison for Hive --- .../apache/spark/sql/hive/execution/HiveComparisonTest.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index a4587abbf389..c0158f1947d9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -30,6 +30,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.SQLExecution @@ -376,6 +377,8 @@ abstract class HiveComparisonTest (!hiveQuery.logical.isInstanceOf[ShowFunctionsCommand]) && (!hiveQuery.logical.isInstanceOf[DescribeFunctionCommand]) && (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && + (!hiveQuery.logical.isInstanceOf[DescribeTableStatement]) && + (!hiveQuery.logical.isInstanceOf[DescribeColumnStatement]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive From bdae3012c9db94c5e364df3279734706da004b74 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 3 Jul 2019 16:28:56 -0700 Subject: [PATCH 03/15] More test fixes --- .../logical/sql/DescribeTableStatement.scala | 2 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 6 ++- .../sql/execution/SparkSqlParserSuite.scala | 37 ++++++++++--------- 3 files changed, 25 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala index fc7ffef0a799..070b1416711e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala @@ -24,6 +24,6 @@ import org.apache.spark.sql.catalyst.plans.DescribeTableSchemas case class DescribeTableStatement( tableName: Seq[String], partitionSpec: TablePartitionSpec, - extended: Boolean) extends ParsedStatement { + isExtended: Boolean) extends ParsedStatement { override def output: Seq[Attribute] = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 1c8cf6403c6c..13b256cf4daf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -24,6 +24,7 @@ import scala.util.control.NonFatal import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} import org.apache.spark.sql.execution.HiveResult.hiveResultString @@ -338,7 +339,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // Returns true if the plan is supposed to be sorted. def isSorted(plan: LogicalPlan): Boolean = plan match { case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false - case _: DescribeCommandBase | _: DescribeColumnCommand => true + case _: DescribeCommandBase + | _: DescribeColumnCommand + | _: DescribeTableStatement + | _: DescribeColumnStatement => true case PhysicalOperation(_, _, Sort(_, true, _)) => true case _ => plan.children.iterator.exists(isSorted) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index df80311c0ce2..9f84d64b4e06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, Un import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} +import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} @@ -212,13 +213,13 @@ class SparkSqlParserSuite extends AnalysisTest { test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { assertEqual("describe t", - DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = false)) + DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) assertEqual("describe table t", - DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = false)) + DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) assertEqual("describe table extended t", - DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = true)) + DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) assertEqual("describe table formatted t", - DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = true)) + DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) } test("describe query") { @@ -229,27 +230,27 @@ class SparkSqlParserSuite extends AnalysisTest { test("describe table column") { assertEqual("DESCRIBE t col", - DescribeColumnCommand( - TableIdentifier("t"), Seq("col"), isExtended = false)) + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = false)) assertEqual("DESCRIBE t `abc.xyz`", - DescribeColumnCommand( - TableIdentifier("t"), Seq("abc.xyz"), isExtended = false)) + DescribeColumnStatement( + Seq("t"), Seq("abc.xyz"), isExtended = false)) assertEqual("DESCRIBE t abc.xyz", - DescribeColumnCommand( - TableIdentifier("t"), Seq("abc", "xyz"), isExtended = false)) + DescribeColumnStatement( + Seq("t"), Seq("abc", "xyz"), isExtended = false)) assertEqual("DESCRIBE t `a.b`.`x.y`", - DescribeColumnCommand( - TableIdentifier("t"), Seq("a.b", "x.y"), isExtended = false)) + DescribeColumnStatement( + Seq("t"), Seq("a.b", "x.y"), isExtended = false)) assertEqual("DESCRIBE TABLE t col", - DescribeColumnCommand( - TableIdentifier("t"), Seq("col"), isExtended = false)) + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = false)) assertEqual("DESCRIBE TABLE EXTENDED t col", - DescribeColumnCommand( - TableIdentifier("t"), Seq("col"), isExtended = true)) + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = true)) assertEqual("DESCRIBE TABLE FORMATTED t col", - DescribeColumnCommand( - TableIdentifier("t"), Seq("col"), isExtended = true)) + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = true)) intercept("DESCRIBE TABLE t PARTITION (ds='1970-01-01') col", "DESC TABLE COLUMN for a specific partition is not supported") From 527cbc30e437053fe1d43159418d7f1f0c9ed40c Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 10 Jul 2019 12:54:54 -0700 Subject: [PATCH 04/15] Move parser rules to AstBuilder in Catalyst --- .../sql/catalyst/parser/AstBuilder.scala | 34 ++++++++++++++++++- .../spark/sql/execution/SparkSqlParser.scala | 33 ------------------ 2 files changed, 33 insertions(+), 34 deletions(-) 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 f08cb2a780fe..860bac1845ab 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, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -2449,4 +2449,36 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging visitMultipartIdentifier(ctx.multipartIdentifier), visitLocationSpec(ctx.locationSpec)) } + + /** + * Create a [[DescribeColumnStatement]] or [[DescribeTableStatement]] commands. + */ + override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) { + val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null + if (ctx.describeColName != null) { + if (ctx.partitionSpec != null) { + throw new ParseException("DESC TABLE COLUMN for a specific partition is not supported", ctx) + } else { + DescribeColumnStatement( + visitMultipartIdentifier(ctx.multipartIdentifier()), + ctx.describeColName.nameParts.asScala.map(_.getText), + isExtended) + } + } else { + val partitionSpec = if (ctx.partitionSpec != null) { + // According to the syntax, visitPartitionSpec returns `Map[String, Option[String]]`. + visitPartitionSpec(ctx.partitionSpec).map { + case (key, Some(value)) => key -> value + case (key, _) => + throw new ParseException(s"PARTITION specification is incomplete: `$key`", ctx) + } + } else { + Map.empty[String, String] + } + DescribeTableStatement( + visitMultipartIdentifier(ctx.multipartIdentifier()), + partitionSpec, + isExtended) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index a1b08613096a..c4edadba278f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} @@ -328,38 +327,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } } - /** - * Create a [[DescribeColumnCommand]] or [[DescribeTableCommand]] logical commands. - */ - override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) { - val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null - if (ctx.describeColName != null) { - if (ctx.partitionSpec != null) { - throw new ParseException("DESC TABLE COLUMN for a specific partition is not supported", ctx) - } else { - DescribeColumnStatement( - visitMultipartIdentifier(ctx.multipartIdentifier()), - ctx.describeColName.nameParts.asScala.map(_.getText), - isExtended) - } - } else { - val partitionSpec = if (ctx.partitionSpec != null) { - // According to the syntax, visitPartitionSpec returns `Map[String, Option[String]]`. - visitPartitionSpec(ctx.partitionSpec).map { - case (key, Some(value)) => key -> value - case (key, _) => - throw new ParseException(s"PARTITION specification is incomplete: `$key`", ctx) - } - } else { - Map.empty[String, String] - } - DescribeTableStatement( - visitMultipartIdentifier(ctx.multipartIdentifier()), - partitionSpec, - isExtended) - } - } - /** * Create a [[DescribeQueryCommand]] logical command. */ From 6e5f0ea566ba5ac5452de62dbd6040ddc8afd47e Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 10 Jul 2019 13:42:48 -0700 Subject: [PATCH 05/15] Reuse row encoder --- .../execution/datasources/v2/DescribeTableExec.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 9d4d4ca13cc8..c92bd9c72165 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -34,7 +34,7 @@ case class DescribeTableExec( ident: Identifier, isExtended: Boolean) extends LeafExecNode { - private val EMPTY_ROW = toCatalystRow("", "", "") + import DescribeTableExec._ override def output: Seq[AttributeReference] = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES @@ -83,10 +83,14 @@ case class DescribeTableExec( case (key, value) => toCatalystRow(key, value, "") } } +} + +private object DescribeTableExec { + private val ENCODER = RowEncoder(DescribeTableSchemas.DESCRIBE_TABLE_SCHEMA) + private val EMPTY_ROW = toCatalystRow("", "", "") private def toCatalystRow(strs: String*): InternalRow = { - val encoder = RowEncoder(DescribeTableSchemas.DESCRIBE_TABLE_SCHEMA).resolveAndBind() - encoder.toRow( + ENCODER.resolveAndBind().toRow( new GenericRowWithSchema(strs.toArray, DescribeTableSchemas.DESCRIBE_TABLE_SCHEMA)) } } From eb5c84394689d8c70d5859c7ad45d7bff23d9481 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 10 Jul 2019 13:44:31 -0700 Subject: [PATCH 06/15] Remove reduandant drop --- .../spark/sql/execution/datasources/DataSourceResolution.scala | 3 --- 1 file changed, 3 deletions(-) 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 36a8d2ac0734..423d18b3c7d8 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 @@ -104,9 +104,6 @@ case class DataSourceResolution( } DescribeTable(catalog.asTableCatalog, ident, isExtended) - case DropTableStatement(AsTableIdentifier(tableName), ifExists, purge) => - DropTableCommand(tableName, ifExists, isView = false, purge) - case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) => DropTable(catalog.asTableCatalog, ident, ifExists) From 25ea40be7e05132a0fb48cbe74ca3456345ab0d6 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 10 Jul 2019 15:47:44 -0700 Subject: [PATCH 07/15] Fix style, move tests --- .../sql/catalyst/parser/DDLParserSuite.scala | 44 ++++++++++++++++++- .../sql/execution/SparkSqlParserSuite.scala | 39 ---------------- 2 files changed, 43 insertions(+), 40 deletions(-) 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 d008b3c78fac..4c318077ced4 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,12 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, 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, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -593,4 +594,45 @@ class DDLParserSuite extends AnalysisTest { Seq(Seq("x"), Seq("y"), Seq("a", "b", "c")))) } } + + test("describe table column") { + comparePlans(parsePlan("DESCRIBE t col"), + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = false)) + comparePlans(parsePlan("DESCRIBE t `abc.xyz`"), + DescribeColumnStatement( + Seq("t"), Seq("abc.xyz"), isExtended = false)) + comparePlans(parsePlan("DESCRIBE t abc.xyz"), + DescribeColumnStatement( + Seq("t"), Seq("abc", "xyz"), isExtended = false)) + comparePlans(parsePlan("DESCRIBE t `a.b`.`x.y`"), + DescribeColumnStatement( + Seq("t"), Seq("a.b", "x.y"), isExtended = false)) + + comparePlans(parsePlan("DESCRIBE TABLE t col"), + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = false)) + comparePlans(parsePlan("DESCRIBE TABLE EXTENDED t col"), + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = true)) + comparePlans(parsePlan("DESCRIBE TABLE FORMATTED t col"), + DescribeColumnStatement( + Seq("t"), Seq("col"), isExtended = true)) + + val caught = intercept[AnalysisException]( + parsePlan("DESCRIBE TABLE t PARTITION (ds='1970-01-01') col")) + assert(caught.getMessage.contains( + "DESC TABLE COLUMN for a specific partition is not supported")) + } + + test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { + comparePlans(parsePlan("describe t"), + DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) + comparePlans(parsePlan("describe table t"), + DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) + comparePlans(parsePlan("describe table extended t"), + DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) + comparePlans(parsePlan("describe table formatted t"), + DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 9f84d64b4e06..b751fb7c5043 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -211,51 +211,12 @@ class SparkSqlParserSuite extends AnalysisTest { "no viable alternative at input") } - test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { - assertEqual("describe t", - DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) - assertEqual("describe table t", - DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) - assertEqual("describe table extended t", - DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) - assertEqual("describe table formatted t", - DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) - } - test("describe query") { val query = "SELECT * FROM t" assertEqual("DESCRIBE QUERY " + query, DescribeQueryCommand(query, parser.parsePlan(query))) assertEqual("DESCRIBE " + query, DescribeQueryCommand(query, parser.parsePlan(query))) } - test("describe table column") { - assertEqual("DESCRIBE t col", - DescribeColumnStatement( - Seq("t"), Seq("col"), isExtended = false)) - assertEqual("DESCRIBE t `abc.xyz`", - DescribeColumnStatement( - Seq("t"), Seq("abc.xyz"), isExtended = false)) - assertEqual("DESCRIBE t abc.xyz", - DescribeColumnStatement( - Seq("t"), Seq("abc", "xyz"), isExtended = false)) - assertEqual("DESCRIBE t `a.b`.`x.y`", - DescribeColumnStatement( - Seq("t"), Seq("a.b", "x.y"), isExtended = false)) - - assertEqual("DESCRIBE TABLE t col", - DescribeColumnStatement( - Seq("t"), Seq("col"), isExtended = false)) - assertEqual("DESCRIBE TABLE EXTENDED t col", - DescribeColumnStatement( - Seq("t"), Seq("col"), isExtended = true)) - assertEqual("DESCRIBE TABLE FORMATTED t col", - DescribeColumnStatement( - Seq("t"), Seq("col"), isExtended = true)) - - intercept("DESCRIBE TABLE t PARTITION (ds='1970-01-01') col", - "DESC TABLE COLUMN for a specific partition is not supported") - } - test("analyze table statistics") { assertEqual("analyze table t compute statistics", AnalyzeTableCommand(TableIdentifier("t"), noscan = false)) From 4d32701d97afe51fafbb4896d7176a35cda53660 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 11 Jul 2019 12:51:08 -0700 Subject: [PATCH 08/15] Update describe test output --- sql/core/src/test/resources/sql-tests/results/describe.sql.out | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 46d9ec30a8a7..f58bdb5446b6 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -539,7 +539,7 @@ EXPLAIN EXTENDED DESC t struct -- !query 34 output == Parsed Logical Plan == -DescribeTableCommand `t`, false +'DescribeTableStatement [t], false == Analyzed Logical Plan == col_name: string, data_type: string, comment: string From da202b1212d94a7228bf4e34c5c3a81796feed89 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 23 Jul 2019 14:00:29 -0700 Subject: [PATCH 09/15] Merge from master --- core/pom.xml | 2 +- .../spark/ExecutorAllocationManager.scala | 16 +- .../scala/org/apache/spark/SparkContext.scala | 54 +- .../apache/spark/api/python/PythonUtils.scala | 4 + .../apache/spark/api/python/SerDeUtil.scala | 3 - .../HadoopDelegationTokenManager.scala | 4 +- .../spark/internal/config/package.scala | 19 + .../apache/spark/scheduler/StageInfo.scala | 10 +- .../scheduler/dynalloc/ExecutorMonitor.scala | 244 +- .../org/apache/spark/ui/jobs/StagePage.scala | 24 +- .../ExecutorAllocationManagerSuite.scala | 2 +- .../org/apache/spark/SparkConfSuite.scala | 13 + .../org/apache/spark/SparkContextSuite.scala | 11 + .../dynalloc/ExecutorMonitorSuite.scala | 152 +- .../apache/spark/util/FileAppenderSuite.scala | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- dev/deps/spark-deps-hadoop-3.2 | 2 +- dev/merge_spark_pr.py | 2 +- docs/configuration.md | 20 + docs/running-on-yarn.md | 14 +- docs/sql-migration-guide-upgrade.md | 6 + .../structured-streaming-kafka-integration.md | 10 + .../structured-streaming-programming-guide.md | 6 +- .../org/apache/spark/sql/avro/AvroSuite.scala | 6 +- .../jdbc/MsSqlServerIntegrationSuite.scala | 12 +- .../sql/jdbc/OracleIntegrationSuite.scala | 4 +- .../sql/jdbc/PostgresIntegrationSuite.scala | 13 + .../kafka010/KafkaContinuousSourceSuite.scala | 2 +- .../KafkaDontFailOnDataLossSuite.scala | 2 +- .../kafka010/KafkaMicroBatchSourceSuite.scala | 2 +- .../scala/org/apache/spark/ml/Predictor.scala | 4 +- .../spark/ml/classification/Classifier.scala | 6 +- .../spark/ml/classification/OneVsRest.scala | 18 +- .../ProbabilisticClassifier.scala | 4 +- .../spark/ml/clustering/GaussianMixture.scala | 30 +- .../org/apache/spark/ml/clustering/LDA.scala | 17 +- .../spark/ml/feature/StandardScaler.scala | 2 +- .../ml/regression/AFTSurvivalRegression.scala | 27 +- .../ml/regression/DecisionTreeRegressor.scala | 26 +- .../GeneralizedLinearRegression.scala | 25 +- .../mllib/api/python/PythonMLLibAPI.scala | 4 - .../spark/mllib/feature/StandardScaler.scala | 4 +- .../evaluation/MultilabelMetricsSuite.scala | 37 +- .../mllib/fpm/AssociationRulesSuite.scala | 5 +- .../spark/mllib/fpm/FPGrowthSuite.scala | 3 +- .../distributed/IndexedRowMatrixSuite.scala | 3 +- .../random/RandomDataGeneratorSuite.scala | 6 +- .../spark/mllib/random/RandomRDDsSuite.scala | 11 +- .../spark/mllib/stat/CorrelationSuite.scala | 15 +- .../spark/mllib/stat/KernelDensitySuite.scala | 13 +- .../spark/mllib/tree/EnsembleTestHelper.scala | 5 +- pom.xml | 2 +- project/MimaExcludes.scala | 6 +- python/pyspark/ml/classification.py | 157 +- python/pyspark/ml/param/shared.py | 36 - python/pyspark/ml/regression.py | 180 +- python/pyspark/rdd.py | 2 +- python/pyspark/sql/dataframe.py | 7 +- python/pyspark/sql/functions.py | 27 +- python/pyspark/sql/readwriter.py | 20 +- python/pyspark/sql/tests/test_dataframe.py | 2 + python/pyspark/sql/tests/test_functions.py | 10 + python/pyspark/sql/tests/test_readwriter.py | 21 + python/pyspark/sql/tests/test_serde.py | 4 - python/pyspark/sql/utils.py | 13 + .../org/apache/spark/deploy/yarn/Client.scala | 14 +- .../deploy/yarn/ResourceRequestHelper.scala | 46 +- .../spark/deploy/yarn/YarnAllocator.scala | 5 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 18 - .../spark/deploy/yarn/ClientSuite.scala | 19 +- .../yarn/ResourceRequestHelperSuite.scala | 77 +- .../deploy/yarn/YarnAllocatorSuite.scala | 13 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 12 + .../sql/catalog/v2/StagingTableCatalog.java | 142 + .../spark/sql/catalog/v2/TableChange.java | 19 +- .../sql/catalog/v2/utils/CatalogV2Util.scala | 51 +- .../spark/sql/sources/v2/StagedTable.java | 52 + .../scala/org/apache/spark/sql/Encoder.scala | 6 + .../sql/catalyst/CatalystTypeConverters.scala | 5 +- .../sql/catalyst/analysis/Analyzer.scala | 116 +- .../catalyst/analysis/CTESubstitution.scala | 136 + .../CannotReplaceMissingTableException.scala | 29 + .../sql/catalyst/analysis/CheckAnalysis.scala | 54 + .../catalyst/analysis/FunctionRegistry.scala | 4 + .../sql/catalyst/analysis/TypeCoercion.scala | 11 +- .../sql/catalyst/analysis/unresolved.scala | 5 +- .../spark/sql/catalyst/csv/CSVOptions.scala | 7 + .../catalyst/encoders/ExpressionEncoder.scala | 2 + .../expressions/collectionOperations.scala | 22 +- .../expressions/datetimeExpressions.scala | 64 +- .../expressions/mathExpressions.scala | 72 + .../sql/catalyst/expressions/predicates.scala | 4 + .../optimizer/NestedColumnAliasing.scala | 19 +- .../sql/catalyst/optimizer/Optimizer.scala | 20 +- .../sql/catalyst/parser/AstBuilder.scala | 78 + .../sql/catalyst/planning/patterns.scala | 16 +- .../plans/logical/basicLogicalOperators.scala | 78 +- .../logical/sql/ReplaceTableStatement.scala | 60 + .../spark/sql/catalyst/trees/TreeNode.scala | 13 +- .../sql/catalyst/util/DateTimeUtils.scala | 65 +- .../apache/spark/sql/internal/SQLConf.scala | 44 + .../sql/types/CalendarIntervalType.scala | 2 + .../org/apache/spark/sql/types/Decimal.scala | 12 +- .../apache/spark/sql/types/StructType.scala | 48 +- .../ExpressionTypeCheckingSuite.scala | 4 +- .../catalyst/analysis/TypeCoercionSuite.scala | 37 +- .../encoders/ExpressionEncoderSuite.scala | 4 + .../expressions/DateExpressionsSuite.scala | 26 +- .../expressions/ExpressionEvalHelper.scala | 2 +- .../expressions/JsonExpressionsSuite.scala | 2 +- .../expressions/LiteralGenerator.scala | 37 +- .../expressions/MathExpressionsSuite.scala | 32 + .../catalyst/expressions/ScalaUDFSuite.scala | 25 +- .../optimizer/ColumnPruningSuite.scala | 79 +- .../optimizer/FilterPushdownSuite.scala | 34 +- .../optimizer/NestedColumnAliasingSuite.scala | 7 +- .../NormalizeFloatingPointNumbersSuite.scala | 15 +- .../sql/catalyst/parser/DDLParserSuite.scala | 553 +- .../spark/sql/catalyst/plans/PlanTest.scala | 15 +- .../catalyst/util/DateTimeUtilsSuite.scala | 15 +- sql/core/benchmarks/MiscBenchmark-results.txt | 7 + .../apache/spark/sql/streaming/Trigger.java | 11 +- .../spark/sql/KeyValueGroupedDataset.scala | 65 + .../apache/spark/sql/UDFRegistration.scala | 10 +- .../spark/sql/execution/QueryExecution.scala | 24 +- .../spark/sql/execution/SparkStrategies.scala | 6 +- .../adaptive/ReduceNumShufflePartitions.scala | 21 +- .../aggregate/HashAggregateExec.scala | 14 +- .../execution/columnar/InMemoryRelation.scala | 7 + .../columnar/InMemoryTableScanExec.scala | 3 +- .../sql/execution/command/SetCommand.scala | 2 +- .../datasources/DataSourceResolution.scala | 81 + .../SaveIntoDataSourceCommand.scala | 6 + .../datasources/parquet/ParquetFilters.scala | 2 +- .../datasources/v2/AlterTableExec.scala | 47 + .../datasources/v2/DataSourceV2Strategy.scala | 51 +- .../datasources/v2/ReplaceTableExec.scala | 91 + .../v2/WriteToDataSourceV2Exec.scala | 179 +- .../exchange/EnsureRequirements.scala | 78 +- .../exchange/ShuffleExchangeExec.scala | 3 +- .../python/BatchEvalPythonExec.scala | 4 - .../python/UserDefinedPythonFunction.scala | 2 +- .../streaming/MicroBatchExecution.scala | 4 +- .../execution/streaming/TriggerExecutor.scala | 7 +- .../sql/execution/streaming/Triggers.scala | 84 +- .../continuous/ContinuousExecution.scala | 4 +- .../ContinuousRateStreamSource.scala | 6 +- .../continuous/ContinuousTrigger.scala | 57 - .../org/apache/spark/sql/functions.scala | 10 +- .../spark/sql/jdbc/MsSqlServerDialect.scala | 7 +- .../spark/sql/jdbc/PostgresDialect.scala | 3 +- .../sql/streaming/DataStreamWriter.scala | 1 - .../spark/sql/streaming/ProcessingTime.scala | 133 - .../sql/streaming/StreamingQueryManager.scala | 2 +- .../resources/sql-tests/inputs/cte-legacy.sql | 115 + .../sql-tests/inputs/pgSQL/boolean.sql | 27 +- .../resources/sql-tests/inputs/pgSQL/case.sql | 4 - .../resources/sql-tests/inputs/pgSQL/date.sql | 12 +- .../sql-tests/inputs/pgSQL/float8.sql | 500 ++ .../resources/sql-tests/inputs/pgSQL/int2.sql | 6 +- .../resources/sql-tests/inputs/pgSQL/int4.sql | 4 +- .../resources/sql-tests/inputs/pgSQL/int8.sql | 6 +- .../sql-tests/inputs/pgSQL/numeric.sql | 1096 ++++ .../sql-tests/inputs/pgSQL/select.sql | 285 + .../inputs/pgSQL/select_distinct.sql | 86 + .../sql-tests/inputs/pgSQL/select_having.sql | 55 + .../inputs/pgSQL/select_implicit.sql | 160 + .../sql-tests/inputs/pgSQL/timestamp.sql | 247 + .../resources/sql-tests/inputs/pgSQL/with.sql | 1208 +++++ .../inputs/udf/pgSQL/udf-aggregates_part1.sql | 44 +- .../inputs/udf/pgSQL/udf-aggregates_part2.sql | 2 - .../sql-tests/inputs/udf/pgSQL/udf-case.sql | 15 +- .../sql-tests/inputs/udf/udf-cross-join.sql | 37 + .../sql-tests/inputs/udf/udf-except-all.sql | 162 + .../sql-tests/inputs/udf/udf-except.sql | 59 + .../inputs/udf/udf-group-analytics.sql | 64 + .../sql-tests/inputs/udf/udf-group-by.sql | 160 + .../sql-tests/inputs/udf/udf-having.sql | 2 - .../sql-tests/inputs/udf/udf-inline-table.sql | 54 + .../inputs/udf/udf-intersect-all.sql | 162 + .../inputs/udf/udf-join-empty-relation.sql | 35 + .../sql-tests/inputs/udf/udf-natural-join.sql | 2 - .../sql-tests/inputs/udf/udf-outer-join.sql | 45 + .../sql-tests/inputs/udf/udf-pivot.sql | 307 ++ .../inputs/udf/udf-special-values.sql | 8 + .../sql-tests/inputs/udf/udf-udaf.sql | 18 + .../sql-tests/results/ansi/interval.sql.out | 24 +- .../sql-tests/results/cte-legacy.sql.out | 208 + .../resources/sql-tests/results/cte.sql.out | 16 +- .../sql-tests/results/literals.sql.out | 6 +- .../sql-tests/results/pgSQL/boolean.sql.out | 26 +- .../sql-tests/results/pgSQL/case.sql.out | 204 +- .../sql-tests/results/pgSQL/date.sql.out | 46 +- .../sql-tests/results/pgSQL/float8.sql.out | 839 +++ .../sql-tests/results/pgSQL/int2.sql.out | 4 +- .../sql-tests/results/pgSQL/int4.sql.out | 34 +- .../sql-tests/results/pgSQL/int8.sql.out | 82 +- .../sql-tests/results/pgSQL/numeric.sql.out | 4832 +++++++++++++++++ .../sql-tests/results/pgSQL/select.sql.out | 543 ++ .../results/pgSQL/select_distinct.sql.out | 225 + .../results/pgSQL/select_having.sql.out | 187 + .../results/pgSQL/select_implicit.sql.out | 416 ++ .../sql-tests/results/pgSQL/timestamp.sql.out | 130 + .../sql-tests/results/pgSQL/with.sql.out | 471 ++ .../native/dateTimeOperations.sql.out | 54 +- .../native/windowFrameCoercion.sql.out | 6 +- .../udf/pgSQL/udf-aggregates_part1.sql.out | 150 +- .../udf/pgSQL/udf-aggregates_part2.sql.out | 12 +- .../results/udf/pgSQL/udf-case.sql.out | 212 +- .../sql-tests/results/udf/udf-count.sql.out | 8 +- .../results/udf/udf-cross-join.sql.out | 140 + .../results/udf/udf-except-all.sql.out | 346 ++ .../sql-tests/results/udf/udf-except.sql.out | 87 + .../results/udf/udf-group-analytics.sql.out | 394 ++ .../results/udf/udf-group-by.sql.out | 512 ++ .../sql-tests/results/udf/udf-having.sql.out | 8 +- .../results/udf/udf-inline-table.sql.out | 153 + .../results/udf/udf-inner-join.sql.out | 2 +- .../results/udf/udf-intersect-all.sql.out | 307 ++ .../udf/udf-join-empty-relation.sql.out | 194 + .../results/udf/udf-natural-join.sql.out | 2 +- .../results/udf/udf-outer-join.sql.out | 88 + .../sql-tests/results/udf/udf-pivot.sql.out | 460 ++ .../results/udf/udf-special-values.sql.out | 62 + .../sql-tests/results/udf/udf-udaf.sql.out | 72 + .../spark/sql/AggregateHashMapSuite.scala | 29 +- .../spark/sql/DataFrameAggregateSuite.scala | 7 - .../org/apache/spark/sql/DataFrameSuite.scala | 10 +- .../sql/DataFrameWindowFunctionsSuite.scala | 7 - .../org/apache/spark/sql/DatasetSuite.scala | 66 +- .../apache/spark/sql/DateFunctionsSuite.scala | 8 +- .../spark/sql/FileBasedDataSourceSuite.scala | 6 +- .../spark/sql/IntegratedUDFTestUtils.scala | 111 +- .../org/apache/spark/sql/JoinSuite.scala | 55 +- .../apache/spark/sql/JsonFunctionsSuite.scala | 4 +- .../spark/sql/ProcessingTimeSuite.scala | 7 +- .../apache/spark/sql/RuntimeConfigSuite.scala | 6 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 8 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 60 +- .../sql/SparkSessionExtensionSuite.scala | 9 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 9 + .../spark/sql/execution/PlannerSuite.scala | 26 + .../sql/execution/QueryExecutionSuite.scala | 55 +- .../ReduceNumShufflePartitionsSuite.scala | 15 +- .../spark/sql/execution/SQLViewSuite.scala | 7 +- .../execution/SQLWindowFunctionSuite.scala | 5 +- .../execution/WholeStageCodegenSuite.scala | 38 +- .../arrow/ArrowConvertersSuite.scala | 4 +- .../benchmark/AggregateBenchmark.scala | 20 +- .../BuiltInDataSourceWriteBenchmark.scala | 6 +- .../execution/benchmark/MiscBenchmark.scala | 15 + .../columnar/InMemoryColumnarQuerySuite.scala | 2 +- .../columnar/PartitionBatchPruningSuite.scala | 6 +- .../command/PlanResolutionSuite.scala | 3 +- .../datasources/FileSourceStrategySuite.scala | 4 +- .../execution/datasources/csv/CSVSuite.scala | 30 +- .../datasources/json/JsonSuite.scala | 4 +- .../parquet/ParquetFilterSuite.scala | 8 + .../datasources/parquet/ParquetIOSuite.scala | 4 +- .../ParquetPartitionDiscoverySuite.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 6 +- .../metric/SQLMetricsTestUtils.scala | 3 +- .../ProcessingTimeExecutorSuite.scala | 11 +- .../state/StateStoreCoordinatorSuite.scala | 3 +- .../streaming/state/StateStoreSuite.scala | 2 +- .../ui/SQLAppStatusListenerSuite.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 18 +- .../sql/sources/BucketedWriteSuite.scala | 2 +- .../sources/CreateTableAsSelectSuite.scala | 3 +- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 1017 +++- .../sources/v2/TestInMemoryTableCatalog.scala | 181 +- .../streaming/EventTimeWatermarkSuite.scala | 2 +- .../sql/streaming/FileStreamSourceSuite.scala | 2 +- .../spark/sql/streaming/StreamSuite.scala | 6 +- .../StreamingQueryListenersConfSuite.scala | 3 +- .../sql/streaming/StreamingQuerySuite.scala | 8 +- .../ContinuousAggregationSuite.scala | 5 +- .../continuous/ContinuousSuite.scala | 67 +- .../sources/StreamingDataSourceV2Suite.scala | 5 +- .../test/DataStreamReaderWriterSuite.scala | 2 +- .../sql/test/DataFrameReaderWriterSuite.scala | 2 +- .../hive/thriftserver/HiveThriftServer2.scala | 14 +- .../SparkExecuteStatementOperation.scala | 3 +- .../SparkGetColumnsOperation.scala | 9 +- .../SparkGetSchemasOperation.scala | 9 +- .../SparkGetTablesOperation.scala | 9 +- .../thriftserver/ui/ThriftServerPage.scala | 8 +- .../ui/ThriftServerSessionPage.scala | 8 +- .../HiveThriftServer2Suites.scala | 11 +- .../service/cli/thrift/ThriftCLIService.java | 2 +- .../service/cli/thrift/ThriftCLIService.java | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 9 +- .../sql/hive/execution/SaveAsHiveFile.scala | 12 +- .../HiveExternalCatalogVersionsSuite.scala | 23 +- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 30 +- .../execution/AggregationQuerySuite.scala | 2 +- .../sql/hive/orc/HiveOrcSourceSuite.scala | 6 +- .../apache/spark/sql/hive/test/TestHive.scala | 8 +- 298 files changed, 21606 insertions(+), 1908 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala create mode 100644 sql/core/src/test/resources/sql-tests/inputs/cte-legacy.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-cross-join.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-except-all.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-except.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-analytics.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-by.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-inline-table.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-intersect-all.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-pivot.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-special-values.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/udf/udf-udaf.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out diff --git a/core/pom.xml b/core/pom.xml index 8a872dea1de4..4446dbdb5ed0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -378,7 +378,7 @@ net.razorvine pyrolite - 4.23 + 4.30 net.razorvine diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index bceb26cfd4f8..5114cf70e3f2 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -94,6 +94,7 @@ private[spark] class ExecutorAllocationManager( client: ExecutorAllocationClient, listenerBus: LiveListenerBus, conf: SparkConf, + cleaner: Option[ContextCleaner] = None, clock: Clock = new SystemClock()) extends Logging { @@ -148,7 +149,7 @@ private[spark] class ExecutorAllocationManager( // Listener for Spark events that impact the allocation policy val listener = new ExecutorAllocationListener - val executorMonitor = new ExecutorMonitor(conf, client, clock) + val executorMonitor = new ExecutorMonitor(conf, client, listenerBus, clock) // Executor that handles the scheduling task. private val executor = @@ -194,11 +195,13 @@ private[spark] class ExecutorAllocationManager( throw new SparkException( s"s${DYN_ALLOCATION_SUSTAINED_SCHEDULER_BACKLOG_TIMEOUT.key} must be > 0!") } - // Require external shuffle service for dynamic allocation - // Otherwise, we may lose shuffle files when killing executors - if (!conf.get(config.SHUFFLE_SERVICE_ENABLED) && !testing) { - throw new SparkException("Dynamic allocation of executors requires the external " + - "shuffle service. You may enable this through spark.shuffle.service.enabled.") + if (!conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + if (conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING)) { + logWarning("Dynamic allocation without a shuffle service is an experimental feature.") + } else if (!testing) { + throw new SparkException("Dynamic allocation of executors requires the external " + + "shuffle service. You may enable this through spark.shuffle.service.enabled.") + } } if (executorAllocationRatio > 1.0 || executorAllocationRatio <= 0.0) { @@ -214,6 +217,7 @@ private[spark] class ExecutorAllocationManager( def start(): Unit = { listenerBus.addToManagementQueue(listener) listenerBus.addToManagementQueue(executorMonitor) + cleaner.foreach(_.attachListener(executorMonitor)) val scheduleTask = new Runnable() { override def run(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a0d7aa743223..75182b0c9008 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -553,14 +553,22 @@ class SparkContext(config: SparkConf) extends Logging { None } - // Optionally scale number of executors dynamically based on workload. Exposed for testing. + _cleaner = + if (_conf.get(CLEANER_REFERENCE_TRACKING)) { + Some(new ContextCleaner(this)) + } else { + None + } + _cleaner.foreach(_.start()) + val dynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(_conf) _executorAllocationManager = if (dynamicAllocationEnabled) { schedulerBackend match { case b: ExecutorAllocationClient => Some(new ExecutorAllocationManager( - schedulerBackend.asInstanceOf[ExecutorAllocationClient], listenerBus, _conf)) + schedulerBackend.asInstanceOf[ExecutorAllocationClient], listenerBus, _conf, + cleaner = cleaner)) case _ => None } @@ -569,14 +577,6 @@ class SparkContext(config: SparkConf) extends Logging { } _executorAllocationManager.foreach(_.start()) - _cleaner = - if (_conf.get(CLEANER_REFERENCE_TRACKING)) { - Some(new ContextCleaner(this)) - } else { - None - } - _cleaner.foreach(_.start()) - setupAndStartListenerBus() postEnvironmentUpdate() postApplicationStart() @@ -1791,7 +1791,7 @@ class SparkContext(config: SparkConf) extends Logging { * @note A path can be added only once. Subsequent additions of the same path are ignored. */ def addJar(path: String) { - def addJarFile(file: File): String = { + def addLocalJarFile(file: File): String = { try { if (!file.exists()) { throw new FileNotFoundException(s"Jar ${file.getAbsolutePath} not found") @@ -1808,12 +1808,36 @@ class SparkContext(config: SparkConf) extends Logging { } } + def checkRemoteJarFile(path: String): String = { + val hadoopPath = new Path(path) + val scheme = new URI(path).getScheme + if (!Array("http", "https", "ftp").contains(scheme)) { + try { + val fs = hadoopPath.getFileSystem(hadoopConfiguration) + if (!fs.exists(hadoopPath)) { + throw new FileNotFoundException(s"Jar ${path} not found") + } + if (fs.isDirectory(hadoopPath)) { + throw new IllegalArgumentException( + s"Directory ${path} is not allowed for addJar") + } + path + } catch { + case NonFatal(e) => + logError(s"Failed to add $path to Spark environment", e) + null + } + } else { + path + } + } + if (path == null) { logWarning("null specified as parameter to addJar") } else { val key = if (path.contains("\\")) { // For local paths with backslashes on Windows, URI throws an exception - addJarFile(new File(path)) + addLocalJarFile(new File(path)) } else { val uri = new URI(path) // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies @@ -1822,12 +1846,12 @@ class SparkContext(config: SparkConf) extends Logging { // A JAR file which exists only on the driver node case null => // SPARK-22585 path without schema is not url encoded - addJarFile(new File(uri.getRawPath)) + addLocalJarFile(new File(uri.getRawPath)) // A JAR file which exists only on the driver node - case "file" => addJarFile(new File(uri.getPath)) + case "file" => addLocalJarFile(new File(uri.getPath)) // A JAR file which exists locally on every worker node case "local" => "file:" + uri.getPath - case _ => path + case _ => checkRemoteJarFile(path) } } if (key != null) { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index eee6e4b28ac4..62d60475985b 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -81,4 +81,8 @@ private[spark] object PythonUtils { def isEncryptionEnabled(sc: JavaSparkContext): Boolean = { sc.conf.get(org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED) } + + def getBroadcastThreshold(sc: JavaSparkContext): Long = { + sc.conf.get(org.apache.spark.internal.config.BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD) + } } diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 9462dfd950ba..01e64b6972ae 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -186,9 +186,6 @@ private[spark] object SerDeUtil extends Logging { val unpickle = new Unpickler iter.flatMap { row => val obj = unpickle.loads(row) - // `Opcodes.MEMOIZE` of Protocol 4 (Python 3.4+) will store objects in internal map - // of `Unpickler`. This map is cleared when calling `Unpickler.close()`. - unpickle.close() if (batched) { obj match { case array: Array[Any] => array.toSeq diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index a33c2874d1a5..759d857d56e0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -263,12 +263,14 @@ private[spark] class HadoopDelegationTokenManager( val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) logInfo("Successfully logged into KDC.") ugi - } else { + } else if (!SparkHadoopUtil.get.isProxyUser(UserGroupInformation.getCurrentUser())) { logInfo(s"Attempting to load user's ticket cache.") val ccache = sparkConf.getenv("KRB5CCNAME") val user = Option(sparkConf.getenv("KRB5PRINCIPAL")).getOrElse( UserGroupInformation.getCurrentUser().getUserName()) UserGroupInformation.getUGIFromTicketCache(ccache, user) + } else { + UserGroupInformation.getCurrentUser() } } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 488886f1627f..f2b88fe00cdf 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -369,6 +369,17 @@ package object config { .checkValue(_ >= 0L, "Timeout must be >= 0.") .createWithDefault(60) + private[spark] val DYN_ALLOCATION_SHUFFLE_TRACKING = + ConfigBuilder("spark.dynamicAllocation.shuffleTracking.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val DYN_ALLOCATION_SHUFFLE_TIMEOUT = + ConfigBuilder("spark.dynamicAllocation.shuffleTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .checkValue(_ >= 0L, "Timeout must be >= 0.") + .createWithDefault(Long.MaxValue) + private[spark] val DYN_ALLOCATION_SCHEDULER_BACKLOG_TIMEOUT = ConfigBuilder("spark.dynamicAllocation.schedulerBacklogTimeout") .timeConf(TimeUnit.SECONDS).createWithDefault(1) @@ -1246,6 +1257,14 @@ package object config { "mechanisms to guarantee data won't be corrupted during broadcast") .booleanConf.createWithDefault(true) + private[spark] val BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD = + ConfigBuilder("spark.broadcast.UDFCompressionThreshold") + .doc("The threshold at which user-defined functions (UDFs) and Python RDD commands " + + "are compressed by broadcast in bytes unless otherwise specified") + .bytesConf(ByteUnit.BYTE) + .checkValue(v => v >= 0, "The threshold should be non-negative.") + .createWithDefault(1L * 1024 * 1024) + private[spark] val RDD_COMPRESS = ConfigBuilder("spark.rdd.compress") .doc("Whether to compress serialized RDD partitions " + "(e.g. for StorageLevel.MEMORY_ONLY_SER in Scala " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 33a68f24bd53..e3216151462b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -37,7 +37,8 @@ class StageInfo( val parentIds: Seq[Int], val details: String, val taskMetrics: TaskMetrics = null, - private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty) { + private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, + private[spark] val shuffleDepId: Option[Int] = None) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None /** Time when all tasks in the stage completed or when the stage was cancelled. */ @@ -90,6 +91,10 @@ private[spark] object StageInfo { ): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos + val shuffleDepId = stage match { + case sms: ShuffleMapStage => Option(sms.shuffleDep).map(_.shuffleId) + case _ => None + } new StageInfo( stage.id, attemptId, @@ -99,6 +104,7 @@ private[spark] object StageInfo { stage.parents.map(_.id), stage.details, taskMetrics, - taskLocalityPreferences) + taskLocalityPreferences, + shuffleDepId) } } 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 9aac4d2281ec..d0337b6e3496 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 @@ -36,14 +36,19 @@ import org.apache.spark.util.Clock private[spark] class ExecutorMonitor( conf: SparkConf, client: ExecutorAllocationClient, - clock: Clock) extends SparkListener with Logging { + listenerBus: LiveListenerBus, + clock: Clock) extends SparkListener with CleanerListener with Logging { private val idleTimeoutMs = TimeUnit.SECONDS.toMillis( conf.get(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT)) private val storageTimeoutMs = TimeUnit.SECONDS.toMillis( conf.get(DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT)) + private val shuffleTimeoutMs = conf.get(DYN_ALLOCATION_SHUFFLE_TIMEOUT) + private val fetchFromShuffleSvcEnabled = conf.get(SHUFFLE_SERVICE_ENABLED) && conf.get(SHUFFLE_SERVICE_FETCH_RDD_ENABLED) + private val shuffleTrackingEnabled = !conf.get(SHUFFLE_SERVICE_ENABLED) && + conf.get(DYN_ALLOCATION_SHUFFLE_TRACKING) private val executors = new ConcurrentHashMap[String, Tracker]() @@ -64,6 +69,26 @@ private[spark] class ExecutorMonitor( private val nextTimeout = new AtomicLong(Long.MaxValue) private var timedOutExecs = Seq.empty[String] + // Active job tracking. + // + // The following state is used when an external shuffle service is not in use, and allows Spark + // to scale down based on whether the shuffle data stored in executors is in use. + // + // The algorithm works as following: when jobs start, some state is kept that tracks which stages + // are part of that job, and which shuffle ID is attached to those stages. As tasks finish, the + // executor tracking code is updated to include the list of shuffles for which it's storing + // shuffle data. + // + // If executors hold shuffle data that is related to an active job, then the executor is + // considered to be in "shuffle busy" state; meaning that the executor is not allowed to be + // removed. If the executor has shuffle data but it doesn't relate to any active job, then it + // may be removed when idle, following the shuffle-specific timeout configuration. + // + // The following fields are not thread-safe and should be only used from the event thread. + private val shuffleToActiveJobs = new mutable.HashMap[Int, mutable.ArrayBuffer[Int]]() + private val stageToShuffleID = new mutable.HashMap[Int, Int]() + private val jobToStageIDs = new mutable.HashMap[Int, Seq[Int]]() + def reset(): Unit = { executors.clear() nextTimeout.set(Long.MaxValue) @@ -85,7 +110,7 @@ private[spark] class ExecutorMonitor( var newNextTimeout = Long.MaxValue timedOutExecs = executors.asScala - .filter { case (_, exec) => !exec.pendingRemoval } + .filter { case (_, exec) => !exec.pendingRemoval && !exec.hasActiveShuffle } .filter { case (_, exec) => val deadline = exec.timeoutAt if (deadline > now) { @@ -124,6 +149,109 @@ private[spark] class ExecutorMonitor( def pendingRemovalCount: Int = executors.asScala.count { case (_, exec) => exec.pendingRemoval } + override def onJobStart(event: SparkListenerJobStart): Unit = { + if (!shuffleTrackingEnabled) { + return + } + + val shuffleStages = event.stageInfos.flatMap { s => + s.shuffleDepId.toSeq.map { shuffleId => + s.stageId -> shuffleId + } + } + + var updateExecutors = false + shuffleStages.foreach { case (stageId, shuffle) => + val jobIDs = shuffleToActiveJobs.get(shuffle) match { + case Some(jobs) => + // If a shuffle is being re-used, we need to re-scan the executors and update their + // tracker with the information that the shuffle data they're storing is in use. + logDebug(s"Reusing shuffle $shuffle in job ${event.jobId}.") + updateExecutors = true + jobs + + case _ => + logDebug(s"Registered new shuffle $shuffle (from stage $stageId).") + val jobs = new mutable.ArrayBuffer[Int]() + shuffleToActiveJobs(shuffle) = jobs + jobs + } + jobIDs += event.jobId + } + + if (updateExecutors) { + val activeShuffleIds = shuffleStages.map(_._2).toSeq + var needTimeoutUpdate = false + val activatedExecs = new ExecutorIdCollector() + executors.asScala.foreach { case (id, exec) => + if (!exec.hasActiveShuffle) { + exec.updateActiveShuffles(activeShuffleIds) + if (exec.hasActiveShuffle) { + needTimeoutUpdate = true + activatedExecs.add(id) + } + } + } + + logDebug(s"Activated executors $activatedExecs due to shuffle data needed by new job" + + s"${event.jobId}.") + + if (needTimeoutUpdate) { + nextTimeout.set(Long.MinValue) + } + } + + stageToShuffleID ++= shuffleStages + jobToStageIDs(event.jobId) = shuffleStages.map(_._1).toSeq + } + + override def onJobEnd(event: SparkListenerJobEnd): Unit = { + if (!shuffleTrackingEnabled) { + return + } + + var updateExecutors = false + val activeShuffles = new mutable.ArrayBuffer[Int]() + shuffleToActiveJobs.foreach { case (shuffleId, jobs) => + jobs -= event.jobId + if (jobs.nonEmpty) { + activeShuffles += shuffleId + } else { + // If a shuffle went idle we need to update all executors to make sure they're correctly + // tracking active shuffles. + updateExecutors = true + } + } + + if (updateExecutors) { + if (log.isDebugEnabled()) { + if (activeShuffles.nonEmpty) { + logDebug( + s"Job ${event.jobId} ended, shuffles ${activeShuffles.mkString(",")} still active.") + } else { + logDebug(s"Job ${event.jobId} ended, no active shuffles remain.") + } + } + + val deactivatedExecs = new ExecutorIdCollector() + executors.asScala.foreach { case (id, exec) => + if (exec.hasActiveShuffle) { + exec.updateActiveShuffles(activeShuffles) + if (!exec.hasActiveShuffle) { + deactivatedExecs.add(id) + } + } + } + + logDebug(s"Executors $deactivatedExecs do not have active shuffle data after job " + + s"${event.jobId} finished.") + } + + jobToStageIDs.remove(event.jobId).foreach { stages => + stages.foreach { id => stageToShuffleID -= id } + } + } + override def onTaskStart(event: SparkListenerTaskStart): Unit = { val executorId = event.taskInfo.executorId // Guard against a late arriving task start event (SPARK-26927). @@ -137,6 +265,21 @@ private[spark] class ExecutorMonitor( val executorId = event.taskInfo.executorId val exec = executors.get(executorId) if (exec != null) { + // If the task succeeded and the stage generates shuffle data, record that this executor + // holds data for the shuffle. This code will track all executors that generate shuffle + // for the stage, even if speculative tasks generate duplicate shuffle data and end up + // being ignored by the map output tracker. + // + // This means that an executor may be marked as having shuffle data, and thus prevented + // from being removed, even though the data may not be used. + if (shuffleTrackingEnabled && event.reason == Success) { + stageToShuffleID.get(event.stageId).foreach { shuffleId => + exec.addShuffle(shuffleId) + } + } + + // Update the number of running tasks after checking for shuffle data, so that the shuffle + // information is up-to-date in case the executor is going idle. exec.updateRunningTasks(-1) } } @@ -171,7 +314,6 @@ private[spark] class ExecutorMonitor( // available. So don't count blocks that can be served by the external service. if (storageLevel.isValid && (!fetchFromShuffleSvcEnabled || !storageLevel.useDisk)) { val hadCachedBlocks = exec.cachedBlocks.nonEmpty - val blocks = exec.cachedBlocks.getOrElseUpdate(blockId.rddId, new mutable.BitSet(blockId.splitIndex)) blocks += blockId.splitIndex @@ -201,6 +343,25 @@ private[spark] class ExecutorMonitor( } } + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case ShuffleCleanedEvent(id) => cleanupShuffle(id) + case _ => + } + + override def rddCleaned(rddId: Int): Unit = { } + + override def shuffleCleaned(shuffleId: Int): Unit = { + // Because this is called in a completely separate thread, we post a custom event to the + // listener bus so that the internal state is safely updated. + listenerBus.post(ShuffleCleanedEvent(shuffleId)) + } + + override def broadcastCleaned(broadcastId: Long): Unit = { } + + override def accumCleaned(accId: Long): Unit = { } + + override def checkpointCleaned(rddId: Long): Unit = { } + // Visible for testing. private[dynalloc] def isExecutorIdle(id: String): Boolean = { Option(executors.get(id)).map(_.isIdle).getOrElse(throw new NoSuchElementException(id)) @@ -209,7 +370,7 @@ private[spark] class ExecutorMonitor( // Visible for testing private[dynalloc] def timedOutExecutors(when: Long): Seq[String] = { executors.asScala.flatMap { case (id, tracker) => - if (tracker.timeoutAt <= when) Some(id) else None + if (tracker.isIdle && tracker.timeoutAt <= when) Some(id) else None }.toSeq } @@ -236,6 +397,14 @@ private[spark] class ExecutorMonitor( } } + private def cleanupShuffle(id: Int): Unit = { + logDebug(s"Cleaning up state related to shuffle $id.") + shuffleToActiveJobs -= id + executors.asScala.foreach { case (_, exec) => + exec.removeShuffle(id) + } + } + private class Tracker { @volatile var timeoutAt: Long = Long.MaxValue @@ -244,6 +413,7 @@ private[spark] class ExecutorMonitor( @volatile var timedOut: Boolean = false var pendingRemoval: Boolean = false + var hasActiveShuffle: Boolean = false private var idleStart: Long = -1 private var runningTasks: Int = 0 @@ -252,8 +422,11 @@ private[spark] class ExecutorMonitor( // This should only be used in the event thread. val cachedBlocks = new mutable.HashMap[Int, mutable.BitSet]() - // For testing. - def isIdle: Boolean = idleStart >= 0 + // The set of shuffles for which shuffle data is held by the executor. + // This should only be used in the event thread. + private val shuffleIds = if (shuffleTrackingEnabled) new mutable.HashSet[Int]() else null + + def isIdle: Boolean = idleStart >= 0 && !hasActiveShuffle def updateRunningTasks(delta: Int): Unit = { runningTasks = math.max(0, runningTasks + delta) @@ -264,7 +437,19 @@ private[spark] class ExecutorMonitor( def updateTimeout(): Unit = { val oldDeadline = timeoutAt val newDeadline = if (idleStart >= 0) { - idleStart + (if (cachedBlocks.nonEmpty) storageTimeoutMs else idleTimeoutMs) + val timeout = if (cachedBlocks.nonEmpty || (shuffleIds != null && shuffleIds.nonEmpty)) { + val _cacheTimeout = if (cachedBlocks.nonEmpty) storageTimeoutMs else Long.MaxValue + val _shuffleTimeout = if (shuffleIds != null && shuffleIds.nonEmpty) { + shuffleTimeoutMs + } else { + Long.MaxValue + } + math.min(_cacheTimeout, _shuffleTimeout) + } else { + idleTimeoutMs + } + val deadline = idleStart + timeout + if (deadline >= 0) deadline else Long.MaxValue } else { Long.MaxValue } @@ -279,5 +464,50 @@ private[spark] class ExecutorMonitor( updateNextTimeout(newDeadline) } } + + def addShuffle(id: Int): Unit = { + if (shuffleIds.add(id)) { + hasActiveShuffle = true + } + } + + def removeShuffle(id: Int): Unit = { + if (shuffleIds.remove(id) && shuffleIds.isEmpty) { + hasActiveShuffle = false + if (isIdle) { + updateTimeout() + } + } + } + + def updateActiveShuffles(ids: Iterable[Int]): Unit = { + val hadActiveShuffle = hasActiveShuffle + hasActiveShuffle = ids.exists(shuffleIds.contains) + if (hadActiveShuffle && isIdle) { + updateTimeout() + } + } + } + + 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 838fc82d2ee3..54f0f8e22679 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/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 3ba33e358ef0..191b516661e4 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -1008,7 +1008,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private def createManager( conf: SparkConf, clock: Clock = new SystemClock()): ExecutorAllocationManager = { - val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock) + val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock = clock) managers += manager manager.start() manager diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 6be1fedc123d..202b85dcf569 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -389,6 +389,19 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst """.stripMargin.trim) } + test("SPARK-28355: Use Spark conf for threshold at which UDFs are compressed by broadcast") { + val conf = new SparkConf() + + // Check the default value + assert(conf.get(BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD) === 1L * 1024 * 1024) + + // Set the conf + conf.set(BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD, 1L * 1024) + + // Verify that it has been set properly + assert(conf.get(BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD) === 1L * 1024) + } + val defaultIllegalValue = "SomeIllegalValue" val illegalValueTests : Map[String, (SparkConf, String) => Any] = Map( "getTimeAsSeconds" -> (_.getTimeAsSeconds(_)), diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 628ac60fa767..fed3ae35ee0e 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -170,6 +170,17 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } + test("add FS jar files not exists") { + try { + val jarPath = "hdfs:///no/path/to/TestUDTF.jar" + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addJar(jarPath) + assert(sc.listJars().forall(!_.contains("TestUDTF.jar"))) + } finally { + sc.stop() + } + } + test("SPARK-17650: malformed url's throw exceptions before bricking Executors") { try { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) 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 8d1577e835d2..6a25754fcbe5 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 @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.{mock, when} +import org.mockito.Mockito.{doAnswer, mock, when} import org.apache.spark._ import org.apache.spark.internal.config._ @@ -34,10 +34,13 @@ class ExecutorMonitorSuite extends SparkFunSuite { private val idleTimeoutMs = TimeUnit.SECONDS.toMillis(60L) private val storageTimeoutMs = TimeUnit.SECONDS.toMillis(120L) + private val shuffleTimeoutMs = TimeUnit.SECONDS.toMillis(240L) private val conf = new SparkConf() .set(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, "60s") .set(DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT.key, "120s") + .set(DYN_ALLOCATION_SHUFFLE_TIMEOUT.key, "240s") + .set(SHUFFLE_SERVICE_ENABLED, true) private var monitor: ExecutorMonitor = _ private var client: ExecutorAllocationClient = _ @@ -55,7 +58,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { when(client.isExecutorActive(any())).thenAnswer { invocation => knownExecs.contains(invocation.getArguments()(0).asInstanceOf[String]) } - monitor = new ExecutorMonitor(conf, client, clock) + monitor = new ExecutorMonitor(conf, client, null, clock) } test("basic executor timeout") { @@ -205,7 +208,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) conf.set(SHUFFLE_SERVICE_ENABLED, true).set(SHUFFLE_SERVICE_FETCH_RDD_ENABLED, true) - monitor = new ExecutorMonitor(conf, client, clock) + monitor = new ExecutorMonitor(conf, client, null, clock) monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) monitor.onBlockUpdated(rddUpdate(1, 0, "1", level = StorageLevel.MEMORY_ONLY)) @@ -259,8 +262,139 @@ class ExecutorMonitorSuite extends SparkFunSuite { assert(monitor.timedOutExecutors().toSet === Set("2")) } + test("shuffle block tracking") { + val bus = mockListenerBus() + conf.set(DYN_ALLOCATION_SHUFFLE_TRACKING, true).set(SHUFFLE_SERVICE_ENABLED, false) + monitor = new ExecutorMonitor(conf, client, bus, clock) + + // 3 jobs: 2 and 3 share a shuffle, 1 has a separate shuffle. + val stage1 = stageInfo(1, shuffleId = 0) + val stage2 = stageInfo(2) + + val stage3 = stageInfo(3, shuffleId = 1) + val stage4 = stageInfo(4) + + val stage5 = stageInfo(5, shuffleId = 1) + val stage6 = stageInfo(6) + + // Start jobs 1 and 2. Finish a task on each, but don't finish the jobs. This should prevent the + // executor from going idle since there are active shuffles. + monitor.onJobStart(SparkListenerJobStart(1, clock.getTimeMillis(), Seq(stage1, stage2))) + monitor.onJobStart(SparkListenerJobStart(2, clock.getTimeMillis(), Seq(stage3, stage4))) + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + + // First a failed task, to make sure it does not count. + monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", TaskResultLost, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + + monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + monitor.onTaskStart(SparkListenerTaskStart(3, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(3, 0, "foo", Success, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + // Finish the jobs, now the executor should be idle, but with the shuffle timeout, since the + // shuffles are not active. + monitor.onJobEnd(SparkListenerJobEnd(1, clock.getTimeMillis(), JobSucceeded)) + assert(!monitor.isExecutorIdle("1")) + + monitor.onJobEnd(SparkListenerJobEnd(2, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline).isEmpty) + assert(monitor.timedOutExecutors(shuffleDeadline) === Seq("1")) + + // Start job 3. Since it shares a shuffle with job 2, the executor should not be considered + // idle anymore, even if no tasks are run. + monitor.onJobStart(SparkListenerJobStart(3, clock.getTimeMillis(), Seq(stage5, stage6))) + assert(!monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(shuffleDeadline).isEmpty) + + monitor.onJobEnd(SparkListenerJobEnd(3, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(shuffleDeadline) === Seq("1")) + + // Clean up the shuffles, executor now should now time out at the idle deadline. + monitor.shuffleCleaned(0) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + monitor.shuffleCleaned(1) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + } + + test("shuffle tracking with multiple executors and concurrent jobs") { + val bus = mockListenerBus() + conf.set(DYN_ALLOCATION_SHUFFLE_TRACKING, true).set(SHUFFLE_SERVICE_ENABLED, false) + monitor = new ExecutorMonitor(conf, client, bus, clock) + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", null)) + + // Two separate jobs with separate shuffles. The first job will only run tasks on + // executor 1, the second on executor 2. Ensures that jobs finishing don't affect + // executors that are active in other jobs. + + val stage1 = stageInfo(1, shuffleId = 0) + val stage2 = stageInfo(2) + monitor.onJobStart(SparkListenerJobStart(1, clock.getTimeMillis(), Seq(stage1, stage2))) + + val stage3 = stageInfo(3, shuffleId = 1) + val stage4 = stageInfo(4) + monitor.onJobStart(SparkListenerJobStart(2, clock.getTimeMillis(), Seq(stage3, stage4))) + + monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("2")) + + monitor.onTaskStart(SparkListenerTaskStart(3, 0, taskInfo("2", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(3, 0, "foo", Success, taskInfo("2", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + monitor.onJobEnd(SparkListenerJobEnd(1, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.isExecutorIdle("1")) + assert(!monitor.isExecutorIdle("2")) + + monitor.onJobEnd(SparkListenerJobEnd(2, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.isExecutorIdle("2")) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + monitor.shuffleCleaned(0) + monitor.shuffleCleaned(1) + 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 + + private def stageInfo(id: Int, shuffleId: Int = -1): StageInfo = { + new StageInfo(id, 0, s"stage$id", 1, Nil, Nil, "", + shuffleDepId = if (shuffleId >= 0) Some(shuffleId) else None) + } private def taskInfo( execId: String, @@ -286,4 +420,16 @@ class ExecutorMonitorSuite extends SparkFunSuite { RDDBlockId(rddId, splitIndex), level, 1L, 0L)) } + /** + * Mock the listener bus *only* for the functionality needed by the shuffle tracking code. + * Any other event sent through the mock bus will fail. + */ + private def mockListenerBus(): LiveListenerBus = { + val bus = mock(classOf[LiveListenerBus]) + doAnswer { invocation => + monitor.onOtherEvent(invocation.getArguments()(0).asInstanceOf[SparkListenerEvent]) + }.when(bus).post(any()) + bus + } + } diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 242163931f7a..f5f93ece660b 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -128,7 +128,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { val files = testRolling(appender, testOutputStream, textToAppend, 0, isCompressed = true) files.foreach { file => logInfo(file.toString + ": " + file.length + " bytes") - assert(file.length < rolloverSize) + assert(file.length <= rolloverSize) } } diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 2f660ccfd92f..79158bb6edfe 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -170,7 +170,7 @@ parquet-hadoop-bundle-1.6.0.jar parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar -pyrolite-4.23.jar +pyrolite-4.30.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar scala-parser-combinators_2.12-1.1.0.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index e1e114fa08ca..5e03a5951db0 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -189,7 +189,7 @@ parquet-hadoop-1.10.1.jar parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar -pyrolite-4.23.jar +pyrolite-4.30.jar re2j-1.1.jar scala-compiler-2.12.8.jar scala-library-2.12.8.jar diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 593e34983aa0..e51e9560b59d 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -353,7 +353,7 @@ def choose_jira_assignee(issue, asf_jira): except: # assume it's a user id, and try to assign (might fail, we just prompt again) assignee = asf_jira.user(raw_assignee) - asf_jira.assign_issue(issue.key, assignee.key) + asf_jira.assign_issue(issue.key, assignee.name) return assignee except KeyboardInterrupt: raise diff --git a/docs/configuration.md b/docs/configuration.md index 06b040866f13..108862416f8d 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2114,6 +2114,26 @@ Apart from these, the following properties are also available, and may be useful description. + + spark.dynamicAllocation.shuffleTracking.enabled + false + + Experimental. Enables shuffle file tracking for executors, which allows dynamic allocation + without the need for an external shuffle service. This option will try to keep alive executors + that are storing shuffle data for active jobs. + + + + spark.dynamicAllocation.shuffleTimeout + infinity + + When shuffle tracking is enabled, controls the timeout for executors that are holding shuffle + data. The default value means that Spark will rely on the shuffles being garbage collected to be + able to release executors. If for some reason garbage collection is not cleaning up shuffles + quickly enough, this option can be used to control when to time out executors even when they are + storing shuffle data. + + ### Thread Configurations diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index dc93e9cea5bc..9d9b253a5c8e 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -142,20 +142,20 @@ To use a custom metrics.properties for the application master and executors, upd - spark.yarn.am.resource.{resource-type} + spark.yarn.am.resource.{resource-type}.amount (none) Amount of resource to use for the YARN Application Master in client mode. - In cluster mode, use spark.yarn.driver.resource.<resource-type> instead. + In cluster mode, use spark.yarn.driver.resource.<resource-type>.amount instead. Please note that this feature can be used only with YARN 3.0+ For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html

Example: - To request GPU resources from YARN, use: spark.yarn.am.resource.yarn.io/gpu + To request GPU resources from YARN, use: spark.yarn.am.resource.yarn.io/gpu.amount - spark.yarn.driver.resource.{resource-type} + spark.yarn.driver.resource.{resource-type}.amount (none) Amount of resource to use for the YARN Application Master in cluster mode. @@ -163,11 +163,11 @@ To use a custom metrics.properties for the application master and executors, upd For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html

Example: - To request GPU resources from YARN, use: spark.yarn.driver.resource.yarn.io/gpu + To request GPU resources from YARN, use: spark.yarn.driver.resource.yarn.io/gpu.amount - spark.yarn.executor.resource.{resource-type} + spark.yarn.executor.resource.{resource-type}.amount (none) Amount of resource to use per executor process. @@ -175,7 +175,7 @@ To use a custom metrics.properties for the application master and executors, upd For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html

Example: - To request GPU resources from YARN, use: spark.yarn.executor.resource.yarn.io/gpu + To request GPU resources from YARN, use: spark.yarn.executor.resource.yarn.io/gpu.amount diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index d39bd933427f..e9d99b66353e 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -149,6 +149,12 @@ license: | - Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. + - 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 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 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index fe3c60040d0a..b0009e01703b 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 925e2cfe717c..821225753320 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 40bf3b1530fb..a7c9e3fb7d32 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 @@ -50,7 +50,7 @@ abstract class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUti override protected def beforeAll(): Unit = { super.beforeAll() - spark.conf.set("spark.sql.files.maxPartitionBytes", 1024) + spark.conf.set(SQLConf.FILES_MAX_PARTITION_BYTES.key, 1024) } def checkReloadMatchesSaved(originalFile: String, newFile: String): Unit = { @@ -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/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 82ce16c2b7e5..efd7ca74c796 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -120,24 +120,24 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types.length == 12) assert(types(0).equals("class java.lang.Boolean")) assert(types(1).equals("class java.lang.Integer")) - assert(types(2).equals("class java.lang.Integer")) + assert(types(2).equals("class java.lang.Short")) assert(types(3).equals("class java.lang.Integer")) assert(types(4).equals("class java.lang.Long")) assert(types(5).equals("class java.lang.Double")) - assert(types(6).equals("class java.lang.Double")) - assert(types(7).equals("class java.lang.Double")) + assert(types(6).equals("class java.lang.Float")) + assert(types(7).equals("class java.lang.Float")) assert(types(8).equals("class java.math.BigDecimal")) assert(types(9).equals("class java.math.BigDecimal")) assert(types(10).equals("class java.math.BigDecimal")) assert(types(11).equals("class java.math.BigDecimal")) assert(row.getBoolean(0) == false) assert(row.getInt(1) == 255) - assert(row.getInt(2) == 32767) + assert(row.getShort(2) == 32767) assert(row.getInt(3) == 2147483647) assert(row.getLong(4) == 9223372036854775807L) assert(row.getDouble(5) == 1.2345678901234512E14) // float = float(53) has 15-digits precision - assert(row.getDouble(6) == 1.23456788103168E14) // float(24) has 7-digits precision - assert(row.getDouble(7) == 1.23456788103168E14) // real = float(24) + assert(row.getFloat(6) == 1.23456788103168E14) // float(24) has 7-digits precision + assert(row.getFloat(7) == 1.23456788103168E14) // real = float(24) assert(row.getAs[BigDecimal](8).equals(new BigDecimal("123.00"))) assert(row.getAs[BigDecimal](9).equals(new BigDecimal("12345.12000"))) assert(row.getAs[BigDecimal](10).equals(new BigDecimal("922337203685477.5800"))) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 64b9837cc5fa..8cdc4a1806b2 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -376,8 +376,8 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo val e = intercept[org.apache.spark.SparkException] { spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", new Properties()).collect() } - assert(e.getMessage.contains( - "requirement failed: Decimal precision 39 exceeds max precision 38")) + assert(e.getCause().isInstanceOf[ArithmeticException]) + assert(e.getMessage.contains("Decimal precision 39 exceeds max precision 38")) // custom schema can read data val props = new Properties() 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 462f88ff14a8..89da9a1de6f7 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/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala index 9b3e78c84c34..76c25980fc33 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala @@ -21,7 +21,7 @@ import org.apache.kafka.clients.producer.ProducerRecord import org.apache.spark.sql.Dataset import org.apache.spark.sql.execution.datasources.v2.ContinuousScanExec -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger +import org.apache.spark.sql.execution.streaming.ContinuousTrigger import org.apache.spark.sql.streaming.Trigger // Run tests in KafkaSourceSuiteBase in continuous execution mode. diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala index e089e36eba5f..ba8340ea59c1 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala @@ -135,7 +135,7 @@ class KafkaDontFailOnDataLossSuite extends StreamTest with KafkaMissingOffsetsTe test("failOnDataLoss=false should not return duplicated records: microbatch v1") { withSQLConf( - "spark.sql.streaming.disabledV2MicroBatchReaders" -> + SQLConf.DISABLED_V2_STREAMING_MICROBATCH_READERS.key -> classOf[KafkaSourceProvider].getCanonicalName) { verifyMissingOffsetsDontCauseDuplicatedRecords(testStreamingQuery = true) { (df, table) => val query = df.writeStream.format("memory").queryName(table).start() diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 3d14ebe267c4..bb9b3696fe8f 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -1066,7 +1066,7 @@ class KafkaMicroBatchV1SourceSuite extends KafkaMicroBatchSourceSuiteBase { override def beforeAll(): Unit = { super.beforeAll() spark.conf.set( - "spark.sql.streaming.disabledV2MicroBatchReaders", + SQLConf.DISABLED_V2_STREAMING_MICROBATCH_READERS.key, classOf[KafkaSourceProvider].getCanonicalName) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index d8f3dfa87443..58815434cbda 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -204,8 +204,8 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, if ($(predictionCol).nonEmpty) { transformImpl(dataset) } else { - this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + - " since no output columns were set.") + this.logWarning(s"$uid: Predictor.transform() does nothing" + + " because no output columns were set.") dataset.toDF } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index e35e6ce7fdad..568cdd11a12a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkException -import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, VectorUDT} @@ -204,8 +204,8 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur } if (numColsOutput == 0) { - logWarning(s"$uid: ClassificationModel.transform() was called as NOOP" + - " since no output columns were set.") + logWarning(s"$uid: ClassificationModel.transform() does nothing" + + " because no output columns were set.") } outputData.toDF } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index e1fceb1fc96a..675315e3bb07 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -169,9 +169,9 @@ final class OneVsRestModel private[ml] ( // Check schema transformSchema(dataset.schema, logging = true) - if (getPredictionCol == "" && getRawPredictionCol == "") { - logWarning(s"$uid: OneVsRestModel.transform() was called as NOOP" + - " since no output columns were set.") + if (getPredictionCol.isEmpty && getRawPredictionCol.isEmpty) { + logWarning(s"$uid: OneVsRestModel.transform() does nothing" + + " because no output columns were set.") return dataset.toDF } @@ -218,7 +218,7 @@ final class OneVsRestModel private[ml] ( var predictionColNames = Seq.empty[String] var predictionColumns = Seq.empty[Column] - if (getRawPredictionCol != "") { + if (getRawPredictionCol.nonEmpty) { val numClass = models.length // output the RawPrediction as vector @@ -228,18 +228,18 @@ final class OneVsRestModel private[ml] ( Vectors.dense(predArray) } - predictionColNames = predictionColNames :+ getRawPredictionCol - predictionColumns = predictionColumns :+ rawPredictionUDF(col(accColName)) + predictionColNames :+= getRawPredictionCol + predictionColumns :+= rawPredictionUDF(col(accColName)) } - if (getPredictionCol != "") { + if (getPredictionCol.nonEmpty) { // output the index of the classifier with highest confidence as prediction val labelUDF = udf { (predictions: Map[Int, Double]) => predictions.maxBy(_._2)._1.toDouble } - predictionColNames = predictionColNames :+ getPredictionCol - predictionColumns = predictionColumns :+ labelUDF(col(accColName)) + predictionColNames :+= getPredictionCol + predictionColumns :+= labelUDF(col(accColName)) .as(getPredictionCol, labelMetadata) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 730fcab333e1..5046caa568d5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -147,8 +147,8 @@ abstract class ProbabilisticClassificationModel[ } if (numColsOutput == 0) { - this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + - " since no output columns were set.") + this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() does nothing" + + " because no output columns were set.") } outputData.toDF } diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index fb4698ab5564..9a51d2f18846 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -33,7 +33,7 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.udf import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.storage.StorageLevel @@ -110,11 +110,29 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val predUDF = udf((vector: Vector) => predict(vector)) - val probUDF = udf((vector: Vector) => predictProbability(vector)) - dataset - .withColumn($(predictionCol), predUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol))) - .withColumn($(probabilityCol), probUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol))) + + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + + if ($(predictionCol).nonEmpty) { + val predUDF = udf((vector: Vector) => predict(vector)) + predictionColNames :+= $(predictionCol) + predictionColumns :+= predUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)) + } + + if ($(probabilityCol).nonEmpty) { + val probUDF = udf((vector: Vector) => predictProbability(vector)) + predictionColNames :+= $(probabilityCol) + predictionColumns :+= probUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) + } else { + this.logWarning(s"$uid: GaussianMixtureModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() + } } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index aa8103701445..91201e7bd03f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -461,17 +461,10 @@ abstract class LDAModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - if ($(topicDistributionCol).nonEmpty) { - val func = getTopicDistributionMethod - val transformer = udf(func) - - dataset.withColumn($(topicDistributionCol), - transformer(DatasetUtils.columnToVector(dataset, getFeaturesCol))) - } else { - logWarning("LDAModel.transform was called without any output columns. Set an output column" + - " such as topicDistributionCol to produce results.") - dataset.toDF() - } + val func = getTopicDistributionMethod + val transformer = udf(func) + dataset.withColumn($(topicDistributionCol), + transformer(DatasetUtils.columnToVector(dataset, getFeaturesCol))) } /** @@ -490,7 +483,7 @@ abstract class LDAModel private[ml] ( Vectors.zeros(k) } else { val (ids: List[Int], cts: Array[Double]) = vector match { - case v: DenseVector => ((0 until v.size).toList, v.values) + case v: DenseVector => (List.range(0, v.size), v.values) case v: SparseVector => (v.indices.toList, v.values) case other => throw new UnsupportedOperationException( diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 17f2c17c9552..81cf2e1a4ff7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -169,7 +169,7 @@ class StandardScalerModel private[ml] ( case d: DenseVector => d.values.clone() case v: Vector => v.toArray } - val newValues = scaler.transfromWithMean(values) + val newValues = scaler.transformWithMean(values) Vectors.dense(newValues) } else if ($(withStd)) { vector: Vector => diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 067dfa43433e..1565782dd631 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -37,7 +37,7 @@ import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructType} import org.apache.spark.storage.StorageLevel @@ -355,13 +355,28 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val predictUDF = udf { features: Vector => predict(features) } - val predictQuantilesUDF = udf { features: Vector => predictQuantiles(features)} + + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + + if ($(predictionCol).nonEmpty) { + val predictUDF = udf { features: Vector => predict(features) } + predictionColNames :+= $(predictionCol) + predictionColumns :+= predictUDF(col($(featuresCol))) + } + if (hasQuantilesCol) { - dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) - .withColumn($(quantilesCol), predictQuantilesUDF(col($(featuresCol)))) + val predictQuantilesUDF = udf { features: Vector => predictQuantiles(features)} + predictionColNames :+= $(quantilesCol) + predictionColumns :+= predictQuantilesUDF(col($(featuresCol))) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) } else { - dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + this.logWarning(s"$uid: AFTSurvivalRegressionModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index f4f4e56a3578..6348289de516 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -34,7 +34,7 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType @@ -216,16 +216,28 @@ class DecisionTreeRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val predictUDF = udf { (features: Vector) => predict(features) } - val predictVarianceUDF = udf { (features: Vector) => predictVariance(features) } - var output = dataset.toDF() + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + if ($(predictionCol).nonEmpty) { - output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + val predictUDF = udf { (features: Vector) => predict(features) } + predictionColNames :+= $(predictionCol) + predictionColumns :+= predictUDF(col($(featuresCol))) } + if (isDefined(varianceCol) && $(varianceCol).nonEmpty) { - output = output.withColumn($(varianceCol), predictVarianceUDF(col($(featuresCol)))) + val predictVarianceUDF = udf { (features: Vector) => predictVariance(features) } + predictionColNames :+= $(varianceCol) + predictionColumns :+= predictVarianceUDF(col($(featuresCol))) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) + } else { + this.logWarning(s"$uid: DecisionTreeRegressionModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() } - output } @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 885b13bf8dac..b1a8f95c1261 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -1041,18 +1041,31 @@ class GeneralizedLinearRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } - val predictLinkUDF = udf { (features: Vector, offset: Double) => predictLink(features, offset) } + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) - var output = dataset + if ($(predictionCol).nonEmpty) { - output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)), offset)) + val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } + predictionColNames :+= $(predictionCol) + predictionColumns :+= predictUDF(col($(featuresCol)), offset) } + if (hasLinkPredictionCol) { - output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)), offset)) + val predictLinkUDF = + udf { (features: Vector, offset: Double) => predictLink(features, offset) } + predictionColNames :+= $(linkPredictionCol) + predictionColumns :+= predictLinkUDF(col($(featuresCol)), offset) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) + } else { + this.logWarning(s"$uid: GeneralizedLinearRegressionModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() } - output.toDF() } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 4c478a5477c0..4617073f9dec 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -1357,10 +1357,6 @@ private[spark] abstract class SerDeBase { val unpickle = new Unpickler iter.flatMap { row => val obj = unpickle.loads(row) - // `Opcodes.MEMOIZE` of Protocol 4 (Python 3.4+) will store objects in internal map - // of `Unpickler`. This map is cleared when calling `Unpickler.close()`. Pyrolite - // doesn't clear it up, so we manually clear it. - unpickle.close() if (batched) { obj match { case list: JArrayList[_] => list.asScala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index 578b779cd52d..19e53e7eac84 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -141,7 +141,7 @@ class StandardScalerModel @Since("1.3.0") ( case d: DenseVector => d.values.clone() case v: Vector => v.toArray } - val newValues = transfromWithMean(values) + val newValues = transformWithMean(values) Vectors.dense(newValues) } else if (withStd) { vector match { @@ -161,7 +161,7 @@ class StandardScalerModel @Since("1.3.0") ( } } - private[spark] def transfromWithMean(values: Array[Double]): Array[Double] = { + private[spark] def transformWithMean(values: Array[Double]): Array[Double] = { // By default, Scala generates Java methods for member variables. So every time when // the member variables are accessed, `invokespecial` will be called which is expensive. // This can be avoid by having a local reference of `shift`. 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 a660492c7ae5..03afd29e4750 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 dcb1f398b04b..26a75699248d 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 20bd2e5e0dc1..fa8f03be089c 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 566ce95be084..cca4eb4e4260 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 e30ad159676f..8011026e6fa6 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 f464d25c3fbd..9b4dc29d326a 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 e32767edb17a..4613f7fb6f40 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 5feccdf33681..9cbb3d0024da 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 1cc8f342021a..d43e62bb6553 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/pom.xml b/pom.xml index 81e6e7956ab7..dae5eb007015 100644 --- a/pom.xml +++ b/pom.xml @@ -832,7 +832,7 @@ org.mockito mockito-core - 2.23.4 + 2.28.2 test diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index cb3b80309643..5978f88d6a46 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -372,7 +372,11 @@ object MimaExcludes { // [SPARK-26616][MLlib] Expose document frequency in IDFModel ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.IDFModel.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf") + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf"), + + // [SPARK-28199][SS] Remove deprecated ProcessingTime + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.ProcessingTime"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.ProcessingTime$") ) // Exclude rules for 2.4.x diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index c257ace02cfe..ce6543952bf6 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 6405b9fce7ef..56d619072316 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 927cc77e201a..349130f22fad 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/rdd.py b/python/pyspark/rdd.py index 8bcc67ab1c3e..96fdf5f33b39 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2490,7 +2490,7 @@ def _prepare_for_python_RDD(sc, command): # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() pickled_command = ser.dumps(command) - if len(pickled_command) > (1 << 20): # 1M + if len(pickled_command) > sc._jvm.PythonUtils.getBroadcastThreshold(sc._jsc): # Default 1M # The broadcast will have same life cycle as created PythonRDD broadcast = sc.broadcast(pickled_command) pickled_command = ser.dumps(broadcast) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f8be8ee5d4c3..398471234d2b 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -22,8 +22,10 @@ basestring = unicode = str long = int from functools import reduce + from html import escape as html_escape else: from itertools import imap as map + from cgi import escape as html_escape import warnings @@ -375,7 +377,6 @@ def _repr_html_(self): by 'spark.sql.repl.eagerEval.enabled', this only called by REPL you are using support eager evaluation with HTML. """ - import cgi if not self._support_repr_html: self._support_repr_html = True if self.sql_ctx._conf.isReplEagerEvalEnabled(): @@ -390,11 +391,11 @@ def _repr_html_(self): html = "\n" # generate table head - html += "\n" % "\n" % "\n" % "
%s
".join(map(lambda x: cgi.escape(x), head)) + html += "
%s
".join(map(lambda x: html_escape(x), head)) # generate table rows for row in row_data: html += "
%s
".join( - map(lambda x: cgi.escape(x), row)) + map(lambda x: html_escape(x), row)) html += "
\n" if has_more_data: html += "only showing top %d %s\n" % ( diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 6bb7da6b2edb..e531000f3295 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 aa5bf635d187..f9bc2ff72a50 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_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index e5c3b48d548e..5550a093bf80 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -750,6 +750,7 @@ def test_query_execution_listener_on_collect(self): self.spark._jvm.OnSuccessCall.isCalled(), "The callback from the query execution listener should not be called before 'collect'") self.spark.sql("SELECT * FROM range(1)").collect() + self.spark.sparkContext._jsc.sc().listenerBus().waitUntilEmpty(10000) self.assertTrue( self.spark._jvm.OnSuccessCall.isCalled(), "The callback from the query execution listener should be called after 'collect'") @@ -764,6 +765,7 @@ def test_query_execution_listener_on_collect_with_arrow(self): "The callback from the query execution listener should not be " "called before 'toPandas'") self.spark.sql("SELECT * FROM range(1)").toPandas() + self.spark.sparkContext._jsc.sc().listenerBus().waitUntilEmpty(10000) self.assertTrue( self.spark._jvm.OnSuccessCall.isCalled(), "The callback from the query execution listener should be called after 'toPandas'") diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 7dfc75797009..64f2fd6a3919 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 a70807248960..2530cc2ebf22 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/tests/test_serde.py b/python/pyspark/sql/tests/test_serde.py index f9bed7604b13..ea2a686cddaa 100644 --- a/python/pyspark/sql/tests/test_serde.py +++ b/python/pyspark/sql/tests/test_serde.py @@ -128,10 +128,6 @@ def test_BinaryType_serialization(self): def test_int_array_serialization(self): # Note that this test seems dependent on parallelism. - # This issue is because internal object map in Pyrolite is not cleared after op code - # STOP. If we use protocol 4 to pickle Python objects, op code MEMOIZE will store - # objects in the map. We need to clear up it to make sure next unpickling works on - # clear map. data = self.spark.sparkContext.parallelize([[1, 2, 3, 4]] * 100, numSlices=12) df = self.spark.createDataFrame(data, "array") self.assertEqual(len(list(filter(lambda r: None in r.value, df.collect()))), 0) diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index ca5e85bb3a9b..c30cc1482750 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/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 5b361d17c01a..651e706021fc 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -51,7 +51,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.api.python.PythonUtils import org.apache.spark.deploy.{SparkApplication, SparkHadoopUtil} import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -241,12 +241,12 @@ private[spark] class Client( newApp: YarnClientApplication, containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { - val yarnAMResources = - if (isClusterMode) { - sparkConf.getAllWithPrefix(config.YARN_DRIVER_RESOURCE_TYPES_PREFIX).toMap - } else { - sparkConf.getAllWithPrefix(config.YARN_AM_RESOURCE_TYPES_PREFIX).toMap - } + val componentName = if (isClusterMode) { + config.YARN_DRIVER_RESOURCE_TYPES_PREFIX + } else { + config.YARN_AM_RESOURCE_TYPES_PREFIX + } + val yarnAMResources = getYarnResourcesAndAmounts(sparkConf, componentName) val amResources = yarnAMResources ++ getYarnResourcesFromSparkResources(SPARK_DRIVER_PREFIX, sparkConf) logDebug(s"AM resources: $amResources") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala index cb0c68d1d346..522c16b3a108 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala @@ -26,11 +26,11 @@ import scala.util.Try import org.apache.hadoop.yarn.api.records.Resource import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceID +import org.apache.spark.resource.ResourceUtils.{AMOUNT, FPGA, GPU} import org.apache.spark.util.{CausedBy, Utils} /** @@ -40,6 +40,45 @@ import org.apache.spark.util.{CausedBy, Utils} private object ResourceRequestHelper extends Logging { private val AMOUNT_AND_UNIT_REGEX = "([0-9]+)([A-Za-z]*)".r private val RESOURCE_INFO_CLASS = "org.apache.hadoop.yarn.api.records.ResourceInformation" + val YARN_GPU_RESOURCE_CONFIG = "yarn.io/gpu" + val YARN_FPGA_RESOURCE_CONFIG = "yarn.io/fpga" + + private[yarn] def getYarnResourcesAndAmounts( + sparkConf: SparkConf, + componentName: String): Map[String, String] = { + sparkConf.getAllWithPrefix(s"$componentName").map { case (key, value) => + val splitIndex = key.lastIndexOf('.') + if (splitIndex == -1) { + val errorMessage = s"Missing suffix for ${componentName}${key}, you must specify" + + s" a suffix - $AMOUNT is currently the only supported suffix." + throw new IllegalArgumentException(errorMessage.toString()) + } + val resourceName = key.substring(0, splitIndex) + val resourceSuffix = key.substring(splitIndex + 1) + if (!AMOUNT.equals(resourceSuffix)) { + val errorMessage = s"Unsupported suffix: $resourceSuffix in: ${componentName}${key}, " + + s"only .$AMOUNT is supported." + throw new IllegalArgumentException(errorMessage.toString()) + } + (resourceName, value) + }.toMap + } + + /** + * Convert Spark resources into YARN resources. + * The only resources we know how to map from spark configs to yarn configs are + * gpus and fpgas, everything else the user has to specify them in both the + * spark.yarn.*.resource and the spark.*.resource configs. + */ + private[yarn] def getYarnResourcesFromSparkResources( + confPrefix: String, + sparkConf: SparkConf + ): Map[String, String] = { + Map(GPU -> YARN_GPU_RESOURCE_CONFIG, FPGA -> YARN_FPGA_RESOURCE_CONFIG).map { + case (rName, yarnName) => + (yarnName -> sparkConf.get(ResourceID(confPrefix, rName).amountConf, "0")) + }.filter { case (_, count) => count.toLong > 0 } + } /** * Validates sparkConf and throws a SparkException if any of standard resources (memory or cores) @@ -81,8 +120,9 @@ private object ResourceRequestHelper extends Logging { val errorMessage = new mutable.StringBuilder() resourceDefinitions.foreach { case (sparkName, resourceRequest) => - if (sparkConf.contains(resourceRequest)) { - errorMessage.append(s"Error: Do not use $resourceRequest, " + + val resourceRequestAmount = s"${resourceRequest}.${AMOUNT}" + if (sparkConf.contains(resourceRequestAmount)) { + errorMessage.append(s"Error: Do not use $resourceRequestAmount, " + s"please use $sparkName instead!\n") } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 6e634b921fcd..8ec7bd66b250 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.{SecurityManager, SparkConf, SparkException} +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging @@ -142,8 +143,8 @@ private[yarn] class YarnAllocator( protected val executorCores = sparkConf.get(EXECUTOR_CORES) private val executorResourceRequests = - sparkConf.getAllWithPrefix(config.YARN_EXECUTOR_RESOURCE_TYPES_PREFIX).toMap ++ - getYarnResourcesFromSparkResources(SPARK_EXECUTOR_PREFIX, sparkConf) + getYarnResourcesAndAmounts(sparkConf, config.YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) ++ + getYarnResourcesFromSparkResources(SPARK_EXECUTOR_PREFIX, sparkConf) // Resource capability requested for each executor private[yarn] val resource: Resource = { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 6b87eec795f9..11035520ae18 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -41,24 +41,6 @@ object YarnSparkHadoopUtil { val MEMORY_OVERHEAD_MIN = 384L val ANY_HOST = "*" - val YARN_GPU_RESOURCE_CONFIG = "yarn.io/gpu" - val YARN_FPGA_RESOURCE_CONFIG = "yarn.io/fpga" - - /** - * Convert Spark resources into YARN resources. - * The only resources we know how to map from spark configs to yarn configs are - * gpus and fpgas, everything else the user has to specify them in both the - * spark.yarn.*.resource and the spark.*.resource configs. - */ - private[yarn] def getYarnResourcesFromSparkResources( - confPrefix: String, - sparkConf: SparkConf - ): Map[String, String] = { - Map(GPU -> YARN_GPU_RESOURCE_CONFIG, FPGA -> YARN_FPGA_RESOURCE_CONFIG).map { - case (rName, yarnName) => - (yarnName -> sparkConf.get(ResourceID(confPrefix, rName).amountConf, "0")) - }.filter { case (_, count) => count.toLong > 0 } - } // All RM requests are issued with same priority : we do not (yet) have any distinction between // request types (like map/reduce in hadoop for example) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index d5f1992a09f5..847fc3773de5 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -38,10 +38,11 @@ import org.mockito.Mockito.{spy, verify} import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkException, SparkFunSuite, TestUtils} -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceID +import org.apache.spark.resource.ResourceUtils.AMOUNT import org.apache.spark.util.{SparkConfWithEnv, Utils} class ClientSuite extends SparkFunSuite with Matchers { @@ -372,7 +373,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, deployMode) resources.foreach { case (name, v) => - conf.set(prefix + name, v.toString) + conf.set(s"${prefix}${name}.${AMOUNT}", v.toString) } val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) @@ -397,7 +398,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, "cluster") resources.keys.foreach { yarnName => - conf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${yarnName}", "2") + conf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${yarnName}.${AMOUNT}", "2") } resources.values.foreach { rName => conf.set(ResourceID(SPARK_DRIVER_PREFIX, rName).amountConf, "3") @@ -407,9 +408,9 @@ class ClientSuite extends SparkFunSuite with Matchers { ResourceRequestHelper.validateResources(conf) }.getMessage() - assert(error.contains("Do not use spark.yarn.driver.resource.yarn.io/fpga," + + assert(error.contains("Do not use spark.yarn.driver.resource.yarn.io/fpga.amount," + " please use spark.driver.resource.fpga.amount")) - assert(error.contains("Do not use spark.yarn.driver.resource.yarn.io/gpu," + + assert(error.contains("Do not use spark.yarn.driver.resource.yarn.io/gpu.amount," + " please use spark.driver.resource.gpu.amount")) } @@ -420,7 +421,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, "cluster") resources.keys.foreach { yarnName => - conf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnName}", "2") + conf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnName}.${AMOUNT}", "2") } resources.values.foreach { rName => conf.set(ResourceID(SPARK_EXECUTOR_PREFIX, rName).amountConf, "3") @@ -430,9 +431,9 @@ class ClientSuite extends SparkFunSuite with Matchers { ResourceRequestHelper.validateResources(conf) }.getMessage() - assert(error.contains("Do not use spark.yarn.executor.resource.yarn.io/fpga," + + assert(error.contains("Do not use spark.yarn.executor.resource.yarn.io/fpga.amount," + " please use spark.executor.resource.fpga.amount")) - assert(error.contains("Do not use spark.yarn.executor.resource.yarn.io/gpu," + + assert(error.contains("Do not use spark.yarn.executor.resource.yarn.io/gpu.amount," + " please use spark.executor.resource.gpu.amount")) } @@ -450,7 +451,7 @@ class ClientSuite extends SparkFunSuite with Matchers { conf.set(ResourceID(SPARK_DRIVER_PREFIX, rName).amountConf, "3") } // also just set yarn one that we don't convert - conf.set(YARN_DRIVER_RESOURCE_TYPES_PREFIX + yarnMadeupResource, "5") + conf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${yarnMadeupResource}.${AMOUNT}", "5") val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala index 9e3cc6ec01df..f5ec531e26e0 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala @@ -22,9 +22,11 @@ import org.apache.hadoop.yarn.util.Records import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.ResourceRequestTestHelper.ResourceInformation import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.config.{DRIVER_CORES, DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_MEMORY} +import org.apache.spark.resource.ResourceUtils.AMOUNT class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { @@ -32,16 +34,18 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { private val CUSTOM_RES_2 = "custom-resource-type-2" private val MEMORY = "memory" private val CORES = "cores" - private val NEW_CONFIG_EXECUTOR_MEMORY = YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + MEMORY - private val NEW_CONFIG_EXECUTOR_CORES = YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + CORES - private val NEW_CONFIG_AM_MEMORY = YARN_AM_RESOURCE_TYPES_PREFIX + MEMORY - private val NEW_CONFIG_AM_CORES = YARN_AM_RESOURCE_TYPES_PREFIX + CORES - private val NEW_CONFIG_DRIVER_MEMORY = YARN_DRIVER_RESOURCE_TYPES_PREFIX + MEMORY - private val NEW_CONFIG_DRIVER_CORES = YARN_DRIVER_RESOURCE_TYPES_PREFIX + CORES + private val NEW_CONFIG_EXECUTOR_MEMORY = + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${MEMORY}.${AMOUNT}" + private val NEW_CONFIG_EXECUTOR_CORES = + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${CORES}.${AMOUNT}" + private val NEW_CONFIG_AM_MEMORY = s"${YARN_AM_RESOURCE_TYPES_PREFIX}${MEMORY}.${AMOUNT}" + private val NEW_CONFIG_AM_CORES = s"${YARN_AM_RESOURCE_TYPES_PREFIX}${CORES}.${AMOUNT}" + private val NEW_CONFIG_DRIVER_MEMORY = s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${MEMORY}.${AMOUNT}" + private val NEW_CONFIG_DRIVER_CORES = s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${CORES}.${AMOUNT}" test("empty SparkConf should be valid") { val sparkConf = new SparkConf() - ResourceRequestHelper.validateResources(sparkConf) + validateResources(sparkConf) } test("just normal resources are defined") { @@ -50,7 +54,44 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { sparkConf.set(DRIVER_CORES.key, "4") sparkConf.set(EXECUTOR_MEMORY.key, "4G") sparkConf.set(EXECUTOR_CORES.key, "2") - ResourceRequestHelper.validateResources(sparkConf) + validateResources(sparkConf) + } + + test("get yarn resources from configs") { + val sparkConf = new SparkConf() + val resources = Map(YARN_GPU_RESOURCE_CONFIG -> "2G", + YARN_FPGA_RESOURCE_CONFIG -> "3G", "custom" -> "4") + resources.foreach { case (name, value) => + sparkConf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${name}.${AMOUNT}", value) + sparkConf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${name}.${AMOUNT}", value) + sparkConf.set(s"${YARN_AM_RESOURCE_TYPES_PREFIX}${name}.${AMOUNT}", value) + } + var parsedResources = getYarnResourcesAndAmounts(sparkConf, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) + assert(parsedResources === resources) + parsedResources = getYarnResourcesAndAmounts(sparkConf, YARN_DRIVER_RESOURCE_TYPES_PREFIX) + assert(parsedResources === resources) + parsedResources = getYarnResourcesAndAmounts(sparkConf, YARN_AM_RESOURCE_TYPES_PREFIX) + assert(parsedResources === resources) + } + + test("get invalid yarn resources from configs") { + val sparkConf = new SparkConf() + + val missingAmountConfig = s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}missingAmount" + // missing .amount + sparkConf.set(missingAmountConfig, "2g") + var thrown = intercept[IllegalArgumentException] { + getYarnResourcesAndAmounts(sparkConf, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) + } + thrown.getMessage should include("Missing suffix for") + + sparkConf.remove(missingAmountConfig) + sparkConf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}customResource.invalid", "2g") + + thrown = intercept[IllegalArgumentException] { + getYarnResourcesAndAmounts(sparkConf, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) + } + thrown.getMessage should include("Unsupported suffix") } Seq( @@ -60,14 +101,14 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { ResourceInformation(CUSTOM_RES_2, 10, "G")) ).foreach { case (name, resources) => test(s"valid request: $name") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + assume(isYarnResourceTypesAvailable()) val resourceDefs = resources.map { r => r.name } val requests = resources.map { r => (r.name, r.value.toString + r.unit) }.toMap ResourceRequestTestHelper.initializeResourceTypes(resourceDefs) val resource = createResource() - ResourceRequestHelper.setResourceRequests(requests, resource) + setResourceRequests(requests, resource) resources.foreach { r => val requested = ResourceRequestTestHelper.getResourceInformationByName(resource, r.name) @@ -82,12 +123,12 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { ("invalid unit", CUSTOM_RES_1, "123ppp") ).foreach { case (name, key, value) => test(s"invalid request: $name") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + assume(isYarnResourceTypesAvailable()) ResourceRequestTestHelper.initializeResourceTypes(Seq(key)) val resource = createResource() val thrown = intercept[IllegalArgumentException] { - ResourceRequestHelper.setResourceRequests(Map(key -> value), resource) + setResourceRequests(Map(key -> value), resource) } thrown.getMessage should include (key) } @@ -95,20 +136,20 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { Seq( NEW_CONFIG_EXECUTOR_MEMORY -> "30G", - YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "memory-mb" -> "30G", - YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "mb" -> "30G", + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}memory-mb.$AMOUNT" -> "30G", + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}mb.$AMOUNT" -> "30G", NEW_CONFIG_EXECUTOR_CORES -> "5", - YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "vcores" -> "5", + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}vcores.$AMOUNT" -> "5", NEW_CONFIG_AM_MEMORY -> "1G", NEW_CONFIG_DRIVER_MEMORY -> "1G", NEW_CONFIG_AM_CORES -> "3", NEW_CONFIG_DRIVER_CORES -> "1G" ).foreach { case (key, value) => test(s"disallowed resource request: $key") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + assume(isYarnResourceTypesAvailable()) val conf = new SparkConf(false).set(key, value) val thrown = intercept[SparkException] { - ResourceRequestHelper.validateResources(conf) + validateResources(conf) } thrown.getMessage should include (key) } @@ -126,7 +167,7 @@ class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { sparkConf.set(NEW_CONFIG_DRIVER_MEMORY, "2G") val thrown = intercept[SparkException] { - ResourceRequestHelper.validateResources(sparkConf) + validateResources(sparkConf) } thrown.getMessage should ( include(NEW_CONFIG_EXECUTOR_MEMORY) and diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index ca89af26230f..4ac27ede6483 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -31,9 +31,11 @@ import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceUtils.{AMOUNT, GPU} import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.SplitInfo @@ -160,12 +162,12 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter } test("custom resource requested from yarn") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + assume(isYarnResourceTypesAvailable()) ResourceRequestTestHelper.initializeResourceTypes(List("gpu")) val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) val handler = createAllocator(1, mockAmClient, - Map(YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "gpu" -> "2G")) + Map(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${GPU}.${AMOUNT}" -> "2G")) handler.updateResourceRequests() val container = createContainer("host1", resource = handler.resource) @@ -174,7 +176,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter // get amount of memory and vcores from resource, so effectively skipping their validation val expectedResources = Resource.newInstance(handler.resource.getMemory(), handler.resource.getVirtualCores) - ResourceRequestHelper.setResourceRequests(Map("gpu" -> "2G"), expectedResources) + setResourceRequests(Map("gpu" -> "2G"), expectedResources) val captor = ArgumentCaptor.forClass(classOf[ContainerRequest]) verify(mockAmClient).addContainerRequest(captor.capture()) @@ -183,15 +185,16 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter } test("custom spark resource mapped to yarn resource configs") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + assume(isYarnResourceTypesAvailable()) val yarnMadeupResource = "yarn.io/madeup" val yarnResources = Seq(YARN_GPU_RESOURCE_CONFIG, YARN_FPGA_RESOURCE_CONFIG, yarnMadeupResource) ResourceRequestTestHelper.initializeResourceTypes(yarnResources) val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) + val madeupConfigName = s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnMadeupResource}.${AMOUNT}" val sparkResources = Map(EXECUTOR_GPU_ID.amountConf -> "3", EXECUTOR_FPGA_ID.amountConf -> "2", - s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnMadeupResource}" -> "5") + madeupConfigName -> "5") val handler = createAllocator(1, mockAmClient, sparkResources) handler.updateResourceRequests() 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 a285a57f5243..837dd0332151 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 000000000000..fc055e91a6ac --- /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/catalog/v2/TableChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java index 9b87e676d9b2..7eef6aea8812 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java @@ -227,6 +227,10 @@ public String property() { } } + interface ColumnChange extends TableChange { + String[] fieldNames(); + } + /** * A TableChange to add a field. *

@@ -234,7 +238,7 @@ public String property() { * If the new field is nested and its parent does not exist or is not a struct, the change must * result in an {@link IllegalArgumentException}. */ - final class AddColumn implements TableChange { + final class AddColumn implements ColumnChange { private final String[] fieldNames; private final DataType dataType; private final boolean isNullable; @@ -247,6 +251,7 @@ private AddColumn(String[] fieldNames, DataType dataType, boolean isNullable, St this.comment = comment; } + @Override public String[] fieldNames() { return fieldNames; } @@ -272,7 +277,7 @@ public String comment() { *

* If the field does not exist, the change must result in an {@link IllegalArgumentException}. */ - final class RenameColumn implements TableChange { + final class RenameColumn implements ColumnChange { private final String[] fieldNames; private final String newName; @@ -281,6 +286,7 @@ private RenameColumn(String[] fieldNames, String newName) { this.newName = newName; } + @Override public String[] fieldNames() { return fieldNames; } @@ -297,7 +303,7 @@ public String newName() { *

* If the field does not exist, the change must result in an {@link IllegalArgumentException}. */ - final class UpdateColumnType implements TableChange { + final class UpdateColumnType implements ColumnChange { private final String[] fieldNames; private final DataType newDataType; private final boolean isNullable; @@ -308,6 +314,7 @@ private UpdateColumnType(String[] fieldNames, DataType newDataType, boolean isNu this.isNullable = isNullable; } + @Override public String[] fieldNames() { return fieldNames; } @@ -328,7 +335,7 @@ public boolean isNullable() { *

* If the field does not exist, the change must result in an {@link IllegalArgumentException}. */ - final class UpdateColumnComment implements TableChange { + final class UpdateColumnComment implements ColumnChange { private final String[] fieldNames; private final String newComment; @@ -337,6 +344,7 @@ private UpdateColumnComment(String[] fieldNames, String newComment) { this.newComment = newComment; } + @Override public String[] fieldNames() { return fieldNames; } @@ -351,13 +359,14 @@ public String newComment() { *

* If the field does not exist, the change must result in an {@link IllegalArgumentException}. */ - final class DeleteColumn implements TableChange { + final class DeleteColumn implements ColumnChange { private final String[] fieldNames; private DeleteColumn(String[] fieldNames) { this.fieldNames = fieldNames; } + @Override public String[] fieldNames() { return fieldNames; } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala index 6de1ef5660e5..7cc80c41a901 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableChange} import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RemoveProperty, RenameColumn, SetProperty, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.sources.v2.Table -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} object CatalogV2Util { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ @@ -132,16 +132,45 @@ object CatalogV2Util { val pos = struct.getFieldIndex(fieldNames.head) .getOrElse(throw new IllegalArgumentException(s"Cannot find field: ${fieldNames.head}")) val field = struct.fields(pos) - val replacement: Option[StructField] = if (fieldNames.tail.isEmpty) { - update(field) - } else { - field.dataType match { - case nestedStruct: StructType => - val updatedType: StructType = replace(nestedStruct, fieldNames.tail, update) - Some(StructField(field.name, updatedType, field.nullable, field.metadata)) - case _ => - throw new IllegalArgumentException(s"Not a struct: ${fieldNames.head}") - } + val replacement: Option[StructField] = (fieldNames.tail, field.dataType) match { + case (Seq(), _) => + update(field) + + case (names, struct: StructType) => + val updatedType: StructType = replace(struct, names, update) + Some(StructField(field.name, updatedType, field.nullable, field.metadata)) + + case (Seq("key"), map @ MapType(keyType, _, _)) => + val updated = update(StructField("key", keyType, nullable = false)) + .getOrElse(throw new IllegalArgumentException(s"Cannot delete map key")) + Some(field.copy(dataType = map.copy(keyType = updated.dataType))) + + case (Seq("key", names @ _*), map @ MapType(keyStruct: StructType, _, _)) => + Some(field.copy(dataType = map.copy(keyType = replace(keyStruct, names, update)))) + + case (Seq("value"), map @ MapType(_, mapValueType, isNullable)) => + val updated = update(StructField("value", mapValueType, nullable = isNullable)) + .getOrElse(throw new IllegalArgumentException(s"Cannot delete map value")) + Some(field.copy(dataType = map.copy( + valueType = updated.dataType, + valueContainsNull = updated.nullable))) + + case (Seq("value", names @ _*), map @ MapType(_, valueStruct: StructType, _)) => + Some(field.copy(dataType = map.copy(valueType = replace(valueStruct, names, update)))) + + case (Seq("element"), array @ ArrayType(elementType, isNullable)) => + val updated = update(StructField("element", elementType, nullable = isNullable)) + .getOrElse(throw new IllegalArgumentException(s"Cannot delete array element")) + Some(field.copy(dataType = array.copy( + elementType = updated.dataType, + containsNull = updated.nullable))) + + case (Seq("element", names @ _*), array @ ArrayType(elementStruct: StructType, _)) => + Some(field.copy(dataType = array.copy(elementType = replace(elementStruct, names, update)))) + + case (names, dataType) => + throw new IllegalArgumentException( + s"Cannot find field: ${names.head} in ${dataType.simpleString}") } val newFields = struct.fields.zipWithIndex.flatMap { 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 000000000000..b2baa93b146a --- /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 9853a4fcc2f9..29d81c553ff6 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 6020b068155f..488252aa0c7b 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/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 1d0dba262c10..e55cdfedd323 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, LookupCatalog} +import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, LookupCatalog, TableChange} import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects._ 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} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL @@ -165,6 +166,7 @@ class Analyzer( new SubstituteUnresolvedOrdinals(conf)), Batch("Resolution", fixedPoint, ResolveTableValuedFunctions :: + ResolveAlterTable :: ResolveTables :: ResolveRelations :: ResolveReferences :: @@ -211,38 +213,6 @@ class Analyzer( CleanupAliases) ) - /** - * Analyze cte definitions and substitute child plan with analyzed cte definitions. - */ - object CTESubstitution extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case With(child, relations) => - // substitute CTE expressions right-to-left to resolve references to previous CTEs: - // with a as (select * from t), b as (select * from a) select * from b - relations.foldRight(child) { - case ((cteName, ctePlan), currentPlan) => - substituteCTE(currentPlan, cteName, ctePlan) - } - case other => other - } - - private def substituteCTE( - plan: LogicalPlan, - cteName: String, - ctePlan: LogicalPlan): LogicalPlan = { - plan resolveOperatorsUp { - case UnresolvedRelation(Seq(table)) if resolver(cteName, table) => - ctePlan - case other => - // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. - other transformExpressions { - case e: SubqueryExpression => - e.withNewPlan(substituteCTE(e.plan, cteName, ctePlan)) - } - } - } - } - /** * Substitute child plan with WindowSpecDefinitions. */ @@ -787,6 +757,86 @@ class Analyzer( } } + /** + * Resolve ALTER TABLE statements that use a DSv2 catalog. + * + * This rule converts unresolved ALTER TABLE statements to v2 when a v2 catalog is responsible + * for the table identifier. A v2 catalog is responsible for an identifier when the identifier + * has a catalog specified, like prod_catalog.db.table, or when a default v2 catalog is set and + * the table identifier does not include a catalog. + */ + object ResolveAlterTable extends Rule[LogicalPlan] { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case alter @ AlterTableAddColumnsStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), cols) => + val changes = cols.map { col => + TableChange.addColumn(col.name.toArray, col.dataType, true, col.comment.orNull) + } + + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + changes) + + case alter @ AlterTableAlterColumnStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), colName, dataType, comment) => + val typeChange = dataType.map { newDataType => + TableChange.updateColumnType(colName.toArray, newDataType, true) + } + + val commentChange = comment.map { newComment => + TableChange.updateColumnComment(colName.toArray, newComment) + } + + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + typeChange.toSeq ++ commentChange.toSeq) + + case alter @ AlterTableRenameColumnStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), col, newName) => + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + Seq(TableChange.renameColumn(col.toArray, newName))) + + case alter @ AlterTableDropColumnsStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), cols) => + val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + changes) + + case alter @ AlterTableSetPropertiesStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), props) => + val changes = props.map { + case (key, value) => + TableChange.setProperty(key, value) + } + + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + changes.toSeq) + + case alter @ AlterTableUnsetPropertiesStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), keys, _) => + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + keys.map(key => TableChange.removeProperty(key))) + + case alter @ AlterTableSetLocationStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), newLoc) => + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + Seq(TableChange.setProperty("location", newLoc))) + } + } + /** * Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s from * a logical plan node's children. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala new file mode 100644 index 000000000000..60e6bf8db06d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, With} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.LEGACY_CTE_PRECEDENCE_ENABLED + +/** + * Analyze WITH nodes and substitute child plan with CTE definitions. + */ +object CTESubstitution extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + if (SQLConf.get.getConf(LEGACY_CTE_PRECEDENCE_ENABLED)) { + legacyTraverseAndSubstituteCTE(plan) + } else { + traverseAndSubstituteCTE(plan, false) + } + } + + private def legacyTraverseAndSubstituteCTE(plan: LogicalPlan): LogicalPlan = { + plan.resolveOperatorsUp { + case With(child, relations) => + // substitute CTE expressions right-to-left to resolve references to previous CTEs: + // with a as (select * from t), b as (select * from a) select * from b + relations.foldRight(child) { + case ((cteName, ctePlan), currentPlan) => substituteCTE(currentPlan, cteName, ctePlan) + } + } + } + + /** + * Traverse the plan and expression nodes as a tree and replace matching references to CTE + * definitions. + * - If the rule encounters a WITH node then it substitutes the child of the node with CTE + * definitions of the node right-to-left order as a definition can reference to a previous + * one. + * For example the following query is valid: + * WITH + * t AS (SELECT 1), + * t2 AS (SELECT * FROM t) + * SELECT * FROM t2 + * - If a CTE definition contains an inner WITH node then substitution of inner should take + * precedence because it can shadow an outer CTE definition. + * For example the following query should return 2: + * WITH + * t AS (SELECT 1), + * t2 AS ( + * WITH t AS (SELECT 2) + * SELECT * FROM t + * ) + * SELECT * FROM t2 + * - If a CTE definition contains a subquery that contains an inner WITH node then substitution + * of inner should take precedence because it can shadow an outer CTE definition. + * For example the following query should return 2: + * WITH t AS (SELECT 1 AS c) + * SELECT max(c) FROM ( + * WITH t AS (SELECT 2 AS c) + * SELECT * FROM t + * ) + * - If a CTE definition contains a subquery expression that contains an inner WITH node then + * substitution of inner should take precedence because it can shadow an outer CTE + * definition. + * For example the following query should return 2: + * WITH t AS (SELECT 1) + * SELECT ( + * WITH t AS (SELECT 2) + * SELECT * FROM t + * ) + * @param plan the plan to be traversed + * @param inTraverse whether the current traverse is called from another traverse, only in this + * case name collision can occur + * @return the plan where CTE substitution is applied + */ + private def traverseAndSubstituteCTE(plan: LogicalPlan, inTraverse: Boolean): LogicalPlan = { + plan.resolveOperatorsUp { + case With(child: LogicalPlan, relations) => + // child might contain an inner CTE that has priority so traverse and substitute inner CTEs + // in child first + val traversedChild: LogicalPlan = child transformExpressions { + case e: SubqueryExpression => e.withNewPlan(traverseAndSubstituteCTE(e.plan, true)) + } + + // Substitute CTE definitions from last to first as a CTE definition can reference a + // previous one + relations.foldRight(traversedChild) { + case ((cteName, ctePlan), currentPlan) => + // A CTE definition might contain an inner CTE that has priority, so traverse and + // substitute CTE defined in ctePlan. + // A CTE definition might not be used at all or might be used multiple times. To avoid + // computation if it is not used and to avoid multiple recomputation if it is used + // multiple times we use a lazy construct with call-by-name parameter passing. + lazy val substitutedCTEPlan = traverseAndSubstituteCTE(ctePlan, true) + substituteCTE(currentPlan, cteName, substitutedCTEPlan) + } + + // CTE name collision can occur only when inTraverse is true, it helps to avoid eager CTE + // substitution in a subquery expression. + case other if inTraverse => + other.transformExpressions { + case e: SubqueryExpression => e.withNewPlan(traverseAndSubstituteCTE(e.plan, true)) + } + } + } + + private def substituteCTE( + plan: LogicalPlan, + cteName: String, + ctePlan: => LogicalPlan): LogicalPlan = + plan resolveOperatorsUp { + case UnresolvedRelation(Seq(table)) if plan.conf.resolver(cteName, table) => ctePlan + + case other => + // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. + other transformExpressions { + case e: SubqueryExpression => e.withNewPlan(substituteCTE(e.plan, cteName, ctePlan)) + } + } +} 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 000000000000..3036f7c21093 --- /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/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 02031e758d83..ae19d02e4475 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression @@ -353,6 +354,59 @@ trait CheckAnalysis extends PredicateHelper { case _ => } + case alter: AlterTable if alter.childrenResolved => + val table = alter.table + def findField(operation: String, fieldName: Array[String]): StructField = { + // include collections because structs nested in maps and arrays may be altered + val field = table.schema.findNestedField(fieldName, includeCollections = true) + if (field.isEmpty) { + throw new AnalysisException( + s"Cannot $operation missing field in ${table.name} schema: ${fieldName.quoted}") + } + field.get + } + + alter.changes.foreach { + case add: AddColumn => + val parent = add.fieldNames.init + if (parent.nonEmpty) { + findField("add to", parent) + } + case update: UpdateColumnType => + val field = findField("update", update.fieldNames) + val fieldName = update.fieldNames.quoted + update.newDataType match { + case _: StructType => + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName type: " + + s"update a struct by adding, deleting, or updating its fields") + case _: MapType => + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName type: " + + s"update a map by updating $fieldName.key or $fieldName.value") + case _: ArrayType => + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName type: " + + s"update the element by updating $fieldName.element") + case _: AtomicType => + // update is okay + } + if (!Cast.canUpCast(field.dataType, update.newDataType)) { + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName: " + + s"${field.dataType.simpleString} cannot be cast to " + + s"${update.newDataType.simpleString}") + } + case rename: RenameColumn => + findField("rename", rename.fieldNames) + case update: UpdateColumnComment => + findField("update", update.fieldNames) + case delete: DeleteColumn => + findField("delete", delete.fieldNames) + case _ => + // no validation needed for set and remove property + } + case _ => // Fallbacks to the following checks } 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 9fe95671cda0..3408b496d9d2 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 @@ -222,9 +222,12 @@ object FunctionRegistry { // math functions expression[Acos]("acos"), + expression[Acosh]("acosh"), expression[Asin]("asin"), + expression[Asinh]("asinh"), expression[Atan]("atan"), expression[Atan2]("atan2"), + expression[Atanh]("atanh"), expression[Bin]("bin"), expression[BRound]("bround"), expression[Cbrt]("cbrt"), @@ -411,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/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 1fdec89e258a..3125f8cb732d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -59,7 +59,7 @@ object TypeCoercion { CaseWhenCoercion :: IfCoercion :: StackCoercion :: - Division :: + Division(conf) :: ImplicitTypeCasts :: DateTimeOperations :: WindowFrameCoercion :: @@ -666,7 +666,7 @@ object TypeCoercion { * Hive only performs integral division with the DIV operator. The arguments to / are always * converted to fractional types. */ - object Division extends TypeCoercionRule { + case class Division(conf: SQLConf) extends TypeCoercionRule { override protected def coerceTypes( plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who has not been resolved yet, @@ -677,7 +677,12 @@ object TypeCoercion { case d: Divide if d.dataType == DoubleType => d case d: Divide if d.dataType.isInstanceOf[DecimalType] => d case Divide(left, right) if isNumericOrNull(left) && isNumericOrNull(right) => - Divide(Cast(left, DoubleType), Cast(right, DoubleType)) + (left.dataType, right.dataType) match { + case (_: IntegralType, _: IntegralType) if conf.preferIntegralDivision => + IntegralDivide(left, right) + case _ => + Divide(Cast(left, DoubleType), Cast(right, DoubleType)) + } } private def isNumericOrNull(ex: Expression): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index b700c336e6ae..9e0e0d528a96 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -40,12 +40,15 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str * * @param multipartIdentifier table name */ -case class UnresolvedRelation(multipartIdentifier: Seq[String]) extends LeafNode { +case class UnresolvedRelation( + multipartIdentifier: Seq[String]) extends LeafNode with NamedRelation { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ /** Returns a `.` separated name for this relation. */ def tableName: String = multipartIdentifier.quoted + override def name: String = tableName + override def output: Seq[Attribute] = Nil override lazy val resolved = false 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 1268fcffcfcd..7d52847216cc 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 c97303be1d27..bd499671d644 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 8477e63135e3..5314821ea3a5 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( @@ -2641,8 +2642,8 @@ object Sequence { while (t < exclusiveItem ^ stepSign < 0) { arr(i) = fromLong(t / scale) - t = timestampAddInterval(t, stepMonths, stepMicros, timeZone) i += 1 + 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""" @@ -2676,12 +2677,6 @@ object Sequence { |${genSequenceLengthCode(ctx, startMicros, stopMicros, intervalInMicros, arrLength)} """.stripMargin - val timestampAddIntervalCode = - s""" - |$t = org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampAddInterval( - | $t, $stepMonths, $stepMicros, $genTimeZone); - """.stripMargin - s""" |final int $stepMonths = $step.months; |final long $stepMicros = $step.microseconds; @@ -2705,8 +2700,9 @@ object Sequence { | | while ($t < $exclusiveItem ^ $stepSign < 0) { | $arr[$i] = ($elemType) ($t / ${scale}L); - | $timestampAddIntervalCode | $i += 1; + | $t = org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampAddInterval( + | $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 ccf6b36effa0..edb5382ae443 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 bdeb9ed29e0a..6dd2fa716e6b 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 @@ -287,6 +287,29 @@ case class Cos(child: Expression) extends UnaryMathExpression(math.cos, "COS") """) case class Cosh(child: Expression) extends UnaryMathExpression(math.cosh, "COSH") +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns inverse hyperbolic cosine of `expr`. + """, + arguments = """ + Arguments: + * expr - hyperbolic angle + """, + examples = """ + Examples: + > SELECT _FUNC_(1); + 0.0 + > SELECT _FUNC_(0); + NaN + """, + since = "3.0.0") +case class Acosh(child: Expression) + extends UnaryMathExpression((x: Double) => math.log(x + math.sqrt(x * x - 1.0)), "ACOSH") { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, c => s"java.lang.Math.log($c + java.lang.Math.sqrt($c * $c - 1.0))") + } +} + /** * Convert a num from one base to another * @@ -557,6 +580,31 @@ case class Sin(child: Expression) extends UnaryMathExpression(math.sin, "SIN") """) case class Sinh(child: Expression) extends UnaryMathExpression(math.sinh, "SINH") +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns inverse hyperbolic sine of `expr`. + """, + arguments = """ + Arguments: + * expr - hyperbolic angle + """, + examples = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + """, + since = "3.0.0") +case class Asinh(child: Expression) + extends UnaryMathExpression((x: Double) => x match { + case Double.NegativeInfinity => Double.NegativeInfinity + case _ => math.log(x + math.sqrt(x * x + 1.0)) }, "ASINH") { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, c => + s"$c == Double.NEGATIVE_INFINITY ? Double.NEGATIVE_INFINITY : " + + s"java.lang.Math.log($c + java.lang.Math.sqrt($c * $c + 1.0))") + } +} + @ExpressionDescription( usage = "_FUNC_(expr) - Returns the square root of `expr`.", examples = """ @@ -617,6 +665,29 @@ case class Cot(child: Expression) """) case class Tanh(child: Expression) extends UnaryMathExpression(math.tanh, "TANH") +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns inverse hyperbolic tangent of `expr`. + """, + arguments = """ + Arguments: + * expr - hyperbolic angle + """, + examples = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + > SELECT _FUNC_(2); + NaN + """, + since = "3.0.0") +case class Atanh(child: Expression) + extends UnaryMathExpression((x: Double) => 0.5 * math.log((1.0 + x) / (1.0 - x)), "ATANH") { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, c => s"0.5 * java.lang.Math.log((1.0 + $c)/(1.0 - $c))") + } +} + @ExpressionDescription( usage = "_FUNC_(expr) - Converts radians to degrees.", arguments = """ @@ -1106,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/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 06985ac85b70..02d5a1f27aa7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -116,6 +116,10 @@ trait PredicateHelper { // non-correlated subquery will be replaced as literal e.children.isEmpty case a: AttributeReference => true + // PythonUDF will be executed by dedicated physical operator later. + // For PythonUDFs that can't be evaluated in join condition, `PullOutPythonUDFInJoinCondition` + // will pull them out later. + case _: PythonUDF => true case e: Unevaluable => false case e => e.children.forall(canEvaluateWithinJoin) } 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 95aefb6422d6..43a6006f9b5c 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 c99d2c06fac6..206d09a6f79e 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 e4bde8934e21..3164ab7d5376 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,13 @@ 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._ +<<<<<<< HEAD import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} +||||||| merged common ancestors +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} +>>>>>>> origin/master import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -2127,6 +2133,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 +2309,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/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index a816922f49ae..51d2a73ea97b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -118,19 +118,23 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { // Replace null with default value for joining key, then those rows with null in it could // be joined together case EqualNullSafe(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => - Some((Coalesce(Seq(l, Literal.default(l.dataType))), - Coalesce(Seq(r, Literal.default(r.dataType))))) + Seq((Coalesce(Seq(l, Literal.default(l.dataType))), + Coalesce(Seq(r, Literal.default(r.dataType)))), + (IsNull(l), IsNull(r)) + ) case EqualNullSafe(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => - Some((Coalesce(Seq(r, Literal.default(r.dataType))), - Coalesce(Seq(l, Literal.default(l.dataType))))) + Seq((Coalesce(Seq(r, Literal.default(r.dataType))), + Coalesce(Seq(l, Literal.default(l.dataType)))), + (IsNull(r), IsNull(l)) + ) case other => None } val otherPredicates = predicates.filterNot { case EqualTo(l, r) if l.references.isEmpty || r.references.isEmpty => false - case EqualTo(l, r) => + case Equality(l, r) => canEvaluate(l, left) && canEvaluate(r, right) || canEvaluate(l, right) && canEvaluate(r, left) - case other => false + case _ => false } if (joinKeys.nonEmpty) { 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 789d4f442519..30f3b8336a56 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 @@ -17,7 +17,8 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, ColumnChange} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.AliasIdentifier import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} @@ -442,6 +443,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. */ @@ -517,6 +559,40 @@ case class DropTable( ident: Identifier, ifExists: Boolean) extends Command +/** + * Alter a table. + */ +case class AlterTable( + catalog: TableCatalog, + ident: Identifier, + table: NamedRelation, + changes: Seq[TableChange]) extends Command { + + override def children: Seq[LogicalPlan] = Seq(table) + + override lazy val resolved: Boolean = childrenResolved && { + changes.forall { + case add: AddColumn => + add.fieldNames match { + case Array(_) => + // a top-level field can always be added + true + case _ => + // the parent field must exist + table.schema.findNestedField(add.fieldNames.init, includeCollections = true).isDefined + } + + case colChange: ColumnChange => + // the column that will be changed must exist + table.schema.findNestedField(colChange.fieldNames, includeCollections = true).isDefined + + case _ => + // property changes require no resolution checks + true + } + } +} + /** * Insert some data into a table. Note that this plan is unresolved and has to be replaced by the 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 000000000000..2808892b089b --- /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 0596dc00985a..e79000d58350 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 7017aae9ad52..10a7f9bd550e 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._ @@ -505,60 +504,12 @@ object DateTimeUtils { LocalDate.ofEpochDay(date).getDayOfMonth } - /** - * The number of days for each month (not leap year) - */ - private val monthDays = Array(31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31) - - /** - * Returns the date value for the first day of the given month. - * The month is expressed in months since year zero (17999 BC), starting from 0. - */ - private def firstDayOfMonth(absoluteMonth: Int): SQLDate = { - val absoluteYear = absoluteMonth / 12 - var monthInYear = absoluteMonth - absoluteYear * 12 - var date = getDateFromYear(absoluteYear) - if (monthInYear >= 2 && isLeap(absoluteYear + YearZero)) { - date += 1 - } - while (monthInYear > 0) { - date += monthDays(monthInYear - 1) - monthInYear -= 1 - } - date - } - - /** - * Returns the date value for January 1 of the given year. - * The year is expressed in years since year zero (17999 BC), starting from 0. - */ - private def getDateFromYear(absoluteYear: Int): SQLDate = { - val absoluteDays = (absoluteYear * 365 + absoluteYear / 400 - absoluteYear / 100 - + absoluteYear / 4) - absoluteDays - toYearZero - } - /** * Add date and year-month interval. * Returns a date value, expressed in days since 1.1.1970. */ def dateAddMonths(days: SQLDate, months: Int): SQLDate = { - val (year, monthInYear, dayOfMonth, daysToMonthEnd) = splitDate(days) - val absoluteMonth = (year - YearZero) * 12 + monthInYear - 1 + months - val nonNegativeMonth = if (absoluteMonth >= 0) absoluteMonth else 0 - val currentMonthInYear = nonNegativeMonth % 12 - val currentYear = nonNegativeMonth / 12 - - val leapDay = if (currentMonthInYear == 1 && isLeap(currentYear + YearZero)) 1 else 0 - val lastDayOfMonth = monthDays(currentMonthInYear) + leapDay - - val currentDayInMonth = if (daysToMonthEnd == 0 || dayOfMonth >= lastDayOfMonth) { - // last day of the month - lastDayOfMonth - } else { - dayOfMonth - } - firstDayOfMonth(nonNegativeMonth) + currentDayInMonth - 1 + LocalDate.ofEpochDay(days).plusMonths(months).toEpochDay.toInt } /** @@ -569,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 feb3b46df0cd..fbdb1c5f957d 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 @@ -234,6 +234,13 @@ object SQLConf { .booleanConf .createWithDefault(true) + val IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED = + buildConf("spark.sql.inMemoryTableScanStatistics.enable") + .internal() + .doc("When true, enable in-memory table scan accumulators.") + .booleanConf + .createWithDefault(false) + val CACHE_VECTORIZED_READER_ENABLED = buildConf("spark.sql.inMemoryColumnarStorage.enableVectorizedReader") .doc("Enables vectorized reader for columnar caching.") @@ -1024,6 +1031,13 @@ object SQLConf { .booleanConf .createWithDefault(true) + val ENABLE_VECTORIZED_HASH_MAP = + buildConf("spark.sql.codegen.aggregate.map.vectorized.enable") + .internal() + .doc("Enable vectorized aggregate hash map. This is for testing/benchmarking only.") + .booleanConf + .createWithDefault(false) + val MAX_NESTED_VIEW_DEPTH = buildConf("spark.sql.view.maxNestedViewDepth") .internal() @@ -1510,6 +1524,12 @@ object SQLConf { .booleanConf .createWithDefault(false) + val PREFER_INTEGRAL_DIVISION = buildConf("spark.sql.function.preferIntegralDivision") + .doc("When true, will perform integral division with the / operator " + + "if both sides are integral types.") + .booleanConf + .createWithDefault(false) + val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") .internal() @@ -1636,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() @@ -1837,6 +1867,12 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_CTE_PRECEDENCE_ENABLED = buildConf("spark.sql.legacy.ctePrecedence.enabled") + .internal() + .doc("When true, outer CTE definitions takes precedence over inner definitions.") + .booleanConf + .createWithDefault(false) + val LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC = buildConf("spark.sql.legacy.arrayExistsFollowsThreeValuedLogic") .doc("When true, the ArrayExists will follow the three-valued boolean logic.") @@ -2109,6 +2145,8 @@ class SQLConf extends Serializable with Logging { def inMemoryPartitionPruning: Boolean = getConf(IN_MEMORY_PARTITION_PRUNING) + def inMemoryTableScanStatisticsEnabled: Boolean = getConf(IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED) + def offHeapColumnVectorEnabled: Boolean = getConf(COLUMN_VECTOR_OFFHEAP_ENABLED) def columnNameOfCorruptRecord: String = getConf(COLUMN_NAME_OF_CORRUPT_RECORD) @@ -2148,6 +2186,8 @@ class SQLConf extends Serializable with Logging { def enableTwoLevelAggMap: Boolean = getConf(ENABLE_TWOLEVEL_AGG_MAP) + def enableVectorizedHashMap: Boolean = getConf(ENABLE_VECTORIZED_HASH_MAP) + def useObjectHashAggregation: Boolean = getConf(USE_OBJECT_HASH_AGG) def objectAggSortBasedFallbackThreshold: Int = getConf(OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD) @@ -2270,6 +2310,8 @@ class SQLConf extends Serializable with Logging { def eltOutputAsString: Boolean = getConf(ELT_OUTPUT_AS_STRING) + def preferIntegralDivision: Boolean = getConf(PREFER_INTEGRAL_DIVISION) + def allowCreatingManagedTableUsingNonemptyLocation: Boolean = getConf(ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION) @@ -2283,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 8e297874a0d6..ea94cf626698 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 1bf322af2179..a5d1a72d62d5 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/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index edf8d2c1b31a..236f73ba3832 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -310,20 +310,46 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru /** * Returns a field in this struct and its child structs. * - * This does not support finding fields nested in maps or arrays. + * If includeCollections is true, this will return fields that are nested in maps and arrays. */ - private[sql] def findNestedField(fieldNames: Seq[String]): Option[StructField] = { + private[sql] def findNestedField( + fieldNames: Seq[String], + includeCollections: Boolean = false): Option[StructField] = { fieldNames.headOption.flatMap(nameToField.get) match { case Some(field) => - if (fieldNames.tail.isEmpty) { - Some(field) - } else { - field.dataType match { - case struct: StructType => - struct.findNestedField(fieldNames.tail) - case _ => - None - } + (fieldNames.tail, field.dataType, includeCollections) match { + case (Seq(), _, _) => + Some(field) + + case (names, struct: StructType, _) => + struct.findNestedField(names, includeCollections) + + case (_, _, false) => + None // types nested in maps and arrays are not used + + case (Seq("key"), MapType(keyType, _, _), true) => + // return the key type as a struct field to include nullability + Some(StructField("key", keyType, nullable = false)) + + case (Seq("key", names @ _*), MapType(struct: StructType, _, _), true) => + struct.findNestedField(names, includeCollections) + + case (Seq("value"), MapType(_, valueType, isNullable), true) => + // return the value type as a struct field to include nullability + Some(StructField("value", valueType, nullable = isNullable)) + + case (Seq("value", names @ _*), MapType(_, struct: StructType, _), true) => + struct.findNestedField(names, includeCollections) + + case (Seq("element"), ArrayType(elementType, isNullable), true) => + // return the element type as a struct field to include nullability + Some(StructField("element", elementType, nullable = isNullable)) + + case (Seq("element", names @ _*), ArrayType(struct: StructType, _), true) => + struct.findNestedField(names, includeCollections) + + case _ => + None } case _ => None 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 725764755c62..4440ac9e281c 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/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 2c3ba1b0daf4..949bb30d1550 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1126,14 +1126,14 @@ class TypeCoercionSuite extends AnalysisTest { Concat(Seq(Cast(Literal(new java.sql.Date(0)), StringType), Cast(Literal(new Timestamp(0)), StringType)))) - withSQLConf("spark.sql.function.concatBinaryAsString" -> "true") { + withSQLConf(SQLConf.CONCAT_BINARY_AS_STRING.key -> "true") { ruleTest(rule, Concat(Seq(Literal("123".getBytes), Literal("456".getBytes))), Concat(Seq(Cast(Literal("123".getBytes), StringType), Cast(Literal("456".getBytes), StringType)))) } - withSQLConf("spark.sql.function.concatBinaryAsString" -> "false") { + withSQLConf(SQLConf.CONCAT_BINARY_AS_STRING.key -> "false") { ruleTest(rule, Concat(Seq(Literal("123".getBytes), Literal("456".getBytes))), Concat(Seq(Literal("123".getBytes), Literal("456".getBytes)))) @@ -1180,14 +1180,14 @@ class TypeCoercionSuite extends AnalysisTest { Elt(Seq(Literal(2), Cast(Literal(new java.sql.Date(0)), StringType), Cast(Literal(new Timestamp(0)), StringType)))) - withSQLConf("spark.sql.function.eltOutputAsString" -> "true") { + withSQLConf(SQLConf.ELT_OUTPUT_AS_STRING.key -> "true") { ruleTest(rule, Elt(Seq(Literal(1), Literal("123".getBytes), Literal("456".getBytes))), Elt(Seq(Literal(1), Cast(Literal("123".getBytes), StringType), Cast(Literal("456".getBytes), StringType)))) } - withSQLConf("spark.sql.function.eltOutputAsString" -> "false") { + withSQLConf(SQLConf.ELT_OUTPUT_AS_STRING.key -> "false") { ruleTest(rule, Elt(Seq(Literal(1), Literal("123".getBytes), Literal("456".getBytes))), Elt(Seq(Literal(1), Literal("123".getBytes), Literal("456".getBytes)))) @@ -1456,7 +1456,7 @@ class TypeCoercionSuite extends AnalysisTest { test("SPARK-15776 Divide expression's dataType should be casted to Double or Decimal " + "in aggregation function like sum") { - val rules = Seq(FunctionArgumentConversion, Division) + val rules = Seq(FunctionArgumentConversion, Division(conf)) // Casts Integer to Double ruleTest(rules, sum(Divide(4, 3)), sum(Divide(Cast(4, DoubleType), Cast(3, DoubleType)))) // Left expression is Double, right expression is Int. Another rule ImplicitTypeCasts will @@ -1475,12 +1475,35 @@ class TypeCoercionSuite extends AnalysisTest { } test("SPARK-17117 null type coercion in divide") { - val rules = Seq(FunctionArgumentConversion, Division, ImplicitTypeCasts) + val rules = Seq(FunctionArgumentConversion, Division(conf), ImplicitTypeCasts) val nullLit = Literal.create(null, NullType) ruleTest(rules, Divide(1L, nullLit), Divide(Cast(1L, DoubleType), Cast(nullLit, DoubleType))) ruleTest(rules, Divide(nullLit, 1L), Divide(Cast(nullLit, DoubleType), Cast(1L, DoubleType))) } + test("SPARK-28395 Division operator support integral division") { + val rules = Seq(FunctionArgumentConversion, Division(conf)) + Seq(true, false).foreach { preferIntegralDivision => + withSQLConf(SQLConf.PREFER_INTEGRAL_DIVISION.key -> s"$preferIntegralDivision") { + val result1 = if (preferIntegralDivision) { + IntegralDivide(1L, 1L) + } else { + Divide(Cast(1L, DoubleType), Cast(1L, DoubleType)) + } + ruleTest(rules, Divide(1L, 1L), result1) + val result2 = if (preferIntegralDivision) { + IntegralDivide(1, Cast(1, ShortType)) + } else { + Divide(Cast(1, DoubleType), Cast(Cast(1, ShortType), DoubleType)) + } + ruleTest(rules, Divide(1, Cast(1, ShortType)), result2) + + ruleTest(rules, Divide(1L, 1D), Divide(Cast(1L, DoubleType), Cast(1D, DoubleType))) + ruleTest(rules, Divide(Decimal(1.1), 1L), Divide(Decimal(1.1), 1L)) + } + } + } + test("binary comparison with string promotion") { val rule = TypeCoercion.PromoteStrings(conf) ruleTest(rule, @@ -1498,7 +1521,7 @@ class TypeCoercionSuite extends AnalysisTest { DoubleType))) Seq(true, false).foreach { convertToTS => withSQLConf( - "spark.sql.legacy.compareDateTimestampInTimestamp" -> convertToTS.toString) { + SQLConf.COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP.key -> convertToTS.toString) { val date0301 = Literal(java.sql.Date.valueOf("2017-03-01")) val timestamp0301000000 = Literal(Timestamp.valueOf("2017-03-01 00:00:00")) val timestamp0301000001 = Literal(Timestamp.valueOf("2017-03-01 00:00:01")) 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 f4feeca1d05a..9380c7e3f5f7 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 04bb61a7486e..b4110afd5505 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 @@ -462,13 +462,19 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(AddMonths(Literal.create(null, DateType), Literal(1)), null) checkEvaluation(AddMonths(Literal.create(null, DateType), Literal.create(null, IntegerType)), null) + // Valid range of DateType is [0001-01-01, 9999-12-31] + val maxMonthInterval = 10000 * 12 checkEvaluation( - AddMonths(Literal(Date.valueOf("2015-01-30")), Literal(Int.MinValue)), -7293498) + AddMonths(Literal(Date.valueOf("0001-01-01")), Literal(maxMonthInterval)), 2933261) checkEvaluation( - AddMonths(Literal(Date.valueOf("2016-02-28")), positiveIntLit), 1014213) - checkEvaluation( - AddMonths(Literal(Date.valueOf("2016-02-28")), negativeIntLit), -980528) - checkConsistencyBetweenInterpretedAndCodegen(AddMonths, DateType, IntegerType) + AddMonths(Literal(Date.valueOf("9999-12-31")), Literal(-1 * maxMonthInterval)), -719529) + // Test evaluation results between Interpreted mode and Codegen mode + forAll ( + LiteralGenerator.randomGen(DateType), + LiteralGenerator.monthIntervalLiterGen + ) { (l1: Literal, l2: Literal) => + cmpInterpretWithCodegen(EmptyRow, AddMonths(l1, l2)) + } } test("months_between") { @@ -912,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/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index 1c91adab7137..a2c0ce35df23 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -398,7 +398,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa } } - private def cmpInterpretWithCodegen(inputRow: InternalRow, expr: Expression): Unit = { + def cmpInterpretWithCodegen(inputRow: InternalRow, expr: Expression): Unit = { val interpret = try { evaluateWithoutCodegen(expr, inputRow) } catch { 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 b190d6f5caa1..f8400a590606 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/LiteralGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala index be5fdb5b42ea..b111797c3588 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} +import java.time.{Duration, Instant, LocalDate} +import java.util.concurrent.TimeUnit import org.scalacheck.{Arbitrary, Gen} @@ -100,23 +102,44 @@ object LiteralGenerator { lazy val booleanLiteralGen: Gen[Literal] = for { b <- Arbitrary.arbBool.arbitrary } yield Literal.create(b, BooleanType) - lazy val dateLiteralGen: Gen[Literal] = - for { d <- Arbitrary.arbInt.arbitrary } yield Literal.create(new Date(d), DateType) + lazy val dateLiteralGen: Gen[Literal] = { + // Valid range for DateType is [0001-01-01, 9999-12-31] + val minDay = LocalDate.of(1, 1, 1).toEpochDay + val maxDay = LocalDate.of(9999, 12, 31).toEpochDay + for { day <- Gen.choose(minDay, maxDay) } + yield Literal.create(new Date(day * DateTimeUtils.MILLIS_PER_DAY), DateType) + } lazy val timestampLiteralGen: Gen[Literal] = { // Catalyst's Timestamp type stores number of microseconds since epoch in // a variable of Long type. To prevent arithmetic overflow of Long on // conversion from milliseconds to microseconds, the range of random milliseconds // since epoch is restricted here. - val maxMillis = Long.MaxValue / DateTimeUtils.MICROS_PER_MILLIS - val minMillis = Long.MinValue / DateTimeUtils.MICROS_PER_MILLIS + // Valid range for TimestampType is [0001-01-01T00:00:00.000000Z, 9999-12-31T23:59:59.999999Z] + val minMillis = Instant.parse("0001-01-01T00:00:00.000000Z").toEpochMilli + val maxMillis = Instant.parse("9999-12-31T23:59:59.999999Z").toEpochMilli for { millis <- Gen.choose(minMillis, maxMillis) } yield Literal.create(new Timestamp(millis), TimestampType) } - lazy val calendarIntervalLiterGen: Gen[Literal] = - for { m <- Arbitrary.arbInt.arbitrary; s <- Arbitrary.arbLong.arbitrary} - yield Literal.create(new CalendarInterval(m, s), CalendarIntervalType) + // Valid range for DateType and TimestampType is [0001-01-01, 9999-12-31] + private val maxIntervalInMonths: Int = 10000 * 12 + + lazy val monthIntervalLiterGen: Gen[Literal] = { + for { months <- Gen.choose(-1 * maxIntervalInMonths, maxIntervalInMonths) } + yield Literal.create(months, IntegerType) + } + + lazy val calendarIntervalLiterGen: Gen[Literal] = { + val maxDurationInSec = Duration.between( + Instant.parse("0001-01-01T00:00:00.000000Z"), + Instant.parse("9999-12-31T23:59:59.999999Z")).getSeconds + val maxMicros = TimeUnit.SECONDS.toMicros(maxDurationInSec) + for { + months <- Gen.choose(-1 * maxIntervalInMonths, maxIntervalInMonths) + micros <- Gen.choose(-1 * maxMicros, maxMicros) + } yield Literal.create(new CalendarInterval(months, micros), CalendarIntervalType) + } // Sometimes, it would be quite expensive when unlimited value is used, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala index 48105571b279..4c048f79741b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala @@ -199,6 +199,18 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Sinh, DoubleType) } + test("asinh") { + testUnary(Asinh, (x: Double) => math.log(x + math.sqrt(x * x + 1.0))) + checkConsistencyBetweenInterpretedAndCodegen(Asinh, DoubleType) + + checkEvaluation(Asinh(Double.NegativeInfinity), Double.NegativeInfinity) + + val nullLit = Literal.create(null, NullType) + val doubleNullLit = Literal.create(null, DoubleType) + checkEvaluation(checkDataTypeAndCast(Asinh(nullLit)), null, EmptyRow) + checkEvaluation(checkDataTypeAndCast(Asinh(doubleNullLit)), null, EmptyRow) + } + test("cos") { testUnary(Cos, math.cos) checkConsistencyBetweenInterpretedAndCodegen(Cos, DoubleType) @@ -215,6 +227,16 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Cosh, DoubleType) } + test("acosh") { + testUnary(Acosh, (x: Double) => math.log(x + math.sqrt(x * x - 1.0))) + checkConsistencyBetweenInterpretedAndCodegen(Cosh, DoubleType) + + val nullLit = Literal.create(null, NullType) + val doubleNullLit = Literal.create(null, DoubleType) + checkEvaluation(checkDataTypeAndCast(Acosh(nullLit)), null, EmptyRow) + checkEvaluation(checkDataTypeAndCast(Acosh(doubleNullLit)), null, EmptyRow) + } + test("tan") { testUnary(Tan, math.tan) checkConsistencyBetweenInterpretedAndCodegen(Tan, DoubleType) @@ -244,6 +266,16 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Tanh, DoubleType) } + test("atanh") { + testUnary(Atanh, (x: Double) => 0.5 * math.log((1.0 + x) / (1.0 - x))) + checkConsistencyBetweenInterpretedAndCodegen(Atanh, DoubleType) + + val nullLit = Literal.create(null, NullType) + val doubleNullLit = Literal.create(null, DoubleType) + checkEvaluation(checkDataTypeAndCast(Atanh(nullLit)), null, EmptyRow) + checkEvaluation(checkDataTypeAndCast(Atanh(doubleNullLit)), null, EmptyRow) + } + test("toDegrees") { testUnary(ToDegrees, math.toDegrees) checkConsistencyBetweenInterpretedAndCodegen(ToDegrees, DoubleType) 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 df92fa3475bd..981ef57c051f 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 78ae13132864..75ff07637fcc 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/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 2db4667fd056..3ec8d18bc871 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -24,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{BooleanType, IntegerType} import org.apache.spark.unsafe.types.CalendarInterval class FilterPushdownSuite extends PlanTest { @@ -41,9 +42,14 @@ class FilterPushdownSuite extends PlanTest { CollapseProject) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val attrA = 'a.int + val attrB = 'b.int + val attrC = 'c.int + val attrD = 'd.int - val testRelation1 = LocalRelation('d.int) + val testRelation = LocalRelation(attrA, attrB, attrC) + + val testRelation1 = LocalRelation(attrD) // This test already passes. test("eliminate subqueries") { @@ -1202,4 +1208,26 @@ class FilterPushdownSuite extends PlanTest { comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze, checkAnalysis = false) } + + test("SPARK-28345: PythonUDF predicate should be able to pushdown to join") { + val pythonUDFJoinCond = { + val pythonUDF = PythonUDF("pythonUDF", null, + IntegerType, + Seq(attrA), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + pythonUDF === attrD + } + + val query = testRelation.join( + testRelation1, + joinType = Cross).where(pythonUDFJoinCond) + + val expected = testRelation.join( + testRelation1, + joinType = Cross, + condition = Some(pythonUDFJoinCond)).analyze + + comparePlans(Optimize.execute(query.analyze), expected) + } } 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 ab2bd6dff126..2351d8321c5f 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/optimizer/NormalizeFloatingPointNumbersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala index 5f616da2978b..f5af416602c9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.KnownFloatingPointNormalized +import org.apache.spark.sql.catalyst.expressions.{And, IsNull, KnownFloatingPointNormalized} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -78,5 +78,18 @@ class NormalizeFloatingPointNumbersSuite extends PlanTest { comparePlans(doubleOptimized, correctAnswer) } + + test("normalize floating points in join keys (equal null safe) - idempotence") { + val query = testRelation1.join(testRelation2, condition = Some(a <=> b)) + + val optimized = Optimize.execute(query) + val doubleOptimized = Optimize.execute(optimized) + val joinCond = IsNull(a) === IsNull(b) && + KnownFloatingPointNormalized(NormalizeNaNAndZero(coalesce(a, 0.0))) === + KnownFloatingPointNormalized(NormalizeNaNAndZero(coalesce(b, 0.0))) + val correctAnswer = testRelation1.join(testRelation2, condition = Some(joinCond)) + + comparePlans(doubleOptimized, correctAnswer) + } } 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 4c318077ced4..931123cf5451 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,12 +19,24 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale +<<<<<<< HEAD import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} +||||||| merged common ancestors +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} +>>>>>>> origin/master 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 +<<<<<<< HEAD import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} +||||||| merged common ancestors +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} +>>>>>>> origin/master import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -48,82 +60,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 ( @@ -136,154 +137,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'"), @@ -294,31 +292,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) } } @@ -353,27 +364,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) } } @@ -404,6 +416,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") { @@ -594,6 +628,7 @@ class DDLParserSuite extends AnalysisTest { Seq(Seq("x"), Seq("y"), Seq("a", "b", "c")))) } } +<<<<<<< HEAD test("describe table column") { comparePlans(parsePlan("DESCRIBE t col"), @@ -635,4 +670,72 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsePlan("describe table formatted t"), DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) } +||||||| merged common ancestors +======= + + 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}.") + } + } + } +>>>>>>> origin/master } 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 5394732f41f2..6e2a8427659c 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 c77c9aec6887..8ff691fb17f2 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") { @@ -359,20 +358,20 @@ class DateTimeUtilsSuite extends SparkFunSuite { test("date add months") { val input = days(1997, 2, 28, 10, 30) - assert(dateAddMonths(input, 36) === days(2000, 2, 29)) - assert(dateAddMonths(input, -13) === days(1996, 1, 31)) + assert(dateAddMonths(input, 36) === days(2000, 2, 28)) + assert(dateAddMonths(input, -13) === days(1996, 1, 28)) } test("timestamp add months") { val ts1 = date(1997, 2, 28, 10, 30, 0) - val ts2 = date(2000, 2, 29, 10, 30, 0, 123000) - assert(timestampAddInterval(ts1, 36, 123000, defaultTz) === ts2) + val ts2 = date(2000, 2, 28, 10, 30, 0, 123000) + 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, 29, 0, 0, 0, 123000, TimeZoneGMT) - assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST) === ts4) - assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT) === ts5) + val ts5 = date(2000, 2, 28, 0, 0, 0, 123000, TimeZoneGMT) + assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST.toZoneId) === ts4) + assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT.toZoneId) === ts5) } test("monthsBetween") { diff --git a/sql/core/benchmarks/MiscBenchmark-results.txt b/sql/core/benchmarks/MiscBenchmark-results.txt index 85acd5789365..c4ae05209565 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/java/org/apache/spark/sql/streaming/Trigger.java b/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java index fd6f7be2abc5..1bd7b825328d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java +++ b/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java @@ -20,9 +20,10 @@ import java.util.concurrent.TimeUnit; import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger; import scala.concurrent.duration.Duration; -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger; +import org.apache.spark.sql.execution.streaming.ContinuousTrigger; import org.apache.spark.sql.execution.streaming.OneTimeTrigger$; /** @@ -40,7 +41,7 @@ public class Trigger { * @since 2.2.0 */ public static Trigger ProcessingTime(long intervalMs) { - return ProcessingTime.create(intervalMs, TimeUnit.MILLISECONDS); + return ProcessingTimeTrigger.create(intervalMs, TimeUnit.MILLISECONDS); } /** @@ -56,7 +57,7 @@ public static Trigger ProcessingTime(long intervalMs) { * @since 2.2.0 */ public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) { - return ProcessingTime.create(interval, timeUnit); + return ProcessingTimeTrigger.create(interval, timeUnit); } /** @@ -71,7 +72,7 @@ public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) { * @since 2.2.0 */ public static Trigger ProcessingTime(Duration interval) { - return ProcessingTime.apply(interval); + return ProcessingTimeTrigger.apply(interval); } /** @@ -84,7 +85,7 @@ public static Trigger ProcessingTime(Duration interval) { * @since 2.2.0 */ public static Trigger ProcessingTime(String interval) { - return ProcessingTime.apply(interval); + return ProcessingTimeTrigger.apply(interval); } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index a3cbea9021f2..0da52d432d25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -520,6 +520,71 @@ class KeyValueGroupedDataset[K, V] private[sql]( col4: TypedColumn[V, U4]): Dataset[(K, U1, U2, U3, U4)] = aggUntyped(col1, col2, col3, col4).asInstanceOf[Dataset[(K, U1, U2, U3, U4)]] + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5]): Dataset[(K, U1, U2, U3, U4, U5)] = + aggUntyped(col1, col2, col3, col4, col5).asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5)]] + + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5, U6]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5], + col6: TypedColumn[V, U6]): Dataset[(K, U1, U2, U3, U4, U5, U6)] = + aggUntyped(col1, col2, col3, col4, col5, col6) + .asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5, U6)]] + + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5, U6, U7]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5], + col6: TypedColumn[V, U6], + col7: TypedColumn[V, U7]): Dataset[(K, U1, U2, U3, U4, U5, U6, U7)] = + aggUntyped(col1, col2, col3, col4, col5, col6, col7) + .asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5, U6, U7)]] + + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5, U6, U7, U8]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5], + col6: TypedColumn[V, U6], + col7: TypedColumn[V, U7], + col8: TypedColumn[V, U8]): Dataset[(K, U1, U2, U3, U4, U5, U6, U7, U8)] = + aggUntyped(col1, col2, col3, col4, col5, col6, col7, col8) + .asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5, U6, U7, U8)]] + /** * Returns a [[Dataset]] that contains a tuple with each key and the number of items present * for that key. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index f0ef6e19b0aa..bb05c76cfee6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -142,16 +142,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs]]" val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") val version = if (i == 0) "2.3.0" else "1.3.0" - val funcCall = if (i == 0) "() => func" else "func" + val funcCall = if (i == 0) s"() => f$anyCast.call($anyParams)" else s"f$anyCast.call($anyParams)" println(s""" |/** | * Register a deterministic Java UDF$i instance as user-defined function (UDF). | * @since $version | */ |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { - | val func = f$anyCast.call($anyParams) + | val func = $funcCall | def builder(e: Seq[Expression]) = if (e.length == $i) { - | ScalaUDF($funcCall, returnType, e, e.map(_ => false), udfName = Some(name)) + | ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $i; Found: " + e.length) @@ -717,9 +717,9 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 2.3.0 */ def register(name: String, f: UDF0[_], returnType: DataType): Unit = { - val func = f.asInstanceOf[UDF0[Any]].call() + val func = () => f.asInstanceOf[UDF0[Any]].call() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(() => func, returnType, e, e.map(_ => false), udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) 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 c8531e9a046a..1583b8d3a1f9 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/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index c4d5a2b9b3f0..550094193644 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -703,7 +703,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Repartition(numPartitions, shuffle, child) => if (shuffle) { - ShuffleExchangeExec(RoundRobinPartitioning(numPartitions), planLater(child)) :: Nil + ShuffleExchangeExec(RoundRobinPartitioning(numPartitions), + planLater(child), canChangeNumPartitions = false) :: Nil } else { execution.CoalesceExec(numPartitions, planLater(child)) :: Nil } @@ -736,7 +737,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r: logical.Range => execution.RangeExec(r) :: Nil case r: logical.RepartitionByExpression => - exchange.ShuffleExchangeExec(r.partitioning, planLater(r.child)) :: Nil + exchange.ShuffleExchangeExec( + r.partitioning, planLater(r.child), canChangeNumPartitions = false) :: Nil case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index d93eb76b9fbc..78923433eaab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -61,12 +61,18 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // If not all leaf nodes are query stages, it's not safe to reduce the number of // shuffle partitions, because we may break the assumption that all children of a spark plan // have same number of output partitions. + return plan + } + + val shuffleStages = plan.collect { + case stage: ShuffleQueryStageExec => stage + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => stage + } + // ShuffleExchanges introduced by repartition do not support changing the number of partitions. + // We change the number of partitions in the stage only if all the ShuffleExchanges support it. + if (!shuffleStages.forall(_.plan.canChangeNumPartitions)) { plan } else { - val shuffleStages = plan.collect { - case stage: ShuffleQueryStageExec => stage - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => stage - } val shuffleMetrics = shuffleStages.map { stage => val metricsFuture = stage.mapOutputStatisticsFuture assert(metricsFuture.isCompleted, "ShuffleQueryStageExec should already be ready") @@ -76,12 +82,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // `ShuffleQueryStageExec` gives null mapOutputStatistics when the input RDD has 0 partitions, // we should skip it when calculating the `partitionStartIndices`. val validMetrics = shuffleMetrics.filter(_ != null) - // We may get different pre-shuffle partition number if user calls repartition manually. - // We don't reduce shuffle partition number in that case. - val distinctNumPreShufflePartitions = - validMetrics.map(stats => stats.bytesByPartitionId.length).distinct - - if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { + if (validMetrics.nonEmpty) { val partitionStartIndices = estimatePartitionStartIndices(validMetrics.toArray) // This transformation adds new nodes, so we must use `transformUp` here. plan.transformUp { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 25ff6584360e..4a95f7638133 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DecimalType, StringType, StructType} import org.apache.spark.unsafe.KVIterator import org.apache.spark.util.Utils @@ -559,7 +560,7 @@ case class HashAggregateExec( private def enableTwoLevelHashMap(ctx: CodegenContext): Unit = { if (!checkIfFastHashMapSupported(ctx)) { if (modes.forall(mode => mode == Partial || mode == PartialMerge) && !Utils.isTesting) { - logInfo("spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but" + logInfo(s"${SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key} is set to true, but" + " current version of codegened fast hashmap does not support this aggregate.") } } else { @@ -567,8 +568,7 @@ case class HashAggregateExec( // This is for testing/benchmarking only. // We enforce to first level to be a vectorized hashmap, instead of the default row-based one. - isVectorizedHashMapEnabled = sqlContext.getConf( - "spark.sql.codegen.aggregate.map.vectorized.enable", "false") == "true" + isVectorizedHashMapEnabled = sqlContext.conf.enableVectorizedHashMap } } @@ -576,12 +576,8 @@ case class HashAggregateExec( val initAgg = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "initAgg") if (sqlContext.conf.enableTwoLevelAggMap) { enableTwoLevelHashMap(ctx) - } else { - sqlContext.getConf("spark.sql.codegen.aggregate.map.vectorized.enable", null) match { - case "true" => - logWarning("Two level hashmap is disabled but vectorized hashmap is enabled.") - case _ => - } + } else if (sqlContext.conf.enableVectorizedHashMap) { + logWarning("Two level hashmap is disabled but vectorized hashmap is enabled.") } val bitMaxCapacity = sqlContext.conf.fastHashAggregateRowMaxCapacityBit 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 1de2b6e0a85d..b77f90d19b62 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/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 7a8c6d65c932..3566ab1aa5a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -297,8 +297,7 @@ case class InMemoryTableScanExec( } } - lazy val enableAccumulatorsForTest: Boolean = - sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled // Accumulators used for testing purposes lazy val readPartitions = sparkContext.longAccumulator 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 45c62b467657..39b08e2894dc 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 33778a4225a0..0a43a0138b57 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,16 @@ 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} +<<<<<<< HEAD import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan} import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} +||||||| merged common ancestors +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} +>>>>>>> origin/master import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand} import org.apache.spark.sql.execution.datasources.v2.{CatalogTableAsV2, DataSourceV2Relation} @@ -94,6 +102,7 @@ case class DataSourceResolution( convertCTAS(v2SessionCatalog.asTableCatalog, identifier, create) } +<<<<<<< HEAD case DescribeColumnStatement( AsTableIdentifier(tableName), colName, isExtended) => DescribeColumnCommand(tableName, colName, isExtended) @@ -113,6 +122,41 @@ case class DataSourceResolution( } DescribeTable(catalog.asTableCatalog, ident, isExtended) +||||||| merged common ancestors +======= + 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) + +>>>>>>> origin/master case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) => DropTable(catalog.asTableCatalog, ident, ifExists) @@ -245,6 +289,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 f29e7869fb27..a1de287b93f9 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/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 8508322f54e8..b9b86adb438e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -613,7 +613,7 @@ class ParquetFilters( } override def keep(value: Binary): Boolean = { - UTF8String.fromBytes(value.getBytes).startsWith( + value != null && UTF8String.fromBytes(value.getBytes).startsWith( UTF8String.fromBytes(strToBinary.getBytes)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala new file mode 100644 index 000000000000..a3fa82b12e93 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.SparkException +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.LeafExecNode + +/** + * Physical plan node for altering a table. + */ +case class AlterTableExec( + catalog: TableCatalog, + ident: Identifier, + changes: Seq[TableChange]) extends LeafExecNode { + + override def output: Seq[Attribute] = Seq.empty + + override protected def doExecute(): RDD[InternalRow] = { + try { + catalog.alterTable(ident, changes: _*) + } catch { + case e: IllegalArgumentException => + throw new SparkException(s"Unsupported table change: ${e.getMessage}", e) + } + + sqlContext.sparkContext.parallelize(Seq.empty, 1) + } +} 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 1da538be21e8..ec64a9ff7391 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,16 @@ 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 +<<<<<<< HEAD import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} +||||||| merged common ancestors +import org.apache.spark.sql.catalyst.plans.logical.{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} +>>>>>>> origin/master import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.datasources.DataSourceStrategy @@ -166,8 +173,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 @@ -206,6 +250,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil + case AlterTable(catalog, ident, _, changes) => + AlterTableExec(catalog, ident, changes) :: Nil + case _ => 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 000000000000..35d86ee2abbb --- /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 6c771ea98832..9f644de1929a 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/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 8184baf50b04..c56a5c015f32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -24,8 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, - SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf /** @@ -94,7 +93,7 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { val defaultPartitioning = distribution.createPartitioning(targetNumPartitions) child match { // If child is an exchange, we replace it with a new one having defaultPartitioning. - case ShuffleExchangeExec(_, c) => ShuffleExchangeExec(defaultPartitioning, c) + case ShuffleExchangeExec(_, c, _) => ShuffleExchangeExec(defaultPartitioning, c) case _ => ShuffleExchangeExec(defaultPartitioning, child) } } @@ -117,25 +116,41 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { } private def reorder( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], + leftKeys: IndexedSeq[Expression], + rightKeys: IndexedSeq[Expression], expectedOrderOfKeys: Seq[Expression], currentOrderOfKeys: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - val leftKeysBuffer = ArrayBuffer[Expression]() - val rightKeysBuffer = ArrayBuffer[Expression]() - val pickedIndexes = mutable.Set[Int]() - val keysAndIndexes = currentOrderOfKeys.zipWithIndex - - expectedOrderOfKeys.foreach(expression => { - val index = keysAndIndexes.find { case (e, idx) => - // As we may have the same key used many times, we need to filter out its occurrence we - // have already used. - e.semanticEquals(expression) && !pickedIndexes.contains(idx) - }.map(_._2).get - pickedIndexes += index - leftKeysBuffer.append(leftKeys(index)) - rightKeysBuffer.append(rightKeys(index)) - }) + if (expectedOrderOfKeys.size != currentOrderOfKeys.size) { + return (leftKeys, rightKeys) + } + + // Build a lookup between an expression and the positions its holds in the current key seq. + val keyToIndexMap = mutable.Map.empty[Expression, mutable.BitSet] + currentOrderOfKeys.zipWithIndex.foreach { + case (key, index) => + keyToIndexMap.getOrElseUpdate(key.canonicalized, mutable.BitSet.empty).add(index) + } + + // Reorder the keys. + val leftKeysBuffer = new ArrayBuffer[Expression](leftKeys.size) + val rightKeysBuffer = new ArrayBuffer[Expression](rightKeys.size) + val iterator = expectedOrderOfKeys.iterator + while (iterator.hasNext) { + // Lookup the current index of this key. + keyToIndexMap.get(iterator.next().canonicalized) match { + case Some(indices) if indices.nonEmpty => + // Take the first available index from the map. + val index = indices.firstKey + indices.remove(index) + + // Add the keys for that index to the reordered keys. + leftKeysBuffer += leftKeys(index) + rightKeysBuffer += rightKeys(index) + case _ => + // The expression cannot be found, or we have exhausted all indices for that expression. + return (leftKeys, rightKeys) + } + } (leftKeysBuffer, rightKeysBuffer) } @@ -145,20 +160,13 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { leftPartitioning: Partitioning, rightPartitioning: Partitioning): (Seq[Expression], Seq[Expression]) = { if (leftKeys.forall(_.deterministic) && rightKeys.forall(_.deterministic)) { - leftPartitioning match { - case HashPartitioning(leftExpressions, _) - if leftExpressions.length == leftKeys.length && - leftKeys.forall(x => leftExpressions.exists(_.semanticEquals(x))) => - reorder(leftKeys, rightKeys, leftExpressions, leftKeys) - - case _ => rightPartitioning match { - case HashPartitioning(rightExpressions, _) - if rightExpressions.length == rightKeys.length && - rightKeys.forall(x => rightExpressions.exists(_.semanticEquals(x))) => - reorder(leftKeys, rightKeys, rightExpressions, rightKeys) - - case _ => (leftKeys, rightKeys) - } + (leftPartitioning, rightPartitioning) match { + case (HashPartitioning(leftExpressions, _), _) => + reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, leftExpressions, leftKeys) + case (_, HashPartitioning(rightExpressions, _)) => + reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, rightExpressions, rightKeys) + case _ => + (leftKeys, rightKeys) } } else { (leftKeys, rightKeys) @@ -191,7 +199,7 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { def apply(plan: SparkPlan): SparkPlan = plan.transformUp { // TODO: remove this after we create a physical operator for `RepartitionByExpression`. - case operator @ ShuffleExchangeExec(upper: HashPartitioning, child) => + case operator @ ShuffleExchangeExec(upper: HashPartitioning, child, _) => child.outputPartitioning match { case lower: HashPartitioning if upper.semanticEquals(lower) => child case _ => operator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 5d0208f1ecc4..fec05a76b451 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -43,7 +43,8 @@ import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordCo */ case class ShuffleExchangeExec( override val outputPartitioning: Partitioning, - child: SparkPlan) extends Exchange { + child: SparkPlan, + canChangeNumPartitions: Boolean = true) extends Exchange { // NOTE: coordinator can be null after serialization/deserialization, // e.g. it can be null on the Executor side diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index 4f352782067c..02bfbc4949b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -81,10 +81,6 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute] outputIterator.flatMap { pickedResult => val unpickledBatch = unpickle.loads(pickedResult) - // `Opcodes.MEMOIZE` of Protocol 4 (Python 3.4+) will store objects in internal map - // of `Unpickler`. This map is cleared when calling `Unpickler.close()`. Pyrolite - // doesn't clear it up, so we manually clear it. - unpickle.close() unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala }.map { result => if (udfs.length == 1) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala index f4c2d02ee942..41521bfae1ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala @@ -32,7 +32,7 @@ case class UserDefinedPythonFunction( pythonEvalType: Int, udfDeterministic: Boolean) { - def builder(e: Seq[Expression]): PythonUDF = { + def builder(e: Seq[Expression]): Expression = { PythonUDF(name, func, dataType, e, pythonEvalType, udfDeterministic) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index fd2638f30469..e7eb2cb558cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.streaming.sources.{RateControlMicroBatchSt import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset => OffsetV2, SparkDataStream} -import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} +import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.util.Clock class MicroBatchExecution( @@ -51,7 +51,7 @@ class MicroBatchExecution( @volatile protected var sources: Seq[SparkDataStream] = Seq.empty private val triggerExecutor = trigger match { - case t: ProcessingTime => ProcessingTimeExecutor(t, triggerClock) + case t: ProcessingTimeTrigger => ProcessingTimeExecutor(t, triggerClock) case OneTimeTrigger => OneTimeExecutor() case _ => throw new IllegalStateException(s"Unknown type of trigger: $trigger") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala index d188566f822b..088471053b6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.internal.Logging -import org.apache.spark.sql.streaming.ProcessingTime import org.apache.spark.util.{Clock, SystemClock} trait TriggerExecutor { @@ -43,10 +42,12 @@ case class OneTimeExecutor() extends TriggerExecutor { /** * A trigger executor that runs a batch every `intervalMs` milliseconds. */ -case class ProcessingTimeExecutor(processingTime: ProcessingTime, clock: Clock = new SystemClock()) +case class ProcessingTimeExecutor( + processingTimeTrigger: ProcessingTimeTrigger, + clock: Clock = new SystemClock()) extends TriggerExecutor with Logging { - private val intervalMs = processingTime.intervalMs + private val intervalMs = processingTimeTrigger.intervalMs require(intervalMs >= 0) override def execute(triggerHandler: () => Boolean): Unit = { 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 4c0db3cb42a8..2bdb3402c14b 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 @@ -17,13 +17,89 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.annotation.{Evolving, Experimental} +import java.util.concurrent.TimeUnit + +import scala.concurrent.duration.Duration + import org.apache.spark.sql.streaming.Trigger +import org.apache.spark.unsafe.types.CalendarInterval + +private object Triggers { + def validate(intervalMs: Long): Unit = { + require(intervalMs >= 0, "the interval of trigger should not be negative") + } + + def convert(interval: String): Long = { + val cal = CalendarInterval.fromCaseInsensitiveString(interval) + if (cal.months > 0) { + throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") + } + TimeUnit.MICROSECONDS.toMillis(cal.microseconds) + } + + def convert(interval: Duration): Long = interval.toMillis + + def convert(interval: Long, unit: TimeUnit): Long = unit.toMillis(interval) +} /** * A [[Trigger]] that processes only one batch of data in a streaming query then terminates * the query. */ -@Experimental -@Evolving -case object OneTimeTrigger extends Trigger +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. + */ +private[sql] case class ProcessingTimeTrigger(intervalMs: Long) extends Trigger { + Triggers.validate(intervalMs) +} + +private[sql] object ProcessingTimeTrigger { + import Triggers._ + + def apply(interval: String): ProcessingTimeTrigger = { + ProcessingTimeTrigger(convert(interval)) + } + + def apply(interval: Duration): ProcessingTimeTrigger = { + ProcessingTimeTrigger(convert(interval)) + } + + def create(interval: String): ProcessingTimeTrigger = { + apply(interval) + } + + def create(interval: Long, unit: TimeUnit): ProcessingTimeTrigger = { + ProcessingTimeTrigger(convert(interval, unit)) + } +} + +/** + * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at + * the specified interval. + */ +private[sql] case class ContinuousTrigger(intervalMs: Long) extends Trigger { + Triggers.validate(intervalMs) +} + +private[sql] object ContinuousTrigger { + import Triggers._ + + def apply(interval: String): ContinuousTrigger = { + ContinuousTrigger(convert(interval)) + } + + def apply(interval: Duration): ContinuousTrigger = { + ContinuousTrigger(convert(interval)) + } + + def create(interval: String): ContinuousTrigger = { + apply(interval) + } + + def create(interval: Long, unit: TimeUnit): ContinuousTrigger = { + ContinuousTrigger(convert(interval, unit)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 509b103faa0d..f6d156ded766 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.streaming.{StreamingRelationV2, _} import org.apache.spark.sql.sources.v2 import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, TableCapability} import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset} -import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} +import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.util.Clock class ContinuousExecution( @@ -93,7 +93,7 @@ class ContinuousExecution( } private val triggerExecutor = trigger match { - case ContinuousTrigger(t) => ProcessingTimeExecutor(ProcessingTime(t), triggerClock) + case ContinuousTrigger(t) => ProcessingTimeExecutor(ProcessingTimeTrigger(t), triggerClock) case _ => throw new IllegalStateException(s"Unsupported type of trigger: $trigger") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index d55f71c7be83..e1b7a8fc283d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -134,8 +134,10 @@ class RateStreamContinuousPartitionReader( nextReadTime += readTimeIncrement try { - while (System.currentTimeMillis < nextReadTime) { - Thread.sleep(nextReadTime - System.currentTimeMillis) + var toWaitMs = nextReadTime - System.currentTimeMillis + while (toWaitMs > 0) { + Thread.sleep(toWaitMs) + toWaitMs = nextReadTime - System.currentTimeMillis } } catch { case _: InterruptedException => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala deleted file mode 100644 index bd343f380603..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.streaming.continuous - -import java.util.concurrent.TimeUnit - -import scala.concurrent.duration.Duration - -import org.apache.spark.annotation.Evolving -import org.apache.spark.sql.streaming.Trigger -import org.apache.spark.unsafe.types.CalendarInterval - -/** - * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at - * the specified interval. - */ -@Evolving -case class ContinuousTrigger(intervalMs: Long) extends Trigger { - require(intervalMs >= 0, "the interval of trigger should not be negative") -} - -private[sql] object ContinuousTrigger { - def apply(interval: String): ContinuousTrigger = { - val cal = CalendarInterval.fromCaseInsensitiveString(interval) - if (cal.months > 0) { - throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") - } - new ContinuousTrigger(TimeUnit.MICROSECONDS.toMillis(cal.microseconds)) - } - - def apply(interval: Duration): ContinuousTrigger = { - ContinuousTrigger(interval.toMillis) - } - - def create(interval: String): ContinuousTrigger = { - apply(interval) - } - - def create(interval: Long, unit: TimeUnit): ContinuousTrigger = { - ContinuousTrigger(unit.toMillis(interval)) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 5fa3fd0a37a6..72a197bdbcfc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3932,7 +3932,7 @@ object functions { val anyTypeArgs = (0 to i).map(_ => "Any").mkString(", ") val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs]]" val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") - val funcCall = if (i == 0) "() => func" else "func" + val funcCall = if (i == 0) s"() => f$anyCast.call($anyParams)" else s"f$anyCast.call($anyParams)" println(s""" |/** | * Defines a Java UDF$i instance as user-defined function (UDF). @@ -3944,8 +3944,8 @@ object functions { | * @since 2.3.0 | */ |def udf(f: UDF$i[$extTypeArgs], returnType: DataType): UserDefinedFunction = { - | val func = f$anyCast.call($anyParams) - | SparkUserDefinedFunction($funcCall, returnType, inputSchemas = Seq.fill($i)(None)) + | val func = $funcCall + | SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill($i)(None)) |}""".stripMargin) } @@ -4145,8 +4145,8 @@ object functions { * @since 2.3.0 */ def udf(f: UDF0[_], returnType: DataType): UserDefinedFunction = { - val func = f.asInstanceOf[UDF0[Any]].call() - SparkUserDefinedFunction(() => func, returnType, inputSchemas = Seq.fill(0)(None)) + val func = () => f.asInstanceOf[UDF0[Any]].call() + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(0)(None)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala index 29500cf2afbc..805f73dee141 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala @@ -30,7 +30,11 @@ private object MsSqlServerDialect extends JdbcDialect { // String is recommend by Microsoft SQL Server for datetimeoffset types in non-MS clients Option(StringType) } else { - None + sqlType match { + case java.sql.Types.SMALLINT => Some(ShortType) + case java.sql.Types.REAL => Some(FloatType) + case _ => None + } } } @@ -39,6 +43,7 @@ private object MsSqlServerDialect extends JdbcDialect { case StringType => Some(JdbcType("NVARCHAR(MAX)", java.sql.Types.NVARCHAR)) case BooleanType => Some(JdbcType("BIT", java.sql.Types.BIT)) case BinaryType => Some(JdbcType("VARBINARY(MAX)", java.sql.Types.VARBINARY)) + case ShortType => Some(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) case _ => None } 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 5be45c973a5f..2645e4c9d528 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/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index d051cf9c1d4a..36104d7a7044 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.execution.streaming.sources._ import org.apache.spark.sql.sources.v2.{SupportsWrite, TableProvider} import org.apache.spark.sql.sources.v2.TableCapability._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala deleted file mode 100644 index 417d698bdbb0..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.streaming - -import java.util.concurrent.TimeUnit - -import scala.concurrent.duration.Duration - -import org.apache.spark.annotation.Evolving -import org.apache.spark.unsafe.types.CalendarInterval - -/** - * A trigger that runs a query periodically based on the processing time. If `interval` is 0, - * the query will run as fast as possible. - * - * Scala Example: - * {{{ - * df.writeStream.trigger(ProcessingTime("10 seconds")) - * - * import scala.concurrent.duration._ - * df.writeStream.trigger(ProcessingTime(10.seconds)) - * }}} - * - * Java Example: - * {{{ - * df.writeStream.trigger(ProcessingTime.create("10 seconds")) - * - * import java.util.concurrent.TimeUnit - * df.writeStream.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) - * }}} - * - * @since 2.0.0 - */ -@Evolving -@deprecated("use Trigger.ProcessingTime(intervalMs)", "2.2.0") -case class ProcessingTime(intervalMs: Long) extends Trigger { - require(intervalMs >= 0, "the interval of trigger should not be negative") -} - -/** - * Used to create [[ProcessingTime]] triggers for [[StreamingQuery]]s. - * - * @since 2.0.0 - */ -@Evolving -@deprecated("use Trigger.ProcessingTime(intervalMs)", "2.2.0") -object ProcessingTime { - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * df.writeStream.trigger(ProcessingTime("10 seconds")) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval) - */ - @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") - def apply(interval: String): ProcessingTime = { - val cal = CalendarInterval.fromCaseInsensitiveString(interval) - if (cal.months > 0) { - throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") - } - new ProcessingTime(TimeUnit.MICROSECONDS.toMillis(cal.microseconds)) - } - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * import scala.concurrent.duration._ - * df.writeStream.trigger(ProcessingTime(10.seconds)) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval) - */ - @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") - def apply(interval: Duration): ProcessingTime = { - new ProcessingTime(interval.toMillis) - } - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * df.writeStream.trigger(ProcessingTime.create("10 seconds")) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval) - */ - @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") - def create(interval: String): ProcessingTime = { - apply(interval) - } - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * import java.util.concurrent.TimeUnit - * df.writeStream.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval, unit) - */ - @deprecated("use Trigger.ProcessingTime(interval, unit)", "2.2.0") - def create(interval: Long, unit: TimeUnit): ProcessingTime = { - new ProcessingTime(unit.toMillis(interval)) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 1705d5624409..abee5f6017df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -31,7 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, ContinuousTrigger} +import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte-legacy.sql b/sql/core/src/test/resources/sql-tests/inputs/cte-legacy.sql new file mode 100644 index 000000000000..2f2606d44d91 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cte-legacy.sql @@ -0,0 +1,115 @@ +create temporary view t as select * from values 0, 1, 2 as t(id); +create temporary view t2 as select * from values 0, 1 as t(id); + +-- CTE legacy substitution +SET spark.sql.legacy.ctePrecedence.enabled=true; + +-- CTE in CTE definition +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t; + +-- CTE in subquery +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +); + +-- CTE in subquery expression +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +); + +-- CTE in CTE definition shadows outer +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2; + +-- CTE in CTE definition shadows outer 2 +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2; + +-- CTE in CTE definition shadows outer 3 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2; + +-- CTE in subquery shadows outer +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +); + +-- CTE in subquery shadows outer 2 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +); + +-- CTE in subquery shadows outer 3 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +); + +-- CTE in subquery expression shadows outer 2 +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer 3 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +); + +-- Clean up +DROP VIEW IF EXISTS t; +DROP VIEW IF EXISTS t2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql index 4e621c68e1ec..fd0d299d7b0f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql @@ -17,19 +17,20 @@ SELECT 1 AS one; SELECT true AS true; -SELECT false AS false; +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT false AS `false`; SELECT boolean('t') AS true; -- [SPARK-27931] Trim the string when cast string type to boolean type -SELECT boolean(' f ') AS false; +SELECT boolean(' f ') AS `false`; SELECT boolean('true') AS true; -- [SPARK-27923] PostgreSQL does not accept 'test' but Spark SQL accepts it and sets it to NULL SELECT boolean('test') AS error; -SELECT boolean('false') AS false; +SELECT boolean('false') AS `false`; -- [SPARK-27923] PostgreSQL does not accept 'foo' but Spark SQL accepts it and sets it to NULL SELECT boolean('foo') AS error; @@ -41,9 +42,9 @@ SELECT boolean('yes') AS true; -- [SPARK-27923] PostgreSQL does not accept 'yeah' but Spark SQL accepts it and sets it to NULL SELECT boolean('yeah') AS error; -SELECT boolean('n') AS false; +SELECT boolean('n') AS `false`; -SELECT boolean('no') AS false; +SELECT boolean('no') AS `false`; -- [SPARK-27923] PostgreSQL does not accept 'nay' but Spark SQL accepts it and sets it to NULL SELECT boolean('nay') AS error; @@ -51,10 +52,10 @@ SELECT boolean('nay') AS error; -- [SPARK-27931] Accept 'on' and 'off' as input for boolean data type SELECT boolean('on') AS true; -SELECT boolean('off') AS false; +SELECT boolean('off') AS `false`; -- [SPARK-27931] Accept unique prefixes thereof -SELECT boolean('of') AS false; +SELECT boolean('of') AS `false`; -- [SPARK-27923] PostgreSQL does not accept 'o' but Spark SQL accepts it and sets it to NULL SELECT boolean('o') AS error; @@ -70,7 +71,7 @@ SELECT boolean('1') AS true; -- [SPARK-27923] PostgreSQL does not accept '11' but Spark SQL accepts it and sets it to NULL SELECT boolean('11') AS error; -SELECT boolean('0') AS false; +SELECT boolean('0') AS `false`; -- [SPARK-27923] PostgreSQL does not accept '000' but Spark SQL accepts it and sets it to NULL SELECT boolean('000') AS error; @@ -82,11 +83,11 @@ SELECT boolean('') AS error; SELECT boolean('t') or boolean('f') AS true; -SELECT boolean('t') and boolean('f') AS false; +SELECT boolean('t') and boolean('f') AS `false`; SELECT not boolean('f') AS true; -SELECT boolean('t') = boolean('f') AS false; +SELECT boolean('t') = boolean('f') AS `false`; SELECT boolean('t') <> boolean('f') AS true; @@ -99,11 +100,11 @@ SELECT boolean('f') < boolean('t') AS true; SELECT boolean('f') <= boolean('t') AS true; -- explicit casts to/from text -SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS false; +SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false`; -- [SPARK-27931] Trim the string when cast to boolean type SELECT boolean(string(' true ')) AS true, - boolean(string(' FALSE')) AS false; -SELECT string(boolean(true)) AS true, string(boolean(false)) AS false; + boolean(string(' FALSE')) AS `false`; +SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false`; -- [SPARK-27923] PostgreSQL does not accept ' tru e ' but Spark SQL accepts it and sets it to NULL SELECT boolean(string(' tru e ')) AS invalid; -- error diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql index 7bb425d3fbe8..6d9c44c67a96 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql @@ -6,9 +6,6 @@ -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/case.sql -- Test the CASE statement -- --- This test suite contains two Cartesian products without using explicit CROSS JOIN syntax. --- Thus, we set spark.sql.crossJoin.enabled to true. -set spark.sql.crossJoin.enabled=true; CREATE TABLE CASE_TBL ( i integer, f double @@ -264,4 +261,3 @@ SELECT CASE DROP TABLE CASE_TBL; DROP TABLE CASE2_TBL; -set spark.sql.crossJoin.enabled=false; 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 e6943803da39..6cd3856e8672 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/float8.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql new file mode 100644 index 000000000000..6f8e3b596e60 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql @@ -0,0 +1,500 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- FLOAT8 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/float8.sql + +CREATE TABLE FLOAT8_TBL(f1 double) USING parquet; + +INSERT INTO FLOAT8_TBL VALUES (' 0.0 '); +INSERT INTO FLOAT8_TBL VALUES ('1004.30 '); +INSERT INTO FLOAT8_TBL VALUES (' -34.84'); +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e+200'); +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e-200'); + +-- [SPARK-28024] Incorrect numeric values when out of range +-- test for underflow and overflow handling +SELECT double('10e400'); +SELECT double('-10e400'); +SELECT double('10e-400'); +SELECT double('-10e-400'); + +-- [SPARK-28061] Support for converting float to binary format +-- test smallest normalized input +-- SELECT float8send('2.2250738585072014E-308'::float8); + +-- [SPARK-27923] Spark SQL insert there bad inputs to NULL +-- bad input +-- INSERT INTO FLOAT8_TBL VALUES (''); +-- INSERT INTO FLOAT8_TBL VALUES (' '); +-- INSERT INTO FLOAT8_TBL VALUES ('xyz'); +-- INSERT INTO FLOAT8_TBL VALUES ('5.0.0'); +-- INSERT INTO FLOAT8_TBL VALUES ('5 . 0'); +-- INSERT INTO FLOAT8_TBL VALUES ('5. 0'); +-- INSERT INTO FLOAT8_TBL VALUES (' - 3'); +-- INSERT INTO FLOAT8_TBL VALUES ('123 5'); + +-- special inputs +SELECT double('NaN'); +-- [SPARK-28060] Double type can not accept some special inputs +SELECT double('nan'); +SELECT double(' NAN '); +SELECT double('infinity'); +SELECT double(' -INFINiTY '); +-- [SPARK-27923] Spark SQL insert there bad special inputs to NULL +-- bad special inputs +SELECT double('N A N'); +SELECT double('NaN x'); +SELECT double(' INFINITY x'); + +SELECT double('Infinity') + 100.0; +-- [SPARK-27768] Infinity, -Infinity, NaN should be recognized in a case insensitive manner +SELECT double('Infinity') / double('Infinity'); +SELECT double('NaN') / double('NaN'); +-- [SPARK-28315] Decimal can not accept NaN as input +SELECT double(decimal('nan')); + +SELECT '' AS five, * FROM FLOAT8_TBL; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <> '1004.3'; + +SELECT '' AS one, f.* FROM FLOAT8_TBL f WHERE f.f1 = '1004.3'; + +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE '1004.3' > f.f1; + +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE f.f1 < '1004.3'; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE '1004.3' >= f.f1; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <= '1004.3'; + +SELECT '' AS three, f.f1, f.f1 * '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 + '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 / '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 - '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark/Hive and exponentiation in Postgres +-- SELECT '' AS one, f.f1 ^ '2.0' AS square_f1 +-- FROM FLOAT8_TBL f where f.f1 = '1004.3'; + +-- [SPARK-28027] Spark SQL does not support prefix operator @ +-- absolute value +-- SELECT '' AS five, f.f1, @f.f1 AS abs_f1 +-- FROM FLOAT8_TBL f; + +-- [SPARK-23906] Support Truncate number +-- truncate +-- SELECT '' AS five, f.f1, trunc(f.f1) AS trunc_f1 +-- FROM FLOAT8_TBL f; + +-- round +SELECT '' AS five, f.f1, round(f.f1) AS round_f1 + FROM FLOAT8_TBL f; + +-- [SPARK-28135] ceil/ceiling/floor returns incorrect values +-- ceil / ceiling +select ceil(f1) as ceil_f1 from float8_tbl f; +select ceiling(f1) as ceiling_f1 from float8_tbl f; + +-- floor +select floor(f1) as floor_f1 from float8_tbl f; + +-- sign +select sign(f1) as sign_f1 from float8_tbl f; + +-- avoid bit-exact output here because operations may not be bit-exact. +-- SET extra_float_digits = 0; + +-- square root +SELECT sqrt(double('64')) AS eight; + +-- [SPARK-28027] Spark SQL does not support prefix operator |/ +-- SELECT |/ float8 '64' AS eight; + +-- SELECT '' AS three, f.f1, |/f.f1 AS sqrt_f1 +-- FROM FLOAT8_TBL f +-- WHERE f.f1 > '0.0'; + +-- power +SELECT power(double('144'), double('0.5')); +SELECT power(double('NaN'), double('0.5')); +SELECT power(double('144'), double('NaN')); +SELECT power(double('NaN'), double('NaN')); +SELECT power(double('-1'), double('NaN')); +-- [SPARK-28135] power returns incorrect values +SELECT power(double('1'), double('NaN')); +SELECT power(double('NaN'), double('0')); + +-- take exp of ln(f.f1) +SELECT '' AS three, f.f1, exp(ln(f.f1)) AS exp_ln_f1 + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +-- [SPARK-28027] Spark SQL does not support prefix operator ||/ +-- cube root +-- SELECT ||/ float8 '27' AS three; + +-- SELECT '' AS five, f.f1, ||/f.f1 AS cbrt_f1 FROM FLOAT8_TBL f; + + +SELECT '' AS five, * FROM FLOAT8_TBL; + +-- UPDATE FLOAT8_TBL +-- SET f1 = FLOAT8_TBL.f1 * '-1' +-- WHERE FLOAT8_TBL.f1 > '0.0'; +-- Update the FLOAT8_TBL to UPDATED_FLOAT8_TBL +CREATE TEMPORARY VIEW UPDATED_FLOAT8_TBL as +SELECT + CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 +FROM FLOAT8_TBL; + +-- [SPARK-27923] Out of range, Spark SQL returns Infinity +SELECT '' AS bad, f.f1 * '1e200' from UPDATED_FLOAT8_TBL f; + +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark/Hive and exponentiation in Postgres +-- SELECT '' AS bad, f.f1 ^ '1e200' from UPDATED_FLOAT8_TBL f; + +-- SELECT 0 ^ 0 + 0 ^ 1 + 0 ^ 0.0 + 0 ^ 0.5; + +-- [SPARK-27923] Cannot take logarithm of zero +-- SELECT '' AS bad, ln(f.f1) from UPDATED_FLOAT8_TBL f where f.f1 = '0.0' ; + +-- [SPARK-27923] Cannot take logarithm of a negative number +-- SELECT '' AS bad, ln(f.f1) from UPDATED_FLOAT8_TBL f where f.f1 < '0.0' ; + +-- [SPARK-28024] Incorrect numeric values when out of range +-- SELECT '' AS bad, exp(f.f1) from UPDATED_FLOAT8_TBL f; + +-- [SPARK-27923] Divide by zero, Spark SQL returns NULL +-- SELECT '' AS bad, f.f1 / '0.0' from UPDATED_FLOAT8_TBL f; + +SELECT '' AS five, * FROM UPDATED_FLOAT8_TBL; + +-- hyperbolic functions +-- we run these with extra_float_digits = 0 too, since different platforms +-- tend to produce results that vary in the last place. +SELECT sinh(double('1')); +SELECT cosh(double('1')); +SELECT tanh(double('1')); +SELECT asinh(double('1')); +SELECT acosh(double('2')); +SELECT atanh(double('0.5')); +-- [SPARK-27768] Infinity, -Infinity, NaN should be recognized in a case insensitive manner +-- test Inf/NaN cases for hyperbolic functions +SELECT sinh(double('Infinity')); +SELECT sinh(double('-Infinity')); +SELECT sinh(double('NaN')); +SELECT cosh(double('Infinity')); +SELECT cosh(double('-Infinity')); +SELECT cosh(double('NaN')); +SELECT tanh(double('Infinity')); +SELECT tanh(double('-Infinity')); +SELECT tanh(double('NaN')); +SELECT asinh(double('Infinity')); +SELECT asinh(double('-Infinity')); +SELECT asinh(double('NaN')); +-- acosh(Inf) should be Inf, but some mingw versions produce NaN, so skip test +SELECT acosh(double('Infinity')); +SELECT acosh(double('-Infinity')); +SELECT acosh(double('NaN')); +SELECT atanh(double('Infinity')); +SELECT atanh(double('-Infinity')); +SELECT atanh(double('NaN')); + +-- RESET extra_float_digits; + +-- [SPARK-28024] Incorrect numeric values when out of range +-- test for over- and underflow +-- INSERT INTO FLOAT8_TBL VALUES ('10e400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('-10e400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('10e-400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('-10e-400'); + +-- maintain external table consistency across platforms +-- delete all values and reinsert well-behaved ones + +TRUNCATE TABLE FLOAT8_TBL; + +INSERT INTO FLOAT8_TBL VALUES ('0.0'); + +INSERT INTO FLOAT8_TBL VALUES ('-34.84'); + +INSERT INTO FLOAT8_TBL VALUES ('-1004.30'); + +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e+200'); + +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e-200'); + +SELECT '' AS five, * FROM FLOAT8_TBL; + +-- [SPARK-28028] Cast numeric to integral type need round +-- [SPARK-28024] Incorrect numeric values when out of range +-- test edge-case coercions to integer +SELECT smallint(double('32767.4')); +SELECT smallint(double('32767.6')); +SELECT smallint(double('-32768.4')); +SELECT smallint(double('-32768.6')); +SELECT int(double('2147483647.4')); +SELECT int(double('2147483647.6')); +SELECT int(double('-2147483648.4')); +SELECT int(double('-2147483648.6')); +SELECT bigint(double('9223372036854773760')); +SELECT bigint(double('9223372036854775807')); +SELECT bigint(double('-9223372036854775808.5')); +SELECT bigint(double('-9223372036854780000')); + +-- [SPARK-28134] Missing Trigonometric Functions +-- test exact cases for trigonometric functions in degrees + +-- SELECT x, +-- sind(x), +-- sind(x) IN (-1,-0.5,0,0.5,1) AS sind_exact +-- FROM (VALUES (0), (30), (90), (150), (180), +-- (210), (270), (330), (360)) AS t(x); + +-- SELECT x, +-- cosd(x), +-- cosd(x) IN (-1,-0.5,0,0.5,1) AS cosd_exact +-- FROM (VALUES (0), (60), (90), (120), (180), +-- (240), (270), (300), (360)) AS t(x); + +-- SELECT x, +-- tand(x), +-- tand(x) IN ('-Infinity'::float8,-1,0, +-- 1,'Infinity'::float8) AS tand_exact, +-- cotd(x), +-- cotd(x) IN ('-Infinity'::float8,-1,0, +-- 1,'Infinity'::float8) AS cotd_exact +-- FROM (VALUES (0), (45), (90), (135), (180), +-- (225), (270), (315), (360)) AS t(x); + +-- SELECT x, +-- asind(x), +-- asind(x) IN (-90,-30,0,30,90) AS asind_exact, +-- acosd(x), +-- acosd(x) IN (0,60,90,120,180) AS acosd_exact +-- FROM (VALUES (-1), (-0.5), (0), (0.5), (1)) AS t(x); + +-- SELECT x, +-- atand(x), +-- atand(x) IN (-90,-45,0,45,90) AS atand_exact +-- FROM (VALUES ('-Infinity'::float8), (-1), (0), (1), +-- ('Infinity'::float8)) AS t(x); + +-- SELECT x, y, +-- atan2d(y, x), +-- atan2d(y, x) IN (-90,0,90,180) AS atan2d_exact +-- FROM (SELECT 10*cosd(a), 10*sind(a) +-- FROM generate_series(0, 360, 90) AS t(a)) AS t(x,y); + +-- We do not support creating types, skip the test below +-- +-- test output (and round-trip safety) of various values. +-- To ensure we're testing what we think we're testing, start with +-- float values specified by bit patterns (as a useful side effect, +-- this means we'll fail on non-IEEE platforms). + +-- create type xfloat8; +-- create function xfloat8in(cstring) returns xfloat8 immutable strict +-- language internal as 'int8in'; +-- create function xfloat8out(xfloat8) returns cstring immutable strict +-- language internal as 'int8out'; +-- create type xfloat8 (input = xfloat8in, output = xfloat8out, like = float8); +-- create cast (xfloat8 as float8) without function; +-- create cast (float8 as xfloat8) without function; +-- create cast (xfloat8 as bigint) without function; +-- create cast (bigint as xfloat8) without function; + +-- float8: seeeeeee eeeeeeee eeeeeeee mmmmmmmm mmmmmmmm(x4) + +-- we don't care to assume the platform's strtod() handles subnormals +-- correctly; those are "use at your own risk". However we do test +-- subnormal outputs, since those are under our control. + +-- with testdata(bits) as (values +-- -- small subnormals +-- (x'0000000000000001'), +-- (x'0000000000000002'), (x'0000000000000003'), +-- (x'0000000000001000'), (x'0000000100000000'), +-- (x'0000010000000000'), (x'0000010100000000'), +-- (x'0000400000000000'), (x'0000400100000000'), +-- (x'0000800000000000'), (x'0000800000000001'), +-- -- these values taken from upstream testsuite +-- (x'00000000000f4240'), +-- (x'00000000016e3600'), +-- (x'0000008cdcdea440'), +-- -- borderline between subnormal and normal +-- (x'000ffffffffffff0'), (x'000ffffffffffff1'), +-- (x'000ffffffffffffe'), (x'000fffffffffffff')) +-- select float8send(flt) as ibits, +-- flt +-- from (select bits::bigint::xfloat8::float8 as flt +-- from testdata +-- offset 0) s; + +-- round-trip tests + +-- with testdata(bits) as (values +-- (x'0000000000000000'), +-- -- smallest normal values +-- (x'0010000000000000'), (x'0010000000000001'), +-- (x'0010000000000002'), (x'0018000000000000'), +-- -- +-- (x'3ddb7cdfd9d7bdba'), (x'3ddb7cdfd9d7bdbb'), (x'3ddb7cdfd9d7bdbc'), +-- (x'3e112e0be826d694'), (x'3e112e0be826d695'), (x'3e112e0be826d696'), +-- (x'3e45798ee2308c39'), (x'3e45798ee2308c3a'), (x'3e45798ee2308c3b'), +-- (x'3e7ad7f29abcaf47'), (x'3e7ad7f29abcaf48'), (x'3e7ad7f29abcaf49'), +-- (x'3eb0c6f7a0b5ed8c'), (x'3eb0c6f7a0b5ed8d'), (x'3eb0c6f7a0b5ed8e'), +-- (x'3ee4f8b588e368ef'), (x'3ee4f8b588e368f0'), (x'3ee4f8b588e368f1'), +-- (x'3f1a36e2eb1c432c'), (x'3f1a36e2eb1c432d'), (x'3f1a36e2eb1c432e'), +-- (x'3f50624dd2f1a9fb'), (x'3f50624dd2f1a9fc'), (x'3f50624dd2f1a9fd'), +-- (x'3f847ae147ae147a'), (x'3f847ae147ae147b'), (x'3f847ae147ae147c'), +-- (x'3fb9999999999999'), (x'3fb999999999999a'), (x'3fb999999999999b'), +-- -- values very close to 1 +-- (x'3feffffffffffff0'), (x'3feffffffffffff1'), (x'3feffffffffffff2'), +-- (x'3feffffffffffff3'), (x'3feffffffffffff4'), (x'3feffffffffffff5'), +-- (x'3feffffffffffff6'), (x'3feffffffffffff7'), (x'3feffffffffffff8'), +-- (x'3feffffffffffff9'), (x'3feffffffffffffa'), (x'3feffffffffffffb'), +-- (x'3feffffffffffffc'), (x'3feffffffffffffd'), (x'3feffffffffffffe'), +-- (x'3fefffffffffffff'), +-- (x'3ff0000000000000'), +-- (x'3ff0000000000001'), (x'3ff0000000000002'), (x'3ff0000000000003'), +-- (x'3ff0000000000004'), (x'3ff0000000000005'), (x'3ff0000000000006'), +-- (x'3ff0000000000007'), (x'3ff0000000000008'), (x'3ff0000000000009'), +-- -- +-- (x'3ff921fb54442d18'), +-- (x'4005bf0a8b14576a'), +-- (x'400921fb54442d18'), +-- -- +-- (x'4023ffffffffffff'), (x'4024000000000000'), (x'4024000000000001'), +-- (x'4058ffffffffffff'), (x'4059000000000000'), (x'4059000000000001'), +-- (x'408f3fffffffffff'), (x'408f400000000000'), (x'408f400000000001'), +-- (x'40c387ffffffffff'), (x'40c3880000000000'), (x'40c3880000000001'), +-- (x'40f869ffffffffff'), (x'40f86a0000000000'), (x'40f86a0000000001'), +-- (x'412e847fffffffff'), (x'412e848000000000'), (x'412e848000000001'), +-- (x'416312cfffffffff'), (x'416312d000000000'), (x'416312d000000001'), +-- (x'4197d783ffffffff'), (x'4197d78400000000'), (x'4197d78400000001'), +-- (x'41cdcd64ffffffff'), (x'41cdcd6500000000'), (x'41cdcd6500000001'), +-- (x'4202a05f1fffffff'), (x'4202a05f20000000'), (x'4202a05f20000001'), +-- (x'42374876e7ffffff'), (x'42374876e8000000'), (x'42374876e8000001'), +-- (x'426d1a94a1ffffff'), (x'426d1a94a2000000'), (x'426d1a94a2000001'), +-- (x'42a2309ce53fffff'), (x'42a2309ce5400000'), (x'42a2309ce5400001'), +-- (x'42d6bcc41e8fffff'), (x'42d6bcc41e900000'), (x'42d6bcc41e900001'), +-- (x'430c6bf52633ffff'), (x'430c6bf526340000'), (x'430c6bf526340001'), +-- (x'4341c37937e07fff'), (x'4341c37937e08000'), (x'4341c37937e08001'), +-- (x'4376345785d89fff'), (x'4376345785d8a000'), (x'4376345785d8a001'), +-- (x'43abc16d674ec7ff'), (x'43abc16d674ec800'), (x'43abc16d674ec801'), +-- (x'43e158e460913cff'), (x'43e158e460913d00'), (x'43e158e460913d01'), +-- (x'4415af1d78b58c3f'), (x'4415af1d78b58c40'), (x'4415af1d78b58c41'), +-- (x'444b1ae4d6e2ef4f'), (x'444b1ae4d6e2ef50'), (x'444b1ae4d6e2ef51'), +-- (x'4480f0cf064dd591'), (x'4480f0cf064dd592'), (x'4480f0cf064dd593'), +-- (x'44b52d02c7e14af5'), (x'44b52d02c7e14af6'), (x'44b52d02c7e14af7'), +-- (x'44ea784379d99db3'), (x'44ea784379d99db4'), (x'44ea784379d99db5'), +-- (x'45208b2a2c280290'), (x'45208b2a2c280291'), (x'45208b2a2c280292'), +-- -- +-- (x'7feffffffffffffe'), (x'7fefffffffffffff'), +-- -- round to even tests (+ve) +-- (x'4350000000000002'), +-- (x'4350000000002e06'), +-- (x'4352000000000003'), +-- (x'4352000000000004'), +-- (x'4358000000000003'), +-- (x'4358000000000004'), +-- (x'435f000000000020'), +-- -- round to even tests (-ve) +-- (x'c350000000000002'), +-- (x'c350000000002e06'), +-- (x'c352000000000003'), +-- (x'c352000000000004'), +-- (x'c358000000000003'), +-- (x'c358000000000004'), +-- (x'c35f000000000020'), +-- -- exercise fixed-point memmoves +-- (x'42dc12218377de66'), +-- (x'42a674e79c5fe51f'), +-- (x'4271f71fb04cb74c'), +-- (x'423cbe991a145879'), +-- (x'4206fee0e1a9e061'), +-- (x'41d26580b487e6b4'), +-- (x'419d6f34540ca453'), +-- (x'41678c29dcd6e9dc'), +-- (x'4132d687e3df217d'), +-- (x'40fe240c9fcb68c8'), +-- (x'40c81cd6e63c53d3'), +-- (x'40934a4584fd0fdc'), +-- (x'405edd3c07fb4c93'), +-- (x'4028b0fcd32f7076'), +-- (x'3ff3c0ca428c59f8'), +-- -- these cases come from the upstream's testsuite +-- -- LotsOfTrailingZeros) +-- (x'3e60000000000000'), +-- -- Regression +-- (x'c352bd2668e077c4'), +-- (x'434018601510c000'), +-- (x'43d055dc36f24000'), +-- (x'43e052961c6f8000'), +-- (x'3ff3c0ca2a5b1d5d'), +-- -- LooksLikePow5 +-- (x'4830f0cf064dd592'), +-- (x'4840f0cf064dd592'), +-- (x'4850f0cf064dd592'), +-- -- OutputLength +-- (x'3ff3333333333333'), +-- (x'3ff3ae147ae147ae'), +-- (x'3ff3be76c8b43958'), +-- (x'3ff3c083126e978d'), +-- (x'3ff3c0c1fc8f3238'), +-- (x'3ff3c0c9539b8887'), +-- (x'3ff3c0ca2a5b1d5d'), +-- (x'3ff3c0ca4283de1b'), +-- (x'3ff3c0ca43db770a'), +-- (x'3ff3c0ca428abd53'), +-- (x'3ff3c0ca428c1d2b'), +-- (x'3ff3c0ca428c51f2'), +-- (x'3ff3c0ca428c58fc'), +-- (x'3ff3c0ca428c59dd'), +-- (x'3ff3c0ca428c59f8'), +-- (x'3ff3c0ca428c59fb'), +-- -- 32-bit chunking +-- (x'40112e0be8047a7d'), +-- (x'40112e0be815a889'), +-- (x'40112e0be826d695'), +-- (x'40112e0be83804a1'), +-- (x'40112e0be84932ad'), +-- -- MinMaxShift +-- (x'0040000000000000'), +-- (x'007fffffffffffff'), +-- (x'0290000000000000'), +-- (x'029fffffffffffff'), +-- (x'4350000000000000'), +-- (x'435fffffffffffff'), +-- (x'1330000000000000'), +-- (x'133fffffffffffff'), +-- (x'3a6fa7161a4d6e0c') +-- ) +-- select float8send(flt) as ibits, +-- flt, +-- flt::text::float8 as r_flt, +-- float8send(flt::text::float8) as obits, +-- float8send(flt::text::float8) = float8send(flt) as correct +-- from (select bits::bigint::xfloat8::float8 as flt +-- from testdata +-- offset 0) s; + +-- clean up, lest opr_sanity complain +-- drop type xfloat8 cascade; +DROP TABLE FLOAT8_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql index 61f350d3e3f4..f64ec5d75afc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql @@ -88,11 +88,9 @@ WHERE f1 > -32767; SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i; --- PostgreSQL `/` is the same with Spark `div` since SPARK-2659. -SELECT '' AS five, i.f1, i.f1 div smallint('2') AS x FROM INT2_TBL i; +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i; --- PostgreSQL `/` is the same with Spark `div` since SPARK-2659. -SELECT '' AS five, i.f1, i.f1 div int('2') AS x FROM INT2_TBL i; +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i; -- corner cases SELECT string(shiftleft(smallint(-1), 15)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql index cbd587889273..86432a845b6e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql @@ -125,7 +125,8 @@ SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true; SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true; -SELECT int('1000') < int('999') AS false; +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT int('1000') < int('999') AS `false`; -- [SPARK-28027] Our ! and !! has different meanings -- SELECT 4! AS twenty_four; @@ -134,7 +135,6 @@ SELECT int('1000') < int('999') AS false; SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten; --- [SPARK-2659] HiveQL: Division operator should always perform fractional division SELECT 2 + 2 / 2 AS three; SELECT (2 + 2) / 2 AS two; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql index 31eef6f34b1d..d29bf3bfad4c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql @@ -67,10 +67,11 @@ SELECT * FROM INT8_TBL WHERE smallint('123') <= q1; SELECT * FROM INT8_TBL WHERE smallint('123') >= q1; -SELECT '' AS five, q1 AS plus, -q1 AS minus FROM INT8_TBL; +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL; SELECT '' AS five, q1, q2, q1 + q2 AS plus FROM INT8_TBL; -SELECT '' AS five, q1, q2, q1 - q2 AS minus FROM INT8_TBL; +SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL; SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL; SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL WHERE q1 < 1000 or (q2 > 0 and q2 < 1000); @@ -84,7 +85,6 @@ SELECT 37 - q1 AS minus4 FROM INT8_TBL; SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL; SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL; --- [SPARK-2659] HiveQL: Division operator should always perform fractional division -- int8 op int4 SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL; -- int4 op int8 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 000000000000..0fe47a4d583e --- /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.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql new file mode 100644 index 000000000000..1f83d6c41661 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql @@ -0,0 +1,285 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT +-- Test int8 64-bit integers. +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select.sql +-- +create or replace temporary view onek2 as select * from onek; +create or replace temporary view INT8_TBL as select * from values + (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), + (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), + (cast('4567890123456789' as bigint),cast('123' as bigint)), + (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), + (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) + as INT8_TBL(q1, q2); + +-- btree index +-- awk '{if($1<10){print;}else{next;}}' onek.data | sort +0n -1 +-- +SELECT * FROM onek + WHERE onek.unique1 < 10 + ORDER BY onek.unique1; + +-- [SPARK-28010] Support ORDER BY ... USING syntax +-- +-- awk '{if($1<20){print $1,$14;}else{next;}}' onek.data | sort +0nr -1 +-- +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$14;}else{next;}}' onek.data | sort +1d -2 +-- +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 > 980 + ORDER BY stringu1 ASC; + +-- +-- awk '{if($1>980){print $1,$16;}else{next;}}' onek.data | +-- sort +1d -2 +0nr -1 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 ASC, unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$16;}else{next;}}' onek.data | +-- sort +1dr -2 +0n -1 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 DESC, unique1 ASC; + +-- +-- awk '{if($1<20){print $1,$16;}else{next;}}' onek.data | +-- sort +0nr -1 +1d -2 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC, string4 ASC; + +-- +-- awk '{if($1<20){print $1,$16;}else{next;}}' onek.data | +-- sort +0n -1 +1dr -2 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 ASC, string4 DESC; + +-- +-- test partial btree indexes +-- +-- As of 7.2, planner probably won't pick an indexscan without stats, +-- so ANALYZE first. Also, we want to prevent it from picking a bitmapscan +-- followed by sort, because that could hide index ordering problems. +-- +-- ANALYZE onek2; + +-- SET enable_seqscan TO off; +-- SET enable_bitmapscan TO off; +-- SET enable_sort TO off; + +-- +-- awk '{if($1<10){print $0;}else{next;}}' onek.data | sort +0n -1 +-- +SELECT onek2.* FROM onek2 WHERE onek2.unique1 < 10; + +-- +-- awk '{if($1<20){print $1,$14;}else{next;}}' onek.data | sort +0nr -1 +-- +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 < 20 + ORDER BY unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$14;}else{next;}}' onek.data | sort +1d -2 +-- +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 > 980; + +-- RESET enable_seqscan; +-- RESET enable_bitmapscan; +-- RESET enable_sort; + +-- [SPARK-28329] SELECT INTO syntax +-- SELECT two, stringu1, ten, string4 +-- INTO TABLE tmp +-- FROM onek; +CREATE TABLE tmp USING parquet AS +SELECT two, stringu1, ten, string4 +FROM onek; + +-- Skip the person table because there is a point data type that we don't support. +-- +-- awk '{print $1,$2;}' person.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - emp.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=2){print $4,$5;}else{print;}}' - stud_emp.data +-- +-- SELECT name, age FROM person*; ??? check if different +-- SELECT p.name, p.age FROM person* p; + +-- +-- awk '{print $1,$2;}' person.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - emp.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=1){print $4,$5;}else{print;}}' - stud_emp.data | +-- sort +1nr -2 +-- +-- SELECT p.name, p.age FROM person* p ORDER BY age DESC, name; + +-- [SPARK-28330] Enhance query limit +-- +-- Test some cases involving whole-row Var referencing a subquery +-- +select foo.* from (select 1) as foo; +select foo.* from (select null) as foo; +select foo.* from (select 'xyzzy',1,null) as foo; + +-- +-- Test VALUES lists +-- +select * from onek, values(147, 'RFAAAA'), (931, 'VJAAAA') as v (i, j) + WHERE onek.unique1 = v.i and onek.stringu1 = v.j; + +-- [SPARK-28296] Improved VALUES support +-- a more complex case +-- looks like we're coding lisp :-) +-- select * from onek, +-- (values ((select i from +-- (values(10000), (2), (389), (1000), (2000), ((select 10029))) as foo(i) +-- order by i asc limit 1))) bar (i) +-- where onek.unique1 = bar.i; + +-- try VALUES in a subquery +-- select * from onek +-- where (unique1,ten) in (values (1,1), (20,0), (99,9), (17,99)) +-- order by unique1; + +-- VALUES is also legal as a standalone query or a set-operation member +VALUES (1,2), (3,4+4), (7,77.7); + +VALUES (1,2), (3,4+4), (7,77.7) +UNION ALL +SELECT 2+2, 57 +UNION ALL +TABLE int8_tbl; + +-- +-- Test ORDER BY options +-- + +CREATE OR REPLACE TEMPORARY VIEW foo AS +SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1); + +-- [SPARK-28333] NULLS FIRST for DESC and NULLS LAST for ASC +SELECT * FROM foo ORDER BY f1; +SELECT * FROM foo ORDER BY f1 ASC; -- same thing +SELECT * FROM foo ORDER BY f1 NULLS FIRST; +SELECT * FROM foo ORDER BY f1 DESC; +SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- check if indexscans do the right things +-- CREATE INDEX fooi ON foo (f1); +-- SET enable_sort = false; + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- DROP INDEX fooi; +-- CREATE INDEX fooi ON foo (f1 DESC); + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- DROP INDEX fooi; +-- CREATE INDEX fooi ON foo (f1 DESC NULLS LAST); + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- +-- Test planning of some cases with partial indexes +-- + +-- partial index is usable +-- explain (costs off) +-- select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- actually run the query with an analyze to use the partial index +-- explain (costs off, analyze on, timing off, summary off) +-- select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- partial index predicate implies clause, so no need for retest +-- explain (costs off) +-- select * from onek2 where unique2 = 11 and stringu1 < 'B'; +select * from onek2 where unique2 = 11 and stringu1 < 'B'; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +-- but if it's an update target, must retest anyway +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' for update; +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' for update; +-- partial index is not applicable +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'C'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'C'; +-- partial index implies clause, but bitmap scan must recheck predicate anyway +-- SET enable_indexscan TO off; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +-- RESET enable_indexscan; +-- check multi-index cases too +-- explain (costs off) +-- select unique1, unique2 from onek2 +-- where (unique2 = 11 or unique1 = 0) and stringu1 < 'B'; +select unique1, unique2 from onek2 + where (unique2 = 11 or unique1 = 0) and stringu1 < 'B'; +-- explain (costs off) +-- select unique1, unique2 from onek2 +-- where (unique2 = 11 and stringu1 < 'B') or unique1 = 0; +select unique1, unique2 from onek2 + where (unique2 = 11 and stringu1 < 'B') or unique1 = 0; + +-- +-- Test some corner cases that have been known to confuse the planner +-- + +-- ORDER BY on a constant doesn't really need any sorting +SELECT 1 AS x ORDER BY x; + +-- But ORDER BY on a set-valued expression does +-- create function sillysrf(int) returns setof int as +-- 'values (1),(10),(2),($1)' language sql immutable; + +-- select sillysrf(42); +-- select sillysrf(-1) order by 1; + +-- drop function sillysrf(int); + +-- X = X isn't a no-op, it's effectively X IS NOT NULL assuming = is strict +-- (see bug #5084) +select * from (values (2),(null),(1)) v(k) where k = k order by k; +select * from (values (2),(null),(1)) v(k) where k = k; + +-- Test partitioned tables with no partitions, which should be handled the +-- same as the non-inheritance case when expanding its RTE. +-- create table list_parted_tbl (a int,b int) partition by list (a); +-- create table list_parted_tbl1 partition of list_parted_tbl +-- for values in (1) partition by list(b); +-- explain (costs off) select * from list_parted_tbl; +-- drop table list_parted_tbl; +drop table tmp; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql new file mode 100644 index 000000000000..5306028e5bd7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql @@ -0,0 +1,86 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_DISTINCT +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_distinct.sql +-- + +CREATE OR REPLACE TEMPORARY VIEW tmp AS +SELECT two, stringu1, ten, string4 +FROM onek; + +-- +-- awk '{print $3;}' onek.data | sort -n | uniq +-- +SELECT DISTINCT two FROM tmp ORDER BY 1; + +-- +-- awk '{print $5;}' onek.data | sort -n | uniq +-- +SELECT DISTINCT ten FROM tmp ORDER BY 1; + +-- +-- awk '{print $16;}' onek.data | sort -d | uniq +-- +SELECT DISTINCT string4 FROM tmp ORDER BY 1; + +-- [SPARK-28010] Support ORDER BY ... USING syntax +-- +-- awk '{print $3,$16,$5;}' onek.data | sort -d | uniq | +-- sort +0n -1 +1d -2 +2n -3 +-- +-- SELECT DISTINCT two, string4, ten +-- FROM tmp +-- ORDER BY two using <, string4 using <, ten using <; +SELECT DISTINCT two, string4, ten + FROM tmp + ORDER BY two ASC, string4 ASC, ten ASC; + +-- Skip the person table because there is a point data type that we don't support. +-- +-- awk '{print $2;}' person.data | +-- awk '{if(NF!=1){print $2;}else{print;}}' - emp.data | +-- awk '{if(NF!=1){print $2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=1){print $5;}else{print;}}' - stud_emp.data | +-- sort -n -r | uniq +-- +-- SELECT DISTINCT p.age FROM person* p ORDER BY age using >; + +-- +-- Check mentioning same column more than once +-- + +-- EXPLAIN (VERBOSE, COSTS OFF) +-- SELECT count(*) FROM +-- (SELECT DISTINCT two, four, two FROM tenk1) ss; + +SELECT count(*) FROM + (SELECT DISTINCT two, four, two FROM tenk1) ss; + +-- +-- Also, some tests of IS DISTINCT FROM, which doesn't quite deserve its +-- very own regression file. +-- + +CREATE OR REPLACE TEMPORARY VIEW disttable AS SELECT * FROM + (VALUES (1), (2), (3), (NULL)) + AS v(f1); + +-- basic cases +SELECT f1, f1 IS DISTINCT FROM 2 as `not 2` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM NULL as `not null` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM f1 as `false` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM f1+1 as `not null` FROM disttable; + +-- check that optimizer constant-folds it properly +SELECT 1 IS DISTINCT FROM 2 as `yes`; +SELECT 2 IS DISTINCT FROM 2 as `no`; +SELECT 2 IS DISTINCT FROM null as `yes`; +SELECT null IS DISTINCT FROM null as `no`; + +-- negated form +SELECT 1 IS NOT DISTINCT FROM 2 as `no`; +SELECT 2 IS NOT DISTINCT FROM 2 as `yes`; +SELECT 2 IS NOT DISTINCT FROM null as `no`; +SELECT null IS NOT DISTINCT FROM null as `yes`; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql new file mode 100644 index 000000000000..2edde8df0804 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql @@ -0,0 +1,55 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_HAVING +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql +-- + +-- load test data +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet; +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A'); +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b'); +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c'); +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D'); +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e'); +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F'); +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g'); +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h'); +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I'); +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j'); + +SELECT b, c FROM test_having + GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c; + +-- HAVING is effectively equivalent to WHERE in this case +SELECT b, c FROM test_having + GROUP BY b, c HAVING b = 3 ORDER BY b, c; + +-- [SPARK-28386] Cannot resolve ORDER BY columns with GROUP BY and HAVING +-- SELECT lower(c), count(c) FROM test_having +-- GROUP BY lower(c) HAVING count(*) > 2 OR min(a) = max(a) +-- ORDER BY lower(c); + +SELECT c, max(a) FROM test_having + GROUP BY c HAVING count(*) > 2 OR min(a) = max(a) + ORDER BY c; + +-- test degenerate cases involving HAVING without GROUP BY +-- Per SQL spec, these should generate 0 or 1 row, even without aggregates + +SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a); +SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a); + +-- errors: ungrouped column references +SELECT a FROM test_having HAVING min(a) < max(a); +SELECT 1 AS one FROM test_having HAVING a > 1; + +-- the really degenerate case: need not scan table at all +SELECT 1 AS one FROM test_having HAVING 1 > 2; +SELECT 1 AS one FROM test_having HAVING 1 < 2; + +-- and just to prove that we aren't scanning the table: +SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2; + +DROP TABLE test_having; diff --git a/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 000000000000..54b3083a9f4a --- /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 000000000000..02af15a879c8 --- /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/pgSQL/with.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql new file mode 100644 index 000000000000..83c6fd8cbac9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql @@ -0,0 +1,1208 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- WITH +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/with.sql +-- +-- This test uses the generate_series(...) function which is rewritten to EXPLODE(SEQUENCE(...)) as +-- it's feature tracking ticket SPARK-27767 is closed as Won't Do. + +-- +-- Tests for common table expressions (WITH query, ... SELECT ...) +-- + +-- Basic WITH +WITH q1(x,y) AS (SELECT 1,2) +SELECT * FROM q1, q1 AS q2; + +-- Multiple uses are evaluated only once +-- [SPARK-28299] Evaluation of multiple CTE uses +-- [ORIGINAL SQL] +--SELECT count(*) FROM ( +-- WITH q1(x) AS (SELECT random() FROM generate_series(1, 5)) +-- SELECT * FROM q1 +-- UNION +-- SELECT * FROM q1 +--) ss; +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss; + +-- WITH RECURSIVE + +-- sum of 1..100 +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 100 +--) +--SELECT sum(n) FROM t; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT (VALUES(1)) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 5 +--) +--SELECT * FROM t; + +-- recursive view +-- [SPARK-24497] Support recursive SQL query +--CREATE RECURSIVE VIEW nums (n) AS +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM nums WHERE n < 5; +-- +--SELECT * FROM nums; + +-- [SPARK-24497] Support recursive SQL query +--CREATE OR REPLACE RECURSIVE VIEW nums (n) AS +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM nums WHERE n < 6; +-- +--SELECT * FROM nums; + +-- This is an infinite loop with UNION ALL, but not with UNION +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT 1 +--UNION +-- SELECT 10-n FROM t) +--SELECT * FROM t; + +-- This'd be an infinite loop, but outside query reads only as much as needed +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t) +--SELECT * FROM t LIMIT 10; + +-- UNION case should have same property +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT 1 +--UNION +-- SELECT n+1 FROM t) +--SELECT * FROM t LIMIT 10; + +-- Test behavior with an unknown-type literal in the WITH +-- [SPARK-28146] Support IS OF type predicate +--WITH q AS (SELECT 'foo' AS x) +--SELECT x, x IS OF (text) AS is_text FROM q; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28146] Support IS OF type predicate +--WITH RECURSIVE t(n) AS ( +-- SELECT 'foo' +--UNION ALL +-- SELECT n || ' bar' FROM t WHERE length(n) < 20 +--) +--SELECT n, n IS OF (text) AS is_text FROM t; + +-- In a perfect world, this would work and resolve the literal as int ... +-- but for now, we have to be content with resolving to text too soon. +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28146] Support IS OF type predicate +--WITH RECURSIVE t(n) AS ( +-- SELECT '7' +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 10 +--) +--SELECT n, n IS OF (int) AS is_int FROM t; + +-- +-- Some examples with a tree +-- +-- department structure represented here is as follows: +-- +-- ROOT-+->A-+->B-+->C +-- | | +-- | +->D-+->F +-- +->E-+->G + + +-- [ORIGINAL SQL] +--CREATE TEMP TABLE department ( +-- id INTEGER PRIMARY KEY, -- department ID +-- parent_department INTEGER REFERENCES department, -- upper department ID +-- name string -- department name +--); +CREATE TABLE department ( + id INTEGER, -- department ID + parent_department INTEGER, -- upper department ID + name string -- department name +) USING parquet; + +INSERT INTO department VALUES (0, NULL, 'ROOT'); +INSERT INTO department VALUES (1, 0, 'A'); +INSERT INTO department VALUES (2, 1, 'B'); +INSERT INTO department VALUES (3, 2, 'C'); +INSERT INTO department VALUES (4, 2, 'D'); +INSERT INTO department VALUES (5, 0, 'E'); +INSERT INTO department VALUES (6, 4, 'F'); +INSERT INTO department VALUES (7, 5, 'G'); + + +-- extract all departments under 'A'. Result should be A, B, C, D and F +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment AS +--( +-- -- non recursive term +-- SELECT name as root_name, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- extract all departments under 'A' with "level" number +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +--( +-- -- non recursive term +-- SELECT 1, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- extract all departments under 'A' with "level" number. +-- Only shows level 2 or more +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +--( +-- -- non recursive term +-- SELECT 1, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name; + +-- "RECURSIVE" is ignored if the query has no self-reference +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment AS +--( +-- -- note lack of recursive UNION structure +-- SELECT * FROM department WHERE name = 'A' +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- inside subqueries +-- [SPARK-24497] Support recursive SQL query +--SELECT count(*) FROM ( +-- WITH RECURSIVE t(n) AS ( +-- SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 500 +-- ) +-- SELECT * FROM t) AS t WHERE n < ( +-- SELECT count(*) FROM ( +-- WITH RECURSIVE t(n) AS ( +-- SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 100 +-- ) +-- SELECT * FROM t WHERE n < 50000 +-- ) AS t WHERE n < 100); + +-- use same CTE twice at different subquery levels +-- [SPARK-24497] Support recursive SQL query +--WITH q1(x,y) AS ( +-- SELECT hundred, sum(ten) FROM tenk1 GROUP BY hundred +-- ) +--SELECT count(*) FROM q1 WHERE y > (SELECT sum(y)/100 FROM q1 qsub); + +-- via a VIEW +-- [SPARK-24497] Support recursive SQL query +--CREATE TEMPORARY VIEW vsubdepartment AS +-- WITH RECURSIVE subdepartment AS +-- ( +-- -- non recursive term +-- SELECT * FROM department WHERE name = 'A' +-- UNION ALL +-- -- recursive term +-- SELECT d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +-- ) +-- SELECT * FROM subdepartment; +-- +--SELECT * FROM vsubdepartment ORDER BY name; +-- +---- Check reverse listing +--SELECT pg_get_viewdef('vsubdepartment'::regclass); +--SELECT pg_get_viewdef('vsubdepartment'::regclass, true); + +-- Another reverse-listing example +-- [SPARK-24497] Support recursive SQL query +--CREATE VIEW sums_1_100 AS +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 100 +--) +--SELECT sum(n) FROM t; +-- +--\d+ sums_1_100 + +-- corner case in which sub-WITH gets initialized first +-- [SPARK-24497] Support recursive SQL query +--with recursive q as ( +-- select * from department +-- union all +-- (with x as (select * from q) +-- select * from x) +-- ) +--select * from q limit 24; + +-- [SPARK-24497] Support recursive SQL query +--with recursive q as ( +-- select * from department +-- union all +-- (with recursive x as ( +-- select * from department +-- union all +-- (select * from q union all select * from x) +-- ) +-- select * from x) +-- ) +--select * from q limit 32; + +-- recursive term has sub-UNION +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(i,j) AS ( +-- VALUES (1,2) +-- UNION ALL +-- SELECT t2.i, t.j+1 FROM +-- (SELECT 2 AS i UNION ALL SELECT 3 AS i) AS t2 +-- JOIN t ON (t2.i = t.i+1)) +-- +-- SELECT * FROM t; + +-- +-- different tree example +-- +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE tree( +-- id INTEGER PRIMARY KEY, +-- parent_id INTEGER REFERENCES tree(id) +--); +CREATE TABLE tree( + id INTEGER, + parent_id INTEGER +) USING parquet; + +INSERT INTO tree +VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), + (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11); + +-- +-- get all paths from "second level" nodes to leaf nodes +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.*, t2.* FROM t AS t1 JOIN t AS t2 ON +-- (t1.path[1] = t2.path[1] AND +-- array_upper(t1.path,1) = 1 AND +-- array_upper(t2.path,1) > 1) +-- ORDER BY t1.id, t2.id; + +-- just count 'em +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.id, count(t2.*) FROM t AS t1 JOIN t AS t2 ON +-- (t1.path[1] = t2.path[1] AND +-- array_upper(t1.path,1) = 1 AND +-- array_upper(t2.path,1) > 1) +-- GROUP BY t1.id +-- ORDER BY t1.id; + +-- this variant tickled a whole-row-variable bug in 8.4devel +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.id, t2.path, t2 FROM t AS t1 JOIN t AS t2 ON +--(t1.id=t2.id); + +-- +-- test cycle detection +-- +-- [ORIGINAL SQL] +--create temp table graph( f int, t int, label text ); +create table graph( f int, t int, label string ) USING parquet; + +insert into graph values + (1, 2, 'arc 1 -> 2'), + (1, 3, 'arc 1 -> 3'), + (2, 3, 'arc 2 -> 3'), + (1, 4, 'arc 1 -> 4'), + (4, 5, 'arc 4 -> 5'), + (5, 1, 'arc 5 -> 1'); + +-- [SPARK-24497] Support recursive SQL query +--with recursive search_graph(f, t, label, path, cycle) as ( +-- select *, array[row(g.f, g.t)], false from graph g +-- union all +-- select g.*, path || row(g.f, g.t), row(g.f, g.t) = any(path) +-- from graph g, search_graph sg +-- where g.f = sg.t and not cycle +--) +--select * from search_graph; + +-- ordering by the path column has same effect as SEARCH DEPTH FIRST +-- [SPARK-24497] Support recursive SQL query +--with recursive search_graph(f, t, label, path, cycle) as ( +-- select *, array[row(g.f, g.t)], false from graph g +-- union all +-- select g.*, path || row(g.f, g.t), row(g.f, g.t) = any(path) +-- from graph g, search_graph sg +-- where g.f = sg.t and not cycle +--) +--select * from search_graph order by path; + +-- +-- test multiple WITH queries +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- y (id) AS (VALUES (1)), +-- x (id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5) +--SELECT * FROM x; + +-- forward reference OK +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS (values (1)) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM y WHERE id < 10) +-- SELECT y.*, x.* FROM y LEFT JOIN x USING (id); + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 10) +-- SELECT y.*, x.* FROM y LEFT JOIN x USING (id); + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), +-- y(id) AS +-- (SELECT * FROM x UNION ALL SELECT * FROM x), +-- z(id) AS +-- (SELECT * FROM x UNION ALL SELECT id+1 FROM z WHERE id < 10) +-- SELECT * FROM z; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), +-- y(id) AS +-- (SELECT * FROM x UNION ALL SELECT * FROM x), +-- z(id) AS +-- (SELECT * FROM y UNION ALL SELECT id+1 FROM z WHERE id < 10) +-- SELECT * FROM z; + +-- +-- Test WITH attached to a data-modifying statement +-- + +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE y (a INTEGER); +CREATE TABLE y (a INTEGER) USING parquet; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 10); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)); + +-- [SPARK-28147] Support RETURNING clause +--WITH t AS ( +-- SELECT a FROM y +--) +--INSERT INTO y +--SELECT a+20 FROM t RETURNING *; +-- +--SELECT * FROM y; + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- SELECT a FROM y +--) +--UPDATE y SET a = y.a-10 FROM t WHERE y.a > 20 AND t.a = y.a RETURNING y.a; +-- +--SELECT * FROM y; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH RECURSIVE t(a) AS ( +-- SELECT 11 +-- UNION ALL +-- SELECT a+1 FROM t WHERE a < 50 +--) +--DELETE FROM y USING t WHERE t.a = y.a RETURNING y.a; +-- +--SELECT * FROM y; + +DROP TABLE y; + +-- +-- error cases +-- + +-- INTERSECT +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- EXCEPT +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- no non-recursive term +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT n FROM x) +-- SELECT * FROM x; + +-- recursive term in the left hand side (strictly speaking, should allow this) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1) +-- SELECT * FROM x; + +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE y (a INTEGER); +CREATE TABLE y (a INTEGER) USING parquet; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 10); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)); + +-- LEFT JOIN + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM y LEFT JOIN x ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- RIGHT JOIN +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM x RIGHT JOIN y ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- FULL JOIN +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM x FULL JOIN y ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- subquery +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x +-- WHERE n IN (SELECT * FROM x)) +-- SELECT * FROM x; + +-- aggregate functions +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT count(*) FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT sum(n) FROM x) +-- SELECT * FROM x; + +-- ORDER BY +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x ORDER BY 1) +-- SELECT * FROM x; + +-- LIMIT/OFFSET +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x LIMIT 10 OFFSET 1) +-- SELECT * FROM x; + +-- FOR UPDATE +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x FOR UPDATE) +-- SELECT * FROM x; + +-- target list has a recursive query name +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(id) AS (values (1) +-- UNION ALL +-- SELECT (SELECT * FROM x) FROM x WHERE id < 5 +--) SELECT * FROM x; + +-- mutual recursive query (not implemented) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM y WHERE id < 5), +-- y (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 5) +--SELECT * FROM x; + +-- non-linear recursion is not allowed +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- UNION ALL +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- SELECT * FROM +-- (SELECT i+1 FROM foo WHERE i < 10 +-- UNION ALL +-- SELECT i+1 FROM foo WHERE i < 5) AS t +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- EXCEPT +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- INTERSECT +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- Wrong type induced from non-recursive term +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (SELECT i FROM (VALUES(1),(2)) t(i) +-- UNION ALL +-- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) +--SELECT * FROM foo; + +-- rejects different typmod, too (should we allow this?) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (SELECT i::numeric(3,0) FROM (VALUES(1),(2)) t(i) +-- UNION ALL +-- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) +--SELECT * FROM foo; + +-- [NOTE] Spark SQL doesn't support RULEs +-- disallow OLD/NEW reference in CTE +--CREATE TABLE x (n integer) USING parquet; +--CREATE RULE r2 AS ON UPDATE TO x DO INSTEAD +-- WITH t AS (SELECT OLD.*) UPDATE y SET a = t.n FROM t; + +-- +-- test for bug #4902 +-- +-- [SPARK-28296] Improved VALUES support +--with cte(foo) as ( values(42) ) values((select foo from cte)); +with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q; + +-- test CTE referencing an outer-level variable (to see that changed-parameter +-- signaling still works properly after fixing this bug) +-- [SPARK-28296] Improved VALUES support +-- [SPARK-28297] Handling outer links in CTE subquery expressions +--select ( with cte(foo) as ( values(f1) ) +-- select (select foo from cte) ) +--from int4_tbl; + +-- [SPARK-28296] Improved VALUES support +-- [SPARK-28297] Handling outer links in CTE subquery expressions +--select ( with cte(foo) as ( values(f1) ) +-- values((select foo from cte)) ) +--from int4_tbl; + +-- +-- test for nested-recursive-WITH bug +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(j) AS ( +-- WITH RECURSIVE s(i) AS ( +-- VALUES (1) +-- UNION ALL +-- SELECT i+1 FROM s WHERE i < 10 +-- ) +-- SELECT i FROM s +-- UNION ALL +-- SELECT j+1 FROM t WHERE j < 10 +--) +--SELECT * FROM t; + +-- +-- test WITH attached to intermediate-level set operation +-- + +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM innermost + UNION SELECT 3) +) +SELECT * FROM outermost ORDER BY 1; + +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost -- fail + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE outermost(x) AS ( +-- SELECT 1 +-- UNION (WITH innermost as (SELECT 2) +-- SELECT * FROM outermost +-- UNION SELECT * FROM innermost) +--) +--SELECT * FROM outermost ORDER BY 1; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE outermost(x) AS ( +-- WITH innermost as (SELECT 2 FROM outermost) -- fail +-- SELECT * FROM innermost +-- UNION SELECT * from outermost +--) +--SELECT * FROM outermost ORDER BY 1; + +-- +-- This test will fail with the old implementation of PARAM_EXEC parameter +-- assignment, because the "q1" Var passed down to A's targetlist subselect +-- looks exactly like the "A.id" Var passed down to C's subselect, causing +-- the old code to give them the same runtime PARAM_EXEC slot. But the +-- lifespans of the two parameters overlap, thanks to B also reading A. +-- + +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +--with +--A as ( select q2 as id, (select q1) as x from int8_tbl ), +--B as ( select id, row_number() over (partition by id) as r from A ), +--C as ( select A.id, array(select B.id from B where B.id = A.id) from A ) +--select * from C; + +-- +-- Test CTEs read in non-initialization orders +-- + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- tab(id_key,link) AS (VALUES (1,17), (2,17), (3,17), (4,17), (6,17), (5,17)), +-- iter (id_key, row_type, link) AS ( +-- SELECT 0, 'base', 17 +-- UNION ALL ( +-- WITH remaining(id_key, row_type, link, min) AS ( +-- SELECT tab.id_key, 'true'::text, iter.link, MIN(tab.id_key) OVER () +-- FROM tab INNER JOIN iter USING (link) +-- WHERE tab.id_key > iter.id_key +-- ), +-- first_remaining AS ( +-- SELECT id_key, row_type, link +-- FROM remaining +-- WHERE id_key=min +-- ), +-- effect AS ( +-- SELECT tab.id_key, 'new'::text, tab.link +-- FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key +-- WHERE e.row_type = 'false' +-- ) +-- SELECT * FROM first_remaining +-- UNION ALL SELECT * FROM effect +-- ) +-- ) +--SELECT * FROM iter; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- tab(id_key,link) AS (VALUES (1,17), (2,17), (3,17), (4,17), (6,17), (5,17)), +-- iter (id_key, row_type, link) AS ( +-- SELECT 0, 'base', 17 +-- UNION ( +-- WITH remaining(id_key, row_type, link, min) AS ( +-- SELECT tab.id_key, 'true'::text, iter.link, MIN(tab.id_key) OVER () +-- FROM tab INNER JOIN iter USING (link) +-- WHERE tab.id_key > iter.id_key +-- ), +-- first_remaining AS ( +-- SELECT id_key, row_type, link +-- FROM remaining +-- WHERE id_key=min +-- ), +-- effect AS ( +-- SELECT tab.id_key, 'new'::text, tab.link +-- FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key +-- WHERE e.row_type = 'false' +-- ) +-- SELECT * FROM first_remaining +-- UNION ALL SELECT * FROM effect +-- ) +-- ) +--SELECT * FROM iter; + +-- +-- Data-modifying statements in WITH +-- + +-- INSERT ... RETURNING +-- [SPARK-28147] Support RETURNING clause +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (11), +-- (12), +-- (13), +-- (14), +-- (15), +-- (16), +-- (17), +-- (18), +-- (19), +-- (20) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- UPDATE ... RETURNING +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- UPDATE y +-- SET a=a+1 +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- DELETE ... RETURNING +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y +-- WHERE a <= 10 +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- forward reference +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH RECURSIVE t AS ( +-- INSERT INTO y +-- SELECT a+5 FROM t2 WHERE a > 5 +-- RETURNING * +--), t2 AS ( +-- UPDATE y SET a=a-11 RETURNING * +--) +--SELECT * FROM t +--UNION ALL +--SELECT * FROM t2; +-- +--SELECT * FROM y; + +-- unconditional DO INSTEAD rule +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE RULE y_rule AS ON DELETE TO y DO INSTEAD +-- INSERT INTO y VALUES(42) RETURNING *; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +--DROP RULE y_rule ON y; + +-- check merging of outer CTE with CTE in a rule action +--CREATE TEMP TABLE bug6051 AS +-- select i from generate_series(1,3) as t(i); + +--SELECT * FROM bug6051; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) +--INSERT INTO bug6051 SELECT * FROM t1; +-- +--SELECT * FROM bug6051; + +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE TEMP TABLE bug6051_2 (i int); +-- +--CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD +-- INSERT INTO bug6051_2 +-- SELECT NEW.i; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) +--INSERT INTO bug6051 SELECT * FROM t1; +-- +--SELECT * FROM bug6051; +--SELECT * FROM bug6051_2; + +-- a truly recursive CTE in the same list +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(a) AS ( +-- SELECT 0 +-- UNION ALL +-- SELECT a+1 FROM t WHERE a+1 < 5 +--), t2 as ( +-- INSERT INTO y +-- SELECT * FROM t RETURNING * +--) +--SELECT * FROM t2 JOIN y USING (a) ORDER BY a; +-- +--SELECT * FROM y; + +-- data-modifying WITH in a modifying statement +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y +-- WHERE a <= 10 +-- RETURNING * +--) +--INSERT INTO y SELECT -a FROM t RETURNING *; +-- +--SELECT * FROM y; + +-- check that WITH query is run to completion even if outer query isn't +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- UPDATE y SET a = a * 100 RETURNING * +--) +--SELECT * FROM t LIMIT 10; +-- +--SELECT * FROM y; + +-- data-modifying WITH containing INSERT...ON CONFLICT DO UPDATE +-- [ORIGINAL SQL] +--CREATE TABLE withz AS SELECT i AS k, (i || ' v')::text v FROM generate_series(1, 16, 3) i; +CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i || ' v' AS string) v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i); +-- [NOTE] Spark SQL doesn't support UNIQUE constraints +--ALTER TABLE withz ADD UNIQUE (k); + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- INSERT INTO withz SELECT i, 'insert' +-- FROM generate_series(0, 16) i +-- ON CONFLICT (k) DO UPDATE SET v = withz.v || ', now update' +-- RETURNING * +--) +--SELECT * FROM t JOIN y ON t.k = y.a ORDER BY a, k; + +-- Test EXCLUDED.* reference within CTE +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH aa AS ( +-- INSERT INTO withz VALUES(1, 5) ON CONFLICT (k) DO UPDATE SET v = EXCLUDED.v +-- WHERE withz.k != EXCLUDED.k +-- RETURNING * +--) +--SELECT * FROM aa; + +-- New query/snapshot demonstrates side-effects of previous query. +SELECT * FROM withz ORDER BY k; + +-- +-- Ensure subqueries within the update clause work, even if they +-- reference outside values +-- +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = ' update' WHERE withz.k = (SELECT a FROM aa); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); +--WITH aa AS (SELECT 'a' a, 'b' b UNION ALL SELECT 'a' a, 'b' b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 'a' LIMIT 1); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, (SELECT b || ' insert' FROM aa WHERE a = 1 )) +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); + +-- Update a row more than once, in different parts of a wCTE. That is +-- an allowed, presumably very rare, edge case, but since it was +-- broken in the past, having a test seems worthwhile. +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH simpletup AS ( +-- SELECT 2 k, 'Green' v), +--upsert_cte AS ( +-- INSERT INTO withz VALUES(2, 'Blue') ON CONFLICT (k) DO +-- UPDATE SET (k, v) = (SELECT k, v FROM simpletup WHERE simpletup.k = withz.k) +-- RETURNING k, v) +--INSERT INTO withz VALUES(2, 'Red') ON CONFLICT (k) DO +--UPDATE SET (k, v) = (SELECT k, v FROM upsert_cte WHERE upsert_cte.k = withz.k) +--RETURNING k, v; + +DROP TABLE withz; + +-- check that run to completion happens in proper ordering + +TRUNCATE TABLE y; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 3); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)); +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE yy (a INTEGER); +CREATE TABLE yy (a INTEGER) USING parquet; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28147] Support RETURNING clause +--WITH RECURSIVE t1 AS ( +-- INSERT INTO y SELECT * FROM y RETURNING * +--), t2 AS ( +-- INSERT INTO yy SELECT * FROM t1 RETURNING * +--) +--SELECT 1; + +SELECT * FROM y; +SELECT * FROM yy; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28147] Support RETURNING clause +--WITH RECURSIVE t1 AS ( +-- INSERT INTO yy SELECT * FROM t2 RETURNING * +--), t2 AS ( +-- INSERT INTO y SELECT * FROM y RETURNING * +--) +--SELECT 1; + +SELECT * FROM y; +SELECT * FROM yy; + +-- [NOTE] Spark SQL doesn't support TRIGGERs +-- triggers +-- +--TRUNCATE TABLE y; +--INSERT INTO y SELECT generate_series(1, 10); +-- +--CREATE FUNCTION y_trigger() RETURNS trigger AS $$ +--begin +-- raise notice 'y_trigger: a = %', new.a; +-- return new; +--end; +--$$ LANGUAGE plpgsql; +-- +-- +--CREATE TRIGGER y_trig BEFORE INSERT ON y FOR EACH ROW +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (21), +-- (22), +-- (23) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +-- +--CREATE TRIGGER y_trig AFTER INSERT ON y FOR EACH ROW +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (31), +-- (32), +-- (33) +-- RETURNING * +--) +--SELECT * FROM t LIMIT 1; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +-- +--CREATE OR REPLACE FUNCTION y_trigger() RETURNS trigger AS $$ +--begin +-- raise notice 'y_trigger'; +-- return null; +--end; +--$$ LANGUAGE plpgsql; +-- +--CREATE TRIGGER y_trig AFTER INSERT ON y FOR EACH STATEMENT +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (41), +-- (42), +-- (43) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +--DROP FUNCTION y_trigger(); + +-- WITH attached to inherited UPDATE or DELETE + +-- [ORIGINAL SQL] +--CREATE TEMP TABLE parent ( id int, val text ); +CREATE TABLE parent ( id int, val string ) USING parquet; +-- [NOTE] Spark SQL doesn't support INHERITS clause +--CREATE TEMP TABLE child1 ( ) INHERITS ( parent ); +-- [NOTE] Spark SQL doesn't support INHERITS clause +--CREATE TEMP TABLE child2 ( ) INHERITS ( parent ); + +INSERT INTO parent VALUES ( 1, 'p1' ); +--INSERT INTO child1 VALUES ( 11, 'c11' ),( 12, 'c12' ); +--INSERT INTO child2 VALUES ( 23, 'c21' ),( 24, 'c22' ); + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH rcte AS ( SELECT sum(id) AS totalid FROM parent ) +--UPDATE parent SET id = id + totalid FROM rcte; + +SELECT * FROM parent; + +-- [SPARK-28147] Support RETURNING clause +--WITH wcte AS ( INSERT INTO child1 VALUES ( 42, 'new' ) RETURNING id AS newid ) +--UPDATE parent SET id = id + newid FROM wcte; +-- +--SELECT * FROM parent; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH rcte AS ( SELECT max(id) AS maxid FROM parent ) +--DELETE FROM parent USING rcte WHERE id = maxid; + +SELECT * FROM parent; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH wcte AS ( INSERT INTO child2 VALUES ( 42, 'new2' ) RETURNING id AS newid ) +--DELETE FROM parent USING wcte WHERE id = newid; +-- +--SELECT * FROM parent; + +-- check EXPLAIN VERBOSE for a wCTE with RETURNING + +-- [NOTE] Spark SQL doesn't support DELETE statement +--EXPLAIN (VERBOSE, COSTS OFF) +--WITH wcte AS ( INSERT INTO int8_tbl VALUES ( 42, 47 ) RETURNING q2 ) +--DELETE FROM a USING wcte WHERE aa = q2; + +-- error cases + +-- data-modifying WITH tries to use its own output +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t AS ( +-- INSERT INTO y +-- SELECT * FROM t +--) +--VALUES(FALSE); + +-- no RETURNING in a referenced data-modifying WITH +-- [SPARK-24497] Support recursive SQL query +--WITH t AS ( +-- INSERT INTO y VALUES(0) +--) +--SELECT * FROM t; + +-- data-modifying WITH allowed only at the top level +-- [SPARK-28147] Support RETURNING clause +--SELECT * FROM ( +-- WITH t AS (UPDATE y SET a=a+1 RETURNING *) +-- SELECT * FROM t +--) ss; + +-- most variants of rules aren't allowed +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE RULE y_rule AS ON INSERT TO y WHERE a=0 DO INSTEAD DELETE FROM y; +--WITH t AS ( +-- INSERT INTO y VALUES(0) +--) +--VALUES(FALSE); +--DROP RULE y_rule ON y; + +-- check that parser lookahead for WITH doesn't cause any odd behavior +create table foo (with baz); -- fail, WITH is a reserved word +create table foo (with ordinality); -- fail, WITH is a reserved word +with ordinality as (select 1 as x) select * from ordinality; + +-- check sane response to attempt to modify CTE relation +WITH test AS (SELECT 42) INSERT INTO test VALUES (1); + +-- check response to attempt to modify table with same name as a CTE (perhaps +-- surprisingly it works, because CTEs don't hide tables from data-modifying +-- statements) +-- [ORIGINAL SQL] +--create temp table test (i int); +create table test (i int) USING parquet; +with test as (select 42) insert into test select * from test; +select * from test; +drop table test; + +-- +-- Clean up +-- + +DROP TABLE department; +DROP TABLE tree; +DROP TABLE graph; +DROP TABLE y; +DROP TABLE yy; +DROP TABLE parent; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql index 5b97d3d72641..d829a5c1159f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql @@ -9,12 +9,10 @@ -- SET extra_float_digits = 0; -- This test file was converted from pgSQL/aggregates_part1.sql. --- Note that currently registered UDF returns a string. So there are some differences, for instance --- in string cast within UDF in Scala and Python. -SELECT CAST(avg(udf(four)) AS decimal(10,3)) AS avg_1 FROM onek; +SELECT avg(udf(four)) AS avg_1 FROM onek; -SELECT CAST(udf(avg(a)) AS decimal(10,3)) AS avg_32 FROM aggtest WHERE a < 100; +SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100; -- In 7.1, avg(float4) is computed using float8 arithmetic. -- Round the result to 3 digits to avoid platform-specific results. @@ -23,32 +21,32 @@ select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest; -- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] -- SELECT avg(gpa) AS avg_3_4 FROM ONLY student; -SELECT CAST(sum(udf(four)) AS int) AS sum_1500 FROM onek; +SELECT sum(udf(four)) AS sum_1500 FROM onek; SELECT udf(sum(a)) AS sum_198 FROM aggtest; -SELECT CAST(udf(udf(sum(b))) AS decimal(10,3)) AS avg_431_773 FROM aggtest; +SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest; -- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] -- SELECT sum(gpa) AS avg_6_8 FROM ONLY student; SELECT udf(max(four)) AS max_3 FROM onek; -SELECT max(CAST(udf(a) AS int)) AS max_100 FROM aggtest; -SELECT CAST(udf(udf(max(aggtest.b))) AS decimal(10,3)) AS max_324_78 FROM aggtest; +SELECT max(udf(a)) AS max_100 FROM aggtest; +SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest; -- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] -- SELECT max(student.gpa) AS max_3_7 FROM student; -SELECT CAST(stddev_pop(udf(b)) AS decimal(10,3)) FROM aggtest; -SELECT CAST(udf(stddev_samp(b)) AS decimal(10,3)) FROM aggtest; -SELECT CAST(var_pop(udf(b)) AS decimal(10,3)) FROM aggtest; -SELECT CAST(udf(var_samp(b)) AS decimal(10,3)) FROM aggtest; +SELECT stddev_pop(udf(b)) FROM aggtest; +SELECT udf(stddev_samp(b)) FROM aggtest; +SELECT var_pop(udf(b)) FROM aggtest; +SELECT udf(var_samp(b)) FROM aggtest; -SELECT CAST(udf(stddev_pop(CAST(b AS Decimal(38,0)))) AS decimal(10,3)) FROM aggtest; -SELECT CAST(stddev_samp(CAST(udf(b) AS Decimal(38,0))) AS decimal(10,3)) FROM aggtest; -SELECT CAST(udf(var_pop(CAST(b AS Decimal(38,0)))) AS decimal(10,3)) FROM aggtest; -SELECT CAST(var_samp(udf(CAST(b AS Decimal(38,0)))) AS decimal(10,3)) FROM aggtest; +SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest; +SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest; +SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest; +SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest; -- population variance is defined for a single tuple, sample variance -- is not -SELECT CAST(udf(var_pop(1.0)) AS int), var_samp(udf(2.0)); -SELECT CAST(stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))) AS int), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))); +SELECT udf(var_pop(1.0)), var_samp(udf(2.0)); +SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))); -- verify correct results for null and NaN inputs @@ -76,9 +74,9 @@ FROM (VALUES ('-Infinity'), ('Infinity')) v(x); -- test accuracy with a large input offset -SELECT CAST(avg(udf(CAST(x AS DOUBLE))) AS int), CAST(udf(var_pop(CAST(x AS DOUBLE))) AS decimal(10,3)) +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x); -SELECT CAST(avg(udf(CAST(x AS DOUBLE))) AS long), CAST(udf(var_pop(CAST(x AS DOUBLE))) AS decimal(10,3)) +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) FROM (VALUES (7000000000005), (7000000000007)) v(x); -- SQL2003 binary aggregates [SPARK-23907] @@ -89,8 +87,8 @@ FROM (VALUES (7000000000005), (7000000000007)) v(x); -- SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest; -- SELECT regr_r2(b, a) FROM aggtest; -- SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest; -SELECT CAST(udf(covar_pop(b, udf(a))) AS decimal(10,3)), CAST(covar_samp(udf(b), a) as decimal(10,3)) FROM aggtest; -SELECT CAST(corr(b, udf(a)) AS decimal(10,3)) FROM aggtest; +SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest; +SELECT corr(b, udf(a)) FROM aggtest; -- test accum and combine functions directly [SPARK-23907] @@ -122,7 +120,7 @@ SELECT CAST(corr(b, udf(a)) AS decimal(10,3)) FROM aggtest; SELECT count(udf(four)) AS cnt_1000 FROM onek; SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek; -select ten, udf(count(*)), CAST(sum(udf(four)) AS int) from onek +select ten, udf(count(*)), sum(udf(four)) from onek group by ten order by ten; select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql index 57491a32c48f..5636537398a8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql @@ -6,8 +6,6 @@ -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L145-L350 -- -- This test file was converted from pgSQL/aggregates_part2.sql. --- Note that currently registered UDF returns a string. So there are some differences, for instance --- in string cast within UDF in Scala and Python. create temporary view int4_tbl as select * from values (0), diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql index b05c21d24b36..1865ee94ec1f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql @@ -6,14 +6,8 @@ -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/case.sql -- Test the CASE statement -- --- This test suite contains two Cartesian products without using explicit CROSS JOIN syntax. --- Thus, we set spark.sql.crossJoin.enabled to true. - -- This test file was converted from pgSQL/case.sql. --- Note that currently registered UDF returns a string. So there are some differences, for instance --- in string cast within UDF in Scala and Python. -set spark.sql.crossJoin.enabled=true; CREATE TABLE CASE_TBL ( i integer, f double @@ -42,7 +36,7 @@ INSERT INTO CASE2_TBL VALUES (NULL, -6); SELECT '3' AS `One`, CASE - WHEN CAST(udf(1 < 2) AS boolean) THEN 3 + WHEN udf(1 < 2) THEN 3 END AS `Simple WHEN`; SELECT '' AS `One`, @@ -64,7 +58,7 @@ SELECT udf('4') AS `One`, SELECT udf('6') AS `One`, CASE - WHEN CAST(udf(1 > 2) AS boolean) THEN 3 + WHEN udf(1 > 2) THEN 3 WHEN udf(4) < 5 THEN 6 ELSE 7 END AS `Two WHEN with default`; @@ -74,7 +68,7 @@ SELECT '7' AS `None`, END AS `NULL on no matches`; -- Constant-expression folding shouldn't evaluate unreachable subexpressions -SELECT CASE WHEN CAST(udf(1=0) AS boolean) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END; +SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END; SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END; -- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL @@ -146,7 +140,7 @@ SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b - WHERE CAST(udf(COALESCE(f,b.i) = 2) AS boolean); + WHERE udf(COALESCE(f,b.i) = 2); -- We don't support update now. -- @@ -269,4 +263,3 @@ SELECT CASE DROP TABLE CASE_TBL; DROP TABLE CASE2_TBL; -set spark.sql.crossJoin.enabled=false; 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 000000000000..9aa6de7e92b8 --- /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 000000000000..71e05e67698c --- /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 000000000000..17e24880ea1b --- /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 000000000000..618c8df97747 --- /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 000000000000..393de498edfa --- /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-having.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-having.sql index 6ae34ae589fa..ff8573ad7e56 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-having.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-having.sql @@ -1,6 +1,4 @@ -- This test file was converted from having.sql. --- Note that currently registered UDF returns a string. So there are some differences, for instance --- in string cast within UDF in Scala and Python. create temporary view hav as select * from values ("one", 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 000000000000..179e528d74ca --- /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 000000000000..028d4c769599 --- /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 000000000000..47fb70d02394 --- /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-natural-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql index 686268317800..e5eb812d69a1 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql @@ -4,8 +4,6 @@ --SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false -- This test file was converted from natural-join.sql. --- Note that currently registered UDF returns a string. So there are some differences, for instance --- in string cast within UDF in Scala and Python. create temporary view nt1 as select * from values ("one", 1), 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 000000000000..4eb0805c9cc6 --- /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 000000000000..93937930de7f --- /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-special-values.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-special-values.sql new file mode 100644 index 000000000000..9cd15369bb16 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-special-values.sql @@ -0,0 +1,8 @@ +-- This file tests special values such as NaN, Infinity and NULL. + +SELECT udf(x) FROM (VALUES (1), (2), (NULL)) v(x); +SELECT udf(x) FROM (VALUES ('A'), ('B'), (NULL)) v(x); +SELECT udf(x) FROM (VALUES ('NaN'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES ('Infinity'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES ('-Infinity'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES 0.00000001, 0.00000002, 0.00000003) v(x); diff --git a/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 000000000000..c22d552a0ebe --- /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/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 1f8b5b6ebee7..13f72614f577 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/cte-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out new file mode 100644 index 000000000000..5193e2536c0c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out @@ -0,0 +1,208 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +create temporary view t as select * from values 0, 1, 2 as t(id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values 0, 1 as t(id) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SET spark.sql.legacy.ctePrecedence.enabled=true +-- !query 2 schema +struct +-- !query 2 output +spark.sql.legacy.ctePrecedence.enabled true + + +-- !query 3 +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query 3 schema +struct<1:int> +-- !query 3 output +1 + + +-- !query 4 +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query 4 schema +struct +-- !query 4 output +1 + + +-- !query 5 +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query 5 schema +struct +-- !query 5 output +1 + + +-- !query 6 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query 6 schema +struct<1:int> +-- !query 6 output +1 + + +-- !query 7 +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query 7 schema +struct +-- !query 7 output +1 + + +-- !query 8 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query 8 schema +struct<2:int> +-- !query 8 output +2 + + +-- !query 9 +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query 9 schema +struct +-- !query 9 output +2 + + +-- !query 10 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query 10 schema +struct +-- !query 10 output +2 + + +-- !query 11 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query 11 schema +struct +-- !query 11 output +3 + + +-- !query 12 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query 12 schema +struct +-- !query 12 output +1 + + +-- !query 13 +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query 13 schema +struct +-- !query 13 output +1 + + +-- !query 14 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query 14 schema +struct +-- !query 14 output +1 + + +-- !query 15 +DROP VIEW IF EXISTS t +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +DROP VIEW IF EXISTS t2 +-- !query 16 schema +struct<> +-- !query 16 output + diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index 9e90908d92fa..b7dd76c72520 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -204,9 +204,9 @@ WITH ) SELECT * FROM t2 -- !query 16 schema -struct<1:int> +struct<2:int> -- !query 16 output -1 +2 -- !query 17 @@ -224,7 +224,7 @@ SELECT * FROM t2 -- !query 17 schema struct -- !query 17 output -1 +2 -- !query 18 @@ -240,9 +240,9 @@ WITH ) SELECT * FROM t2 -- !query 18 schema -struct<2:int> +struct<3:int> -- !query 18 output -2 +3 -- !query 19 @@ -295,7 +295,7 @@ SELECT ( -- !query 22 schema struct -- !query 22 output -1 +2 -- !query 23 @@ -309,7 +309,7 @@ SELECT ( -- !query 23 schema struct -- !query 23 output -1 +2 -- !query 24 @@ -324,7 +324,7 @@ SELECT ( -- !query 24 schema struct -- !query 24 output -1 +3 -- !query 25 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 9d3668d49d0a..cacc44d2b1e3 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/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out index 99c42ec2eb6c..b7cf3a9f1ad8 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out @@ -19,7 +19,7 @@ true -- !query 2 -SELECT false AS false +SELECT false AS `false` -- !query 2 schema struct -- !query 2 output @@ -35,7 +35,7 @@ true -- !query 4 -SELECT boolean(' f ') AS false +SELECT boolean(' f ') AS `false` -- !query 4 schema struct -- !query 4 output @@ -59,7 +59,7 @@ NULL -- !query 7 -SELECT boolean('false') AS false +SELECT boolean('false') AS `false` -- !query 7 schema struct -- !query 7 output @@ -99,7 +99,7 @@ NULL -- !query 12 -SELECT boolean('n') AS false +SELECT boolean('n') AS `false` -- !query 12 schema struct -- !query 12 output @@ -107,7 +107,7 @@ false -- !query 13 -SELECT boolean('no') AS false +SELECT boolean('no') AS `false` -- !query 13 schema struct -- !query 13 output @@ -131,7 +131,7 @@ NULL -- !query 16 -SELECT boolean('off') AS false +SELECT boolean('off') AS `false` -- !query 16 schema struct -- !query 16 output @@ -139,7 +139,7 @@ NULL -- !query 17 -SELECT boolean('of') AS false +SELECT boolean('of') AS `false` -- !query 17 schema struct -- !query 17 output @@ -187,7 +187,7 @@ NULL -- !query 23 -SELECT boolean('0') AS false +SELECT boolean('0') AS `false` -- !query 23 schema struct -- !query 23 output @@ -219,7 +219,7 @@ true -- !query 27 -SELECT boolean('t') and boolean('f') AS false +SELECT boolean('t') and boolean('f') AS `false` -- !query 27 schema struct -- !query 27 output @@ -235,7 +235,7 @@ true -- !query 29 -SELECT boolean('t') = boolean('f') AS false +SELECT boolean('t') = boolean('f') AS `false` -- !query 29 schema struct -- !query 29 output @@ -283,7 +283,7 @@ true -- !query 35 -SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS false +SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false` -- !query 35 schema struct -- !query 35 output @@ -292,7 +292,7 @@ true false -- !query 36 SELECT boolean(string(' true ')) AS true, - boolean(string(' FALSE')) AS false + boolean(string(' FALSE')) AS `false` -- !query 36 schema struct -- !query 36 output @@ -300,7 +300,7 @@ NULL NULL -- !query 37 -SELECT string(boolean(true)) AS true, string(boolean(false)) AS false +SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false` -- !query 37 schema struct -- !query 37 output diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out index dbd775e5ebba..f95adcde81b3 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out @@ -1,19 +1,22 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 37 +-- Number of queries: 35 -- !query 0 -set spark.sql.crossJoin.enabled=true +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet -- !query 0 schema -struct +struct<> -- !query 0 output -spark.sql.crossJoin.enabled true + -- !query 1 -CREATE TABLE CASE_TBL ( +CREATE TABLE CASE2_TBL ( i integer, - f double + j integer ) USING parquet -- !query 1 schema struct<> @@ -22,10 +25,7 @@ struct<> -- !query 2 -CREATE TABLE CASE2_TBL ( - i integer, - j integer -) USING parquet +INSERT INTO CASE_TBL VALUES (1, 10.1) -- !query 2 schema struct<> -- !query 2 output @@ -33,7 +33,7 @@ struct<> -- !query 3 -INSERT INTO CASE_TBL VALUES (1, 10.1) +INSERT INTO CASE_TBL VALUES (2, 20.2) -- !query 3 schema struct<> -- !query 3 output @@ -41,7 +41,7 @@ struct<> -- !query 4 -INSERT INTO CASE_TBL VALUES (2, 20.2) +INSERT INTO CASE_TBL VALUES (3, -30.3) -- !query 4 schema struct<> -- !query 4 output @@ -49,7 +49,7 @@ struct<> -- !query 5 -INSERT INTO CASE_TBL VALUES (3, -30.3) +INSERT INTO CASE_TBL VALUES (4, NULL) -- !query 5 schema struct<> -- !query 5 output @@ -57,7 +57,7 @@ struct<> -- !query 6 -INSERT INTO CASE_TBL VALUES (4, NULL) +INSERT INTO CASE2_TBL VALUES (1, -1) -- !query 6 schema struct<> -- !query 6 output @@ -65,7 +65,7 @@ struct<> -- !query 7 -INSERT INTO CASE2_TBL VALUES (1, -1) +INSERT INTO CASE2_TBL VALUES (2, -2) -- !query 7 schema struct<> -- !query 7 output @@ -73,7 +73,7 @@ struct<> -- !query 8 -INSERT INTO CASE2_TBL VALUES (2, -2) +INSERT INTO CASE2_TBL VALUES (3, -3) -- !query 8 schema struct<> -- !query 8 output @@ -81,7 +81,7 @@ struct<> -- !query 9 -INSERT INTO CASE2_TBL VALUES (3, -3) +INSERT INTO CASE2_TBL VALUES (2, -4) -- !query 9 schema struct<> -- !query 9 output @@ -89,7 +89,7 @@ struct<> -- !query 10 -INSERT INTO CASE2_TBL VALUES (2, -4) +INSERT INTO CASE2_TBL VALUES (1, NULL) -- !query 10 schema struct<> -- !query 10 output @@ -97,7 +97,7 @@ struct<> -- !query 11 -INSERT INTO CASE2_TBL VALUES (1, NULL) +INSERT INTO CASE2_TBL VALUES (NULL, -6) -- !query 11 schema struct<> -- !query 11 output @@ -105,148 +105,140 @@ struct<> -- !query 12 -INSERT INTO CASE2_TBL VALUES (NULL, -6) --- !query 12 schema -struct<> --- !query 12 output - - - --- !query 13 SELECT '3' AS `One`, CASE WHEN 1 < 2 THEN 3 END AS `Simple WHEN` --- !query 13 schema +-- !query 12 schema struct --- !query 13 output +-- !query 12 output 3 3 --- !query 14 +-- !query 13 SELECT '' AS `One`, CASE WHEN 1 > 2 THEN 3 END AS `Simple default` --- !query 14 schema +-- !query 13 schema struct --- !query 14 output +-- !query 13 output NULL --- !query 15 +-- !query 14 SELECT '3' AS `One`, CASE WHEN 1 < 2 THEN 3 ELSE 4 END AS `Simple ELSE` --- !query 15 schema +-- !query 14 schema struct --- !query 15 output +-- !query 14 output 3 3 --- !query 16 +-- !query 15 SELECT '4' AS `One`, CASE WHEN 1 > 2 THEN 3 ELSE 4 END AS `ELSE default` --- !query 16 schema +-- !query 15 schema struct --- !query 16 output +-- !query 15 output 4 4 --- !query 17 +-- !query 16 SELECT '6' AS `One`, CASE WHEN 1 > 2 THEN 3 WHEN 4 < 5 THEN 6 ELSE 7 END AS `Two WHEN with default` --- !query 17 schema +-- !query 16 schema struct --- !query 17 output +-- !query 16 output 6 6 --- !query 18 +-- !query 17 SELECT '7' AS `None`, CASE WHEN rand() < 0 THEN 1 END AS `NULL on no matches` --- !query 18 schema +-- !query 17 schema struct --- !query 18 output +-- !query 17 output 7 NULL --- !query 19 +-- !query 18 SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +-- !query 18 schema +struct +-- !query 18 output +1 + + +-- !query 19 +SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END -- !query 19 schema -struct +struct -- !query 19 output -1.0 +1 -- !query 20 -SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END +SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl -- !query 20 schema -struct +struct 100) THEN (1 div 0) ELSE 0 END:int> -- !query 20 output -1.0 +0 +0 +0 +0 -- !query 21 -SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl --- !query 21 schema -struct 100) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) ELSE CAST(0 AS DOUBLE) END:double> --- !query 21 output -0.0 -0.0 -0.0 -0.0 - - --- !query 22 SELECT CASE 'a' WHEN 'a' THEN 1 ELSE 2 END --- !query 22 schema +-- !query 21 schema struct --- !query 22 output +-- !query 21 output 1 --- !query 23 +-- !query 22 SELECT '' AS `Five`, CASE WHEN i >= 3 THEN i END AS `>= 3 or Null` FROM CASE_TBL --- !query 23 schema +-- !query 22 schema struct= 3 or Null:int> --- !query 23 output +-- !query 22 output 3 4 NULL NULL --- !query 24 +-- !query 23 SELECT '' AS `Five`, CASE WHEN i >= 3 THEN (i + i) ELSE i END AS `Simplest Math` FROM CASE_TBL --- !query 24 schema +-- !query 23 schema struct --- !query 24 output +-- !query 23 output 1 2 6 8 --- !query 25 +-- !query 24 SELECT '' AS `Five`, i AS `Value`, CASE WHEN (i < 0) THEN 'small' WHEN (i = 0) THEN 'zero' @@ -255,16 +247,16 @@ SELECT '' AS `Five`, i AS `Value`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 25 schema +-- !query 24 schema struct --- !query 25 output +-- !query 24 output 1 one 2 two 3 big 4 big --- !query 26 +-- !query 25 SELECT '' AS `Five`, CASE WHEN ((i < 0) or (i < 0)) THEN 'small' WHEN ((i = 0) or (i = 0)) THEN 'zero' @@ -273,37 +265,37 @@ SELECT '' AS `Five`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 26 schema +-- !query 25 schema struct --- !query 26 output +-- !query 25 output big big one two --- !query 27 +-- !query 26 SELECT * FROM CASE_TBL WHERE COALESCE(f,i) = 4 --- !query 27 schema +-- !query 26 schema struct --- !query 27 output +-- !query 26 output 4 NULL --- !query 28 +-- !query 27 SELECT * FROM CASE_TBL WHERE NULLIF(f,i) = 2 --- !query 28 schema +-- !query 27 schema struct --- !query 28 output +-- !query 27 output --- !query 29 +-- !query 28 SELECT COALESCE(a.f, b.i, b.j) FROM CASE_TBL a, CASE2_TBL b --- !query 29 schema +-- !query 28 schema struct --- !query 29 output +-- !query 28 output -30.3 -30.3 -30.3 @@ -330,24 +322,24 @@ struct 3.0 --- !query 30 +-- !query 29 SELECT * FROM CASE_TBL a, CASE2_TBL b WHERE COALESCE(a.f, b.i, b.j) = 2 --- !query 30 schema +-- !query 29 schema struct --- !query 30 output +-- !query 29 output 4 NULL 2 -2 4 NULL 2 -4 --- !query 31 +-- !query 30 SELECT '' AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, NULLIF(b.i, 4) AS `NULLIF(b.i,4)` FROM CASE_TBL a, CASE2_TBL b --- !query 31 schema +-- !query 30 schema struct --- !query 31 output +-- !query 30 output 1 2 1 2 1 3 @@ -374,18 +366,18 @@ struct NULL 3 --- !query 32 +-- !query 31 SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b WHERE COALESCE(f,b.i) = 2 --- !query 32 schema +-- !query 31 schema struct --- !query 32 output +-- !query 31 output 4 NULL 2 -2 4 NULL 2 -4 --- !query 33 +-- !query 32 SELECT CASE (CASE vol('bar') WHEN 'foo' THEN 'it was foo!' @@ -395,31 +387,23 @@ SELECT CASE WHEN 'it was foo!' THEN 'foo recognized' WHEN 'it was bar!' THEN 'bar recognized' ELSE 'unrecognized' END --- !query 33 schema +-- !query 32 schema struct --- !query 33 output +-- !query 32 output bar recognized --- !query 34 +-- !query 33 DROP TABLE CASE_TBL --- !query 34 schema +-- !query 33 schema struct<> --- !query 34 output +-- !query 33 output --- !query 35 +-- !query 34 DROP TABLE CASE2_TBL --- !query 35 schema +-- !query 34 schema struct<> --- !query 35 output - - +-- !query 34 output --- !query 36 -set spark.sql.crossJoin.enabled=false --- !query 36 schema -struct --- !query 36 output -spark.sql.crossJoin.enabled false 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 0d669ae7ce5b..d4a0e07fdc72 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/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out new file mode 100644 index 000000000000..3e3f24d603ff --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out @@ -0,0 +1,839 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 95 + + +-- !query 0 +CREATE TABLE FLOAT8_TBL(f1 double) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO FLOAT8_TBL VALUES (' 0.0 ') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO FLOAT8_TBL VALUES ('1004.30 ') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO FLOAT8_TBL VALUES (' -34.84') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e+200') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e-200') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT double('10e400') +-- !query 6 schema +struct +-- !query 6 output +Infinity + + +-- !query 7 +SELECT double('-10e400') +-- !query 7 schema +struct +-- !query 7 output +-Infinity + + +-- !query 8 +SELECT double('10e-400') +-- !query 8 schema +struct +-- !query 8 output +0.0 + + +-- !query 9 +SELECT double('-10e-400') +-- !query 9 schema +struct +-- !query 9 output +-0.0 + + +-- !query 10 +SELECT double('NaN') +-- !query 10 schema +struct +-- !query 10 output +NaN + + +-- !query 11 +SELECT double('nan') +-- !query 11 schema +struct +-- !query 11 output +NULL + + +-- !query 12 +SELECT double(' NAN ') +-- !query 12 schema +struct +-- !query 12 output +NULL + + +-- !query 13 +SELECT double('infinity') +-- !query 13 schema +struct +-- !query 13 output +NULL + + +-- !query 14 +SELECT double(' -INFINiTY ') +-- !query 14 schema +struct +-- !query 14 output +NULL + + +-- !query 15 +SELECT double('N A N') +-- !query 15 schema +struct +-- !query 15 output +NULL + + +-- !query 16 +SELECT double('NaN x') +-- !query 16 schema +struct +-- !query 16 output +NULL + + +-- !query 17 +SELECT double(' INFINITY x') +-- !query 17 schema +struct +-- !query 17 output +NULL + + +-- !query 18 +SELECT double('Infinity') + 100.0 +-- !query 18 schema +struct<(CAST(Infinity AS DOUBLE) + CAST(100.0 AS DOUBLE)):double> +-- !query 18 output +Infinity + + +-- !query 19 +SELECT double('Infinity') / double('Infinity') +-- !query 19 schema +struct<(CAST(Infinity AS DOUBLE) / CAST(Infinity AS DOUBLE)):double> +-- !query 19 output +NaN + + +-- !query 20 +SELECT double('NaN') / double('NaN') +-- !query 20 schema +struct<(CAST(NaN AS DOUBLE) / CAST(NaN AS DOUBLE)):double> +-- !query 20 output +NaN + + +-- !query 21 +SELECT double(decimal('nan')) +-- !query 21 schema +struct +-- !query 21 output +NULL + + +-- !query 22 +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query 22 schema +struct +-- !query 22 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + 1004.3 + + +-- !query 23 +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <> '1004.3' +-- !query 23 schema +struct +-- !query 23 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + + +-- !query 24 +SELECT '' AS one, f.* FROM FLOAT8_TBL f WHERE f.f1 = '1004.3' +-- !query 24 schema +struct +-- !query 24 output +1004.3 + + +-- !query 25 +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE '1004.3' > f.f1 +-- !query 25 schema +struct +-- !query 25 output +-34.84 + 0.0 + 1.2345678901234E-200 + + +-- !query 26 +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE f.f1 < '1004.3' +-- !query 26 schema +struct +-- !query 26 output +-34.84 + 0.0 + 1.2345678901234E-200 + + +-- !query 27 +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE '1004.3' >= f.f1 +-- !query 27 schema +struct +-- !query 27 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1004.3 + + +-- !query 28 +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <= '1004.3' +-- !query 28 schema +struct +-- !query 28 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1004.3 + + +-- !query 29 +SELECT '' AS three, f.f1, f.f1 * '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 29 schema +struct +-- !query 29 output +1.2345678901234E-200 -1.2345678901234E-199 + 1.2345678901234E200 -1.2345678901234E201 + 1004.3 -10043.0 + + +-- !query 30 +SELECT '' AS three, f.f1, f.f1 + '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 30 schema +struct +-- !query 30 output +1.2345678901234E-200 -10.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 994.3 + + +-- !query 31 +SELECT '' AS three, f.f1, f.f1 / '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 31 schema +struct +-- !query 31 output +1.2345678901234E-200 -1.2345678901234E-201 + 1.2345678901234E200 -1.2345678901234E199 + 1004.3 -100.42999999999999 + + +-- !query 32 +SELECT '' AS three, f.f1, f.f1 - '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 32 schema +struct +-- !query 32 output +1.2345678901234E-200 10.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 1014.3 + + +-- !query 33 +SELECT '' AS five, f.f1, round(f.f1) AS round_f1 + FROM FLOAT8_TBL f +-- !query 33 schema +struct +-- !query 33 output +-34.84 -35.0 + 0.0 0.0 + 1.2345678901234E-200 0.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 1004.0 + + +-- !query 34 +select ceil(f1) as ceil_f1 from float8_tbl f +-- !query 34 schema +struct +-- !query 34 output +-34 +0 +1 +1005 +9223372036854775807 + + +-- !query 35 +select ceiling(f1) as ceiling_f1 from float8_tbl f +-- !query 35 schema +struct +-- !query 35 output +-34 +0 +1 +1005 +9223372036854775807 + + +-- !query 36 +select floor(f1) as floor_f1 from float8_tbl f +-- !query 36 schema +struct +-- !query 36 output +-35 +0 +0 +1004 +9223372036854775807 + + +-- !query 37 +select sign(f1) as sign_f1 from float8_tbl f +-- !query 37 schema +struct +-- !query 37 output +-1.0 +0.0 +1.0 +1.0 +1.0 + + +-- !query 38 +SELECT sqrt(double('64')) AS eight +-- !query 38 schema +struct +-- !query 38 output +8.0 + + +-- !query 39 +SELECT power(double('144'), double('0.5')) +-- !query 39 schema +struct +-- !query 39 output +12.0 + + +-- !query 40 +SELECT power(double('NaN'), double('0.5')) +-- !query 40 schema +struct +-- !query 40 output +NaN + + +-- !query 41 +SELECT power(double('144'), double('NaN')) +-- !query 41 schema +struct +-- !query 41 output +NaN + + +-- !query 42 +SELECT power(double('NaN'), double('NaN')) +-- !query 42 schema +struct +-- !query 42 output +NaN + + +-- !query 43 +SELECT power(double('-1'), double('NaN')) +-- !query 43 schema +struct +-- !query 43 output +NaN + + +-- !query 44 +SELECT power(double('1'), double('NaN')) +-- !query 44 schema +struct +-- !query 44 output +NaN + + +-- !query 45 +SELECT power(double('NaN'), double('0')) +-- !query 45 schema +struct +-- !query 45 output +1.0 + + +-- !query 46 +SELECT '' AS three, f.f1, exp(ln(f.f1)) AS exp_ln_f1 + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 46 schema +struct +-- !query 46 output +1.2345678901234E-200 1.2345678901233948E-200 + 1.2345678901234E200 1.234567890123379E200 + 1004.3 1004.3000000000004 + + +-- !query 47 +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query 47 schema +struct +-- !query 47 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + 1004.3 + + +-- !query 48 +CREATE TEMPORARY VIEW UPDATED_FLOAT8_TBL as +SELECT + CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 +FROM FLOAT8_TBL +-- !query 48 schema +struct<> +-- !query 48 output + + + +-- !query 49 +SELECT '' AS bad, f.f1 * '1e200' from UPDATED_FLOAT8_TBL f +-- !query 49 schema +struct +-- !query 49 output +-1.0042999999999999E203 + -1.2345678901234 + -3.484E201 + -Infinity + 0.0 + + +-- !query 50 +SELECT '' AS five, * FROM UPDATED_FLOAT8_TBL +-- !query 50 schema +struct +-- !query 50 output +-1.2345678901234E-200 + -1.2345678901234E200 + -1004.3 + -34.84 + 0.0 + + +-- !query 51 +SELECT sinh(double('1')) +-- !query 51 schema +struct +-- !query 51 output +1.1752011936438014 + + +-- !query 52 +SELECT cosh(double('1')) +-- !query 52 schema +struct +-- !query 52 output +1.543080634815244 + + +-- !query 53 +SELECT tanh(double('1')) +-- !query 53 schema +struct +-- !query 53 output +0.7615941559557649 + + +-- !query 54 +SELECT asinh(double('1')) +-- !query 54 schema +struct +-- !query 54 output +0.8813735870195429 + + +-- !query 55 +SELECT acosh(double('2')) +-- !query 55 schema +struct +-- !query 55 output +1.3169578969248166 + + +-- !query 56 +SELECT atanh(double('0.5')) +-- !query 56 schema +struct +-- !query 56 output +0.5493061443340549 + + +-- !query 57 +SELECT sinh(double('Infinity')) +-- !query 57 schema +struct +-- !query 57 output +Infinity + + +-- !query 58 +SELECT sinh(double('-Infinity')) +-- !query 58 schema +struct +-- !query 58 output +-Infinity + + +-- !query 59 +SELECT sinh(double('NaN')) +-- !query 59 schema +struct +-- !query 59 output +NaN + + +-- !query 60 +SELECT cosh(double('Infinity')) +-- !query 60 schema +struct +-- !query 60 output +Infinity + + +-- !query 61 +SELECT cosh(double('-Infinity')) +-- !query 61 schema +struct +-- !query 61 output +Infinity + + +-- !query 62 +SELECT cosh(double('NaN')) +-- !query 62 schema +struct +-- !query 62 output +NaN + + +-- !query 63 +SELECT tanh(double('Infinity')) +-- !query 63 schema +struct +-- !query 63 output +1.0 + + +-- !query 64 +SELECT tanh(double('-Infinity')) +-- !query 64 schema +struct +-- !query 64 output +-1.0 + + +-- !query 65 +SELECT tanh(double('NaN')) +-- !query 65 schema +struct +-- !query 65 output +NaN + + +-- !query 66 +SELECT asinh(double('Infinity')) +-- !query 66 schema +struct +-- !query 66 output +Infinity + + +-- !query 67 +SELECT asinh(double('-Infinity')) +-- !query 67 schema +struct +-- !query 67 output +-Infinity + + +-- !query 68 +SELECT asinh(double('NaN')) +-- !query 68 schema +struct +-- !query 68 output +NaN + + +-- !query 69 +SELECT acosh(double('Infinity')) +-- !query 69 schema +struct +-- !query 69 output +Infinity + + +-- !query 70 +SELECT acosh(double('-Infinity')) +-- !query 70 schema +struct +-- !query 70 output +NaN + + +-- !query 71 +SELECT acosh(double('NaN')) +-- !query 71 schema +struct +-- !query 71 output +NaN + + +-- !query 72 +SELECT atanh(double('Infinity')) +-- !query 72 schema +struct +-- !query 72 output +NaN + + +-- !query 73 +SELECT atanh(double('-Infinity')) +-- !query 73 schema +struct +-- !query 73 output +NaN + + +-- !query 74 +SELECT atanh(double('NaN')) +-- !query 74 schema +struct +-- !query 74 output +NaN + + +-- !query 75 +TRUNCATE TABLE FLOAT8_TBL +-- !query 75 schema +struct<> +-- !query 75 output + + + +-- !query 76 +INSERT INTO FLOAT8_TBL VALUES ('0.0') +-- !query 76 schema +struct<> +-- !query 76 output + + + +-- !query 77 +INSERT INTO FLOAT8_TBL VALUES ('-34.84') +-- !query 77 schema +struct<> +-- !query 77 output + + + +-- !query 78 +INSERT INTO FLOAT8_TBL VALUES ('-1004.30') +-- !query 78 schema +struct<> +-- !query 78 output + + + +-- !query 79 +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e+200') +-- !query 79 schema +struct<> +-- !query 79 output + + + +-- !query 80 +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e-200') +-- !query 80 schema +struct<> +-- !query 80 output + + + +-- !query 81 +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query 81 schema +struct +-- !query 81 output +-1.2345678901234E-200 + -1.2345678901234E200 + -1004.3 + -34.84 + 0.0 + + +-- !query 82 +SELECT smallint(double('32767.4')) +-- !query 82 schema +struct +-- !query 82 output +32767 + + +-- !query 83 +SELECT smallint(double('32767.6')) +-- !query 83 schema +struct +-- !query 83 output +32767 + + +-- !query 84 +SELECT smallint(double('-32768.4')) +-- !query 84 schema +struct +-- !query 84 output +-32768 + + +-- !query 85 +SELECT smallint(double('-32768.6')) +-- !query 85 schema +struct +-- !query 85 output +-32768 + + +-- !query 86 +SELECT int(double('2147483647.4')) +-- !query 86 schema +struct +-- !query 86 output +2147483647 + + +-- !query 87 +SELECT int(double('2147483647.6')) +-- !query 87 schema +struct +-- !query 87 output +2147483647 + + +-- !query 88 +SELECT int(double('-2147483648.4')) +-- !query 88 schema +struct +-- !query 88 output +-2147483648 + + +-- !query 89 +SELECT int(double('-2147483648.6')) +-- !query 89 schema +struct +-- !query 89 output +-2147483648 + + +-- !query 90 +SELECT bigint(double('9223372036854773760')) +-- !query 90 schema +struct +-- !query 90 output +9223372036854773760 + + +-- !query 91 +SELECT bigint(double('9223372036854775807')) +-- !query 91 schema +struct +-- !query 91 output +9223372036854775807 + + +-- !query 92 +SELECT bigint(double('-9223372036854775808.5')) +-- !query 92 schema +struct +-- !query 92 output +-9223372036854775808 + + +-- !query 93 +SELECT bigint(double('-9223372036854780000')) +-- !query 93 schema +struct +-- !query 93 output +-9223372036854775808 + + +-- !query 94 +DROP TABLE FLOAT8_TBL +-- !query 94 schema +struct<> +-- !query 94 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out index 6b9246fed0c6..7a7ce5f37dea 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out @@ -266,7 +266,7 @@ struct -- !query 27 -SELECT '' AS five, i.f1, i.f1 div smallint('2') AS x FROM INT2_TBL i +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i -- !query 27 schema struct -- !query 27 output @@ -278,7 +278,7 @@ struct -- !query 28 -SELECT '' AS five, i.f1, i.f1 div int('2') AS x FROM INT2_TBL i +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i -- !query 28 schema struct -- !query 28 output diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out index 9c17e9a1a197..456b1ef962d4 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out @@ -339,25 +339,25 @@ struct -- !query 33 SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i -- !query 33 schema -struct +struct -- !query 33 output --123456 -61728.0 - -2147483647 -1.0737418235E9 - 0 0.0 - 123456 61728.0 - 2147483647 1.0737418235E9 +-123456 -61728 + -2147483647 -1073741823 + 0 0 + 123456 61728 + 2147483647 1073741823 -- !query 34 SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i -- !query 34 schema -struct +struct -- !query 34 output --123456 -61728.0 - -2147483647 -1.0737418235E9 - 0 0.0 - 123456 61728.0 - 2147483647 1.0737418235E9 +-123456 -61728 + -2147483647 -1073741823 + 0 0 + 123456 61728 + 2147483647 1073741823 -- !query 35 @@ -417,7 +417,7 @@ true -- !query 42 -SELECT int('1000') < int('999') AS false +SELECT int('1000') < int('999') AS `false` -- !query 42 schema struct -- !query 42 output @@ -435,17 +435,17 @@ struct -- !query 44 SELECT 2 + 2 / 2 AS three -- !query 44 schema -struct +struct -- !query 44 output -3.0 +3 -- !query 45 SELECT (2 + 2) / 2 AS two -- !query 45 schema -struct +struct -- !query 45 output -2.0 +2 -- !query 46 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out index 13bc748dd2b4..6d7fae19aa7e 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out @@ -351,7 +351,7 @@ struct -- !query 37 -SELECT '' AS five, q1 AS plus, -q1 AS minus FROM INT8_TBL +SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL -- !query 37 schema struct -- !query 37 output @@ -375,7 +375,7 @@ struct -- !query 39 -SELECT '' AS five, q1, q2, q1 - q2 AS minus FROM INT8_TBL +SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL -- !query 39 schema struct -- !query 39 output @@ -412,13 +412,13 @@ struct -- !query 42 SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL -- !query 42 schema -struct +struct -- !query 42 output -123 456 0.26973684210526316 123 - 123 4567890123456789 2.6927092525360204E-14 123 - 4567890123456789 -4567890123456789 -1.0 0 - 4567890123456789 123 3.713731807688446E13 57 - 4567890123456789 4567890123456789 1.0 0 +123 456 0 123 + 123 4567890123456789 0 123 + 4567890123456789 -4567890123456789 -1 0 + 4567890123456789 123 37137318076884 57 + 4567890123456789 4567890123456789 1 0 -- !query 43 @@ -496,49 +496,49 @@ struct -- !query 49 SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL -- !query 49 schema -struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:double> +struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:bigint> -- !query 49 output -165 81 5166 2.9285714285714284 -165 81 5166 2.9285714285714284 -4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 -4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 -4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +165 81 5166 2 +165 81 5166 2 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 -- !query 50 SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL -- !query 50 schema -struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:double> +struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:bigint> -- !query 50 output -369 123 30258 2.0 -369 123 30258 2.0 -4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 -4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 -4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +369 123 30258 2 +369 123 30258 2 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 -- !query 51 SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL -- !query 51 schema -struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:double> +struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:bigint> -- !query 51 output -165 81 5166 2.9285714285714284 -165 81 5166 2.9285714285714284 -4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 -4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 -4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +165 81 5166 2 +165 81 5166 2 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 -- !query 52 SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL -- !query 52 schema -struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:double> +struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:bigint> -- !query 52 output -369 123 30258 2.0 -369 123 30258 2.0 -4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 -4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 -4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +369 123 30258 2 +369 123 30258 2 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 -- !query 53 @@ -572,7 +572,7 @@ struct -- !query 56 select bigint('9223372036854775800') / bigint('0') -- !query 56 schema -struct<(CAST(CAST(9223372036854775800 AS BIGINT) AS DOUBLE) / CAST(CAST(0 AS BIGINT) AS DOUBLE)):double> +struct<(CAST(9223372036854775800 AS BIGINT) div CAST(0 AS BIGINT)):bigint> -- !query 56 output NULL @@ -580,7 +580,7 @@ NULL -- !query 57 select bigint('-9223372036854775808') / smallint('0') -- !query 57 schema -struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(0 AS SMALLINT) AS DOUBLE)):double> +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(0 AS SMALLINT) AS BIGINT)):bigint> -- !query 57 output NULL @@ -588,7 +588,7 @@ NULL -- !query 58 select smallint('100') / bigint('0') -- !query 58 schema -struct<(CAST(CAST(100 AS SMALLINT) AS DOUBLE) / CAST(CAST(0 AS BIGINT) AS DOUBLE)):double> +struct<(CAST(CAST(100 AS SMALLINT) AS BIGINT) div CAST(0 AS BIGINT)):bigint> -- !query 58 output NULL @@ -744,9 +744,9 @@ struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(-1 AS BIGINT)):bigint> -- !query 74 SELECT bigint((-9223372036854775808)) / bigint((-1)) -- !query 74 schema -struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS BIGINT) AS DOUBLE)):double> +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(-1 AS BIGINT)):bigint> -- !query 74 output -9.223372036854776E18 +-9223372036854775808 -- !query 75 @@ -768,9 +768,9 @@ struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(CAST(-1 AS INT) AS BIGINT)): -- !query 77 SELECT bigint((-9223372036854775808)) / int((-1)) -- !query 77 schema -struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS INT) AS DOUBLE)):double> +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(-1 AS INT) AS BIGINT)):bigint> -- !query 77 output -9.223372036854776E18 +-9223372036854775808 -- !query 78 @@ -792,9 +792,9 @@ struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(CAST(-1 AS SMALLINT) AS BIGI -- !query 80 SELECT bigint((-9223372036854775808)) / smallint((-1)) -- !query 80 schema -struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS SMALLINT) AS DOUBLE)):double> +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> -- !query 80 output -9.223372036854776E18 +-9223372036854775808 -- !query 81 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 000000000000..04425fd4b544 --- /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.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out new file mode 100644 index 000000000000..797f808dad11 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out @@ -0,0 +1,543 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 37 + + +-- !query 0 +create or replace temporary view onek2 as select * from onek +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create or replace temporary view INT8_TBL as select * from values + (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), + (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), + (cast('4567890123456789' as bigint),cast('123' as bigint)), + (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), + (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) + as INT8_TBL(q1, q2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM onek + WHERE onek.unique1 < 10 + ORDER BY onek.unique1 +-- !query 2 schema +struct +-- !query 2 output +0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx +1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx +2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx +3 431 1 3 3 3 3 3 3 3 3 6 7 DAAAAA PQAAAA VVVVxx +4 833 0 0 4 4 4 4 4 4 4 8 9 EAAAAA BGBAAA HHHHxx +5 541 1 1 5 5 5 5 5 5 5 10 11 FAAAAA VUAAAA HHHHxx +6 978 0 2 6 6 6 6 6 6 6 12 13 GAAAAA QLBAAA OOOOxx +7 647 1 3 7 7 7 7 7 7 7 14 15 HAAAAA XYAAAA VVVVxx +8 653 0 0 8 8 8 8 8 8 8 16 17 IAAAAA DZAAAA HHHHxx +9 49 1 1 9 9 9 9 9 9 9 18 19 JAAAAA XBAAAA HHHHxx + + +-- !query 3 +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC +-- !query 3 schema +struct +-- !query 3 output +19 TAAAAA +18 SAAAAA +17 RAAAAA +16 QAAAAA +15 PAAAAA +14 OAAAAA +13 NAAAAA +12 MAAAAA +11 LAAAAA +10 KAAAAA +9 JAAAAA +8 IAAAAA +7 HAAAAA +6 GAAAAA +5 FAAAAA +4 EAAAAA +3 DAAAAA +2 CAAAAA +1 BAAAAA +0 AAAAAA + + +-- !query 4 +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 > 980 + ORDER BY stringu1 ASC +-- !query 4 schema +struct +-- !query 4 output +988 AMAAAA +989 BMAAAA +990 CMAAAA +991 DMAAAA +992 EMAAAA +993 FMAAAA +994 GMAAAA +995 HMAAAA +996 IMAAAA +997 JMAAAA +998 KMAAAA +999 LMAAAA +981 TLAAAA +982 ULAAAA +983 VLAAAA +984 WLAAAA +985 XLAAAA +986 YLAAAA +987 ZLAAAA + + +-- !query 5 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 ASC, unique1 DESC +-- !query 5 schema +struct +-- !query 5 output +999 AAAAxx +995 AAAAxx +983 AAAAxx +982 AAAAxx +981 AAAAxx +998 HHHHxx +997 HHHHxx +993 HHHHxx +990 HHHHxx +986 HHHHxx +996 OOOOxx +991 OOOOxx +988 OOOOxx +987 OOOOxx +985 OOOOxx +994 VVVVxx +992 VVVVxx +989 VVVVxx +984 VVVVxx + + +-- !query 6 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 DESC, unique1 ASC +-- !query 6 schema +struct +-- !query 6 output +984 VVVVxx +989 VVVVxx +992 VVVVxx +994 VVVVxx +985 OOOOxx +987 OOOOxx +988 OOOOxx +991 OOOOxx +996 OOOOxx +986 HHHHxx +990 HHHHxx +993 HHHHxx +997 HHHHxx +998 HHHHxx +981 AAAAxx +982 AAAAxx +983 AAAAxx +995 AAAAxx +999 AAAAxx + + +-- !query 7 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC, string4 ASC +-- !query 7 schema +struct +-- !query 7 output +19 OOOOxx +18 VVVVxx +17 HHHHxx +16 OOOOxx +15 VVVVxx +14 AAAAxx +13 OOOOxx +12 AAAAxx +11 OOOOxx +10 AAAAxx +9 HHHHxx +8 HHHHxx +7 VVVVxx +6 OOOOxx +5 HHHHxx +4 HHHHxx +3 VVVVxx +2 OOOOxx +1 OOOOxx +0 OOOOxx + + +-- !query 8 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 ASC, string4 DESC +-- !query 8 schema +struct +-- !query 8 output +0 OOOOxx +1 OOOOxx +2 OOOOxx +3 VVVVxx +4 HHHHxx +5 HHHHxx +6 OOOOxx +7 VVVVxx +8 HHHHxx +9 HHHHxx +10 AAAAxx +11 OOOOxx +12 AAAAxx +13 OOOOxx +14 AAAAxx +15 VVVVxx +16 OOOOxx +17 HHHHxx +18 VVVVxx +19 OOOOxx + + +-- !query 9 +SELECT onek2.* FROM onek2 WHERE onek2.unique1 < 10 +-- !query 9 schema +struct +-- !query 9 output +0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx +1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx +2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx +3 431 1 3 3 3 3 3 3 3 3 6 7 DAAAAA PQAAAA VVVVxx +4 833 0 0 4 4 4 4 4 4 4 8 9 EAAAAA BGBAAA HHHHxx +5 541 1 1 5 5 5 5 5 5 5 10 11 FAAAAA VUAAAA HHHHxx +6 978 0 2 6 6 6 6 6 6 6 12 13 GAAAAA QLBAAA OOOOxx +7 647 1 3 7 7 7 7 7 7 7 14 15 HAAAAA XYAAAA VVVVxx +8 653 0 0 8 8 8 8 8 8 8 16 17 IAAAAA DZAAAA HHHHxx +9 49 1 1 9 9 9 9 9 9 9 18 19 JAAAAA XBAAAA HHHHxx + + +-- !query 10 +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 < 20 + ORDER BY unique1 DESC +-- !query 10 schema +struct +-- !query 10 output +19 TAAAAA +18 SAAAAA +17 RAAAAA +16 QAAAAA +15 PAAAAA +14 OAAAAA +13 NAAAAA +12 MAAAAA +11 LAAAAA +10 KAAAAA +9 JAAAAA +8 IAAAAA +7 HAAAAA +6 GAAAAA +5 FAAAAA +4 EAAAAA +3 DAAAAA +2 CAAAAA +1 BAAAAA +0 AAAAAA + + +-- !query 11 +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 > 980 +-- !query 11 schema +struct +-- !query 11 output +981 TLAAAA +982 ULAAAA +983 VLAAAA +984 WLAAAA +985 XLAAAA +986 YLAAAA +987 ZLAAAA +988 AMAAAA +989 BMAAAA +990 CMAAAA +991 DMAAAA +992 EMAAAA +993 FMAAAA +994 GMAAAA +995 HMAAAA +996 IMAAAA +997 JMAAAA +998 KMAAAA +999 LMAAAA + + +-- !query 12 +CREATE TABLE tmp USING parquet AS +SELECT two, stringu1, ten, string4 +FROM onek +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +select foo.* from (select 1) as foo +-- !query 13 schema +struct<1:int> +-- !query 13 output +1 + + +-- !query 14 +select foo.* from (select null) as foo +-- !query 14 schema +struct +-- !query 14 output +NULL + + +-- !query 15 +select foo.* from (select 'xyzzy',1,null) as foo +-- !query 15 schema +struct +-- !query 15 output +xyzzy 1 NULL + + +-- !query 16 +select * from onek, values(147, 'RFAAAA'), (931, 'VJAAAA') as v (i, j) + WHERE onek.unique1 = v.i and onek.stringu1 = v.j +-- !query 16 schema +struct +-- !query 16 output +147 0 1 3 7 7 7 47 147 147 147 14 15 RFAAAA AAAAAA AAAAxx 147 RFAAAA +931 1 1 3 1 11 1 31 131 431 931 2 3 VJAAAA BAAAAA HHHHxx 931 VJAAAA + + +-- !query 17 +VALUES (1,2), (3,4+4), (7,77.7) +-- !query 17 schema +struct +-- !query 17 output +1 2 +3 8 +7 77.7 + + +-- !query 18 +VALUES (1,2), (3,4+4), (7,77.7) +UNION ALL +SELECT 2+2, 57 +UNION ALL +TABLE int8_tbl +-- !query 18 schema +struct +-- !query 18 output +1 2 +123 456 +123 4567890123456789 +3 8 +4 57 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 +7 77.7 + + +-- !query 19 +CREATE OR REPLACE TEMPORARY VIEW foo AS +SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1) +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +SELECT * FROM foo ORDER BY f1 +-- !query 20 schema +struct +-- !query 20 output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query 21 +SELECT * FROM foo ORDER BY f1 ASC +-- !query 21 schema +struct +-- !query 21 output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query 22 +-- same thing +SELECT * FROM foo ORDER BY f1 NULLS FIRST +-- !query 22 schema +struct +-- !query 22 output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query 23 +SELECT * FROM foo ORDER BY f1 DESC +-- !query 23 schema +struct +-- !query 23 output +42 +10 +7 +3 +1 +NULL +NULL + + +-- !query 24 +SELECT * FROM foo ORDER BY f1 DESC NULLS LAST +-- !query 24 schema +struct +-- !query 24 output +42 +10 +7 +3 +1 +NULL +NULL + + +-- !query 25 +select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' +-- !query 25 schema +struct +-- !query 25 output +494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx + + +-- !query 26 +select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' +-- !query 26 schema +struct +-- !query 26 output +11 + + +-- !query 27 +select * from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query 27 schema +struct +-- !query 27 output +494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx + + +-- !query 28 +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query 28 schema +struct +-- !query 28 output +11 + + +-- !query 29 +select unique2 from onek2 where unique2 = 11 and stringu1 < 'C' +-- !query 29 schema +struct +-- !query 29 output +11 + + +-- !query 30 +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query 30 schema +struct +-- !query 30 output +11 + + +-- !query 31 +select unique1, unique2 from onek2 + where (unique2 = 11 or unique1 = 0) and stringu1 < 'B' +-- !query 31 schema +struct +-- !query 31 output +0 998 +494 11 + + +-- !query 32 +select unique1, unique2 from onek2 + where (unique2 = 11 and stringu1 < 'B') or unique1 = 0 +-- !query 32 schema +struct +-- !query 32 output +0 998 +494 11 + + +-- !query 33 +SELECT 1 AS x ORDER BY x +-- !query 33 schema +struct +-- !query 33 output +1 + + +-- !query 34 +select * from (values (2),(null),(1)) v(k) where k = k order by k +-- !query 34 schema +struct +-- !query 34 output +1 +2 + + +-- !query 35 +select * from (values (2),(null),(1)) v(k) where k = k +-- !query 35 schema +struct +-- !query 35 output +1 +2 + + +-- !query 36 +drop table tmp +-- !query 36 schema +struct<> +-- !query 36 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out new file mode 100644 index 000000000000..38eae1739f55 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out @@ -0,0 +1,225 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 19 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW tmp AS +SELECT two, stringu1, ten, string4 +FROM onek +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT DISTINCT two FROM tmp ORDER BY 1 +-- !query 1 schema +struct +-- !query 1 output +0 +1 + + +-- !query 2 +SELECT DISTINCT ten FROM tmp ORDER BY 1 +-- !query 2 schema +struct +-- !query 2 output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 3 +SELECT DISTINCT string4 FROM tmp ORDER BY 1 +-- !query 3 schema +struct +-- !query 3 output +AAAAxx +HHHHxx +OOOOxx +VVVVxx + + +-- !query 4 +SELECT DISTINCT two, string4, ten + FROM tmp + ORDER BY two ASC, string4 ASC, ten ASC +-- !query 4 schema +struct +-- !query 4 output +0 AAAAxx 0 +0 AAAAxx 2 +0 AAAAxx 4 +0 AAAAxx 6 +0 AAAAxx 8 +0 HHHHxx 0 +0 HHHHxx 2 +0 HHHHxx 4 +0 HHHHxx 6 +0 HHHHxx 8 +0 OOOOxx 0 +0 OOOOxx 2 +0 OOOOxx 4 +0 OOOOxx 6 +0 OOOOxx 8 +0 VVVVxx 0 +0 VVVVxx 2 +0 VVVVxx 4 +0 VVVVxx 6 +0 VVVVxx 8 +1 AAAAxx 1 +1 AAAAxx 3 +1 AAAAxx 5 +1 AAAAxx 7 +1 AAAAxx 9 +1 HHHHxx 1 +1 HHHHxx 3 +1 HHHHxx 5 +1 HHHHxx 7 +1 HHHHxx 9 +1 OOOOxx 1 +1 OOOOxx 3 +1 OOOOxx 5 +1 OOOOxx 7 +1 OOOOxx 9 +1 VVVVxx 1 +1 VVVVxx 3 +1 VVVVxx 5 +1 VVVVxx 7 +1 VVVVxx 9 + + +-- !query 5 +SELECT count(*) FROM + (SELECT DISTINCT two, four, two FROM tenk1) ss +-- !query 5 schema +struct +-- !query 5 output +4 + + +-- !query 6 +CREATE OR REPLACE TEMPORARY VIEW disttable AS SELECT * FROM + (VALUES (1), (2), (3), (NULL)) + AS v(f1) +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +SELECT f1, f1 IS DISTINCT FROM 2 as `not 2` FROM disttable +-- !query 7 schema +struct +-- !query 7 output +1 true +2 false +3 true +NULL true + + +-- !query 8 +SELECT f1, f1 IS DISTINCT FROM NULL as `not null` FROM disttable +-- !query 8 schema +struct +-- !query 8 output +1 true +2 true +3 true +NULL false + + +-- !query 9 +SELECT f1, f1 IS DISTINCT FROM f1 as `false` FROM disttable +-- !query 9 schema +struct +-- !query 9 output +1 false +2 false +3 false +NULL false + + +-- !query 10 +SELECT f1, f1 IS DISTINCT FROM f1+1 as `not null` FROM disttable +-- !query 10 schema +struct +-- !query 10 output +1 true +2 true +3 true +NULL false + + +-- !query 11 +SELECT 1 IS DISTINCT FROM 2 as `yes` +-- !query 11 schema +struct +-- !query 11 output +true + + +-- !query 12 +SELECT 2 IS DISTINCT FROM 2 as `no` +-- !query 12 schema +struct +-- !query 12 output +false + + +-- !query 13 +SELECT 2 IS DISTINCT FROM null as `yes` +-- !query 13 schema +struct +-- !query 13 output +true + + +-- !query 14 +SELECT null IS DISTINCT FROM null as `no` +-- !query 14 schema +struct +-- !query 14 output +false + + +-- !query 15 +SELECT 1 IS NOT DISTINCT FROM 2 as `no` +-- !query 15 schema +struct +-- !query 15 output +false + + +-- !query 16 +SELECT 2 IS NOT DISTINCT FROM 2 as `yes` +-- !query 16 schema +struct +-- !query 16 output +true + + +-- !query 17 +SELECT 2 IS NOT DISTINCT FROM null as `no` +-- !query 17 schema +struct +-- !query 17 output +false + + +-- !query 18 +SELECT null IS NOT DISTINCT FROM null as `yes` +-- !query 18 schema +struct +-- !query 18 output +true diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out new file mode 100644 index 000000000000..02536ebd8ebe --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/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 b, c FROM test_having + GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c +-- !query 11 schema +struct +-- !query 11 output +1 XXXX +3 bbbb + + +-- !query 12 +SELECT b, c FROM test_having + GROUP BY b, c HAVING b = 3 ORDER BY b, c +-- !query 12 schema +struct +-- !query 12 output +3 BBBB +3 bbbb + + +-- !query 13 +SELECT c, max(a) FROM test_having + GROUP BY c HAVING count(*) > 2 OR min(a) = max(a) + ORDER BY c +-- !query 13 schema +struct +-- !query 13 output +XXXX 0 +bbbb 5 + + +-- !query 14 +SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a) +-- !query 14 schema +struct +-- !query 14 output + + + +-- !query 15 +SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a) +-- !query 15 schema +struct +-- !query 15 output +0 9 + + +-- !query 16 +SELECT a FROM test_having HAVING min(a) < 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 a > 1 +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +cannot resolve '`a`' given input columns: [one]; line 1 pos 40 + + +-- !query 18 +SELECT 1 AS one FROM test_having HAVING 1 > 2 +-- !query 18 schema +struct +-- !query 18 output + + + +-- !query 19 +SELECT 1 AS one FROM test_having HAVING 1 < 2 +-- !query 19 schema +struct +-- !query 19 output +1 + + +-- !query 20 +SELECT 1 AS one FROM test_having WHERE 1/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/pgSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out new file mode 100644 index 000000000000..0675820b381d --- /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 000000000000..200fecce17bc --- /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/pgSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out new file mode 100644 index 000000000000..366b65f3659c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out @@ -0,0 +1,471 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 51 + + +-- !query 0 +WITH q1(x,y) AS (SELECT 1,2) +SELECT * FROM q1, q1 AS q2 +-- !query 0 schema +struct +-- !query 0 output +1 2 1 2 + + +-- !query 1 +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss +-- !query 1 schema +struct +-- !query 1 output +10 + + +-- !query 2 +CREATE TABLE department ( + id INTEGER, -- department ID + parent_department INTEGER, -- upper department ID + name string -- department name +) USING parquet +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO department VALUES (0, NULL, 'ROOT') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO department VALUES (1, 0, 'A') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO department VALUES (2, 1, 'B') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO department VALUES (3, 2, 'C') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO department VALUES (4, 2, 'D') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO department VALUES (5, 0, 'E') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO department VALUES (6, 4, 'F') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO department VALUES (7, 5, 'G') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +CREATE TABLE tree( + id INTEGER, + parent_id INTEGER +) USING parquet +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO tree +VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), + (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11) +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +create table graph( f int, t int, label string ) USING parquet +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +insert into graph values + (1, 2, 'arc 1 -> 2'), + (1, 3, 'arc 1 -> 3'), + (2, 3, 'arc 2 -> 3'), + (1, 4, 'arc 1 -> 4'), + (4, 5, 'arc 4 -> 5'), + (5, 1, 'arc 5 -> 1') +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +CREATE TABLE y (a INTEGER) USING parquet +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +DROP TABLE y +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +CREATE TABLE y (a INTEGER) USING parquet +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q +-- !query 20 schema +struct +-- !query 20 output +42 + + +-- !query 21 +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM innermost + UNION SELECT 3) +) +SELECT * FROM outermost ORDER BY 1 +-- !query 21 schema +struct +-- !query 21 output +1 +2 +3 + + +-- !query 22 +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost -- fail + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1 +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.AnalysisException +Table or view not found: outermost; line 4 pos 23 + + +-- !query 23 +CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i || ' v' AS string) v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i) +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +SELECT * FROM withz ORDER BY k +-- !query 24 schema +struct +-- !query 24 output +1 1 v +4 4 v +7 7 v +10 10 v +13 13 v +16 16 v + + +-- !query 25 +DROP TABLE withz +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +TRUNCATE TABLE y +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)) +-- !query 27 schema +struct<> +-- !query 27 output + + + +-- !query 28 +CREATE TABLE yy (a INTEGER) USING parquet +-- !query 28 schema +struct<> +-- !query 28 output + + + +-- !query 29 +SELECT * FROM y +-- !query 29 schema +struct +-- !query 29 output +1 +2 +3 + + +-- !query 30 +SELECT * FROM yy +-- !query 30 schema +struct +-- !query 30 output + + + +-- !query 31 +SELECT * FROM y +-- !query 31 schema +struct +-- !query 31 output +1 +2 +3 + + +-- !query 32 +SELECT * FROM yy +-- !query 32 schema +struct +-- !query 32 output + + + +-- !query 33 +CREATE TABLE parent ( id int, val string ) USING parquet +-- !query 33 schema +struct<> +-- !query 33 output + + + +-- !query 34 +INSERT INTO parent VALUES ( 1, 'p1' ) +-- !query 34 schema +struct<> +-- !query 34 output + + + +-- !query 35 +SELECT * FROM parent +-- !query 35 schema +struct +-- !query 35 output +1 p1 + + +-- !query 36 +SELECT * FROM parent +-- !query 36 schema +struct +-- !query 36 output +1 p1 + + +-- !query 37 +create table foo (with baz) +-- !query 37 schema +struct<> +-- !query 37 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'with'(line 1, pos 18) + +== SQL == +create table foo (with baz) +------------------^^^ + + +-- !query 38 +-- fail, WITH is a reserved word +create table foo (with ordinality) +-- !query 38 schema +struct<> +-- !query 38 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'with'(line 2, pos 18) + +== SQL == +-- fail, WITH is a reserved word +create table foo (with ordinality) +------------------^^^ + + +-- !query 39 +-- fail, WITH is a reserved word +with ordinality as (select 1 as x) select * from ordinality +-- !query 39 schema +struct +-- !query 39 output +1 + + +-- !query 40 +WITH test AS (SELECT 42) INSERT INTO test VALUES (1) +-- !query 40 schema +struct<> +-- !query 40 output +org.apache.spark.sql.AnalysisException +Table not found: test; + + +-- !query 41 +create table test (i int) USING parquet +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +with test as (select 42) insert into test select * from test +-- !query 42 schema +struct<> +-- !query 42 output + + + +-- !query 43 +select * from test +-- !query 43 schema +struct +-- !query 43 output +42 + + +-- !query 44 +drop table test +-- !query 44 schema +struct<> +-- !query 44 output + + + +-- !query 45 +DROP TABLE department +-- !query 45 schema +struct<> +-- !query 45 output + + + +-- !query 46 +DROP TABLE tree +-- !query 46 schema +struct<> +-- !query 46 output + + + +-- !query 47 +DROP TABLE graph +-- !query 47 schema +struct<> +-- !query 47 output + + + +-- !query 48 +DROP TABLE y +-- !query 48 schema +struct<> +-- !query 48 output + + + +-- !query 49 +DROP TABLE yy +-- !query 49 schema +struct<> +-- !query 49 output + + + +-- !query 50 +DROP TABLE parent +-- !query 50 schema +struct<> +-- !query 50 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 12c1d1617679..a4cd408c04bf 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 4fa2032c66c6..5b77bf9f35f2 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-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out index 98e04b4e2450..a2f64717d73a 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out @@ -3,19 +3,19 @@ -- !query 0 -SELECT CAST(avg(udf(four)) AS decimal(10,3)) AS avg_1 FROM onek +SELECT avg(udf(four)) AS avg_1 FROM onek -- !query 0 schema -struct +struct -- !query 0 output 1.5 -- !query 1 -SELECT CAST(udf(avg(a)) AS decimal(10,3)) AS avg_32 FROM aggtest WHERE a < 100 +SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100 -- !query 1 schema -struct +struct -- !query 1 output -32.667 +32.666666666666664 -- !query 2 @@ -27,9 +27,9 @@ struct -- !query 3 -SELECT CAST(sum(udf(four)) AS int) AS sum_1500 FROM onek +SELECT sum(udf(four)) AS sum_1500 FROM onek -- !query 3 schema -struct +struct -- !query 3 output 1500 @@ -37,29 +37,29 @@ struct -- !query 4 SELECT udf(sum(a)) AS sum_198 FROM aggtest -- !query 4 schema -struct +struct -- !query 4 output 198 -- !query 5 -SELECT CAST(udf(udf(sum(b))) AS decimal(10,3)) AS avg_431_773 FROM aggtest +SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest -- !query 5 schema -struct +struct -- !query 5 output -431.773 +431.77260909229517 -- !query 6 SELECT udf(max(four)) AS max_3 FROM onek -- !query 6 schema -struct +struct -- !query 6 output 3 -- !query 7 -SELECT max(CAST(udf(a) AS int)) AS max_100 FROM aggtest +SELECT max(udf(a)) AS max_100 FROM aggtest -- !query 7 schema struct -- !query 7 output @@ -67,97 +67,97 @@ struct -- !query 8 -SELECT CAST(udf(udf(max(aggtest.b))) AS decimal(10,3)) AS max_324_78 FROM aggtest +SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest -- !query 8 schema -struct +struct -- !query 8 output 324.78 -- !query 9 -SELECT CAST(stddev_pop(udf(b)) AS decimal(10,3)) FROM aggtest +SELECT stddev_pop(udf(b)) FROM aggtest -- !query 9 schema -struct +struct -- !query 9 output -131.107 +131.10703231895047 -- !query 10 -SELECT CAST(udf(stddev_samp(b)) AS decimal(10,3)) FROM aggtest +SELECT udf(stddev_samp(b)) FROM aggtest -- !query 10 schema -struct +struct -- !query 10 output -151.389 +151.38936080399804 -- !query 11 -SELECT CAST(var_pop(udf(b)) AS decimal(10,3)) FROM aggtest +SELECT var_pop(udf(b)) FROM aggtest -- !query 11 schema -struct +struct -- !query 11 output -17189.054 +17189.053923482323 -- !query 12 -SELECT CAST(udf(var_samp(b)) AS decimal(10,3)) FROM aggtest +SELECT udf(var_samp(b)) FROM aggtest -- !query 12 schema -struct +struct -- !query 12 output -22918.739 +22918.738564643096 -- !query 13 -SELECT CAST(udf(stddev_pop(CAST(b AS Decimal(38,0)))) AS decimal(10,3)) FROM aggtest +SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest -- !query 13 schema -struct +struct -- !query 13 output -131.181 +131.18117242958306 -- !query 14 -SELECT CAST(stddev_samp(CAST(udf(b) AS Decimal(38,0))) AS decimal(10,3)) FROM aggtest +SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest -- !query 14 schema -struct +struct -- !query 14 output -151.475 +151.47497042966097 -- !query 15 -SELECT CAST(udf(var_pop(CAST(b AS Decimal(38,0)))) AS decimal(10,3)) FROM aggtest +SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest -- !query 15 schema -struct +struct -- !query 15 output 17208.5 -- !query 16 -SELECT CAST(var_samp(udf(CAST(b AS Decimal(38,0)))) AS decimal(10,3)) FROM aggtest +SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest -- !query 16 schema -struct +struct -- !query 16 output -22944.667 +22944.666666666668 -- !query 17 -SELECT CAST(udf(var_pop(1.0)) AS int), var_samp(udf(2.0)) +SELECT udf(var_pop(1.0)), var_samp(udf(2.0)) -- !query 17 schema -struct +struct -- !query 17 output -0 NaN +0.0 NaN -- !query 18 -SELECT CAST(stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))) AS int), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))) +SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))) -- !query 18 schema -struct +struct -- !query 18 output -0 NaN +0.0 NaN -- !query 19 select sum(udf(CAST(null AS int))) from range(1,4) -- !query 19 schema -struct +struct -- !query 19 output NULL @@ -165,7 +165,7 @@ NULL -- !query 20 select sum(udf(CAST(null AS long))) from range(1,4) -- !query 20 schema -struct +struct -- !query 20 output NULL @@ -173,7 +173,7 @@ NULL -- !query 21 select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4) -- !query 21 schema -struct +struct -- !query 21 output NULL @@ -181,7 +181,7 @@ NULL -- !query 22 select sum(udf(CAST(null AS DOUBLE))) from range(1,4) -- !query 22 schema -struct +struct -- !query 22 output NULL @@ -189,7 +189,7 @@ NULL -- !query 23 select avg(udf(CAST(null AS int))) from range(1,4) -- !query 23 schema -struct +struct -- !query 23 output NULL @@ -197,7 +197,7 @@ NULL -- !query 24 select avg(udf(CAST(null AS long))) from range(1,4) -- !query 24 schema -struct +struct -- !query 24 output NULL @@ -205,7 +205,7 @@ NULL -- !query 25 select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4) -- !query 25 schema -struct +struct -- !query 25 output NULL @@ -213,7 +213,7 @@ NULL -- !query 26 select avg(udf(CAST(null AS DOUBLE))) from range(1,4) -- !query 26 schema -struct +struct -- !query 26 output NULL @@ -221,7 +221,7 @@ NULL -- !query 27 select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4) -- !query 27 schema -struct +struct -- !query 27 output NaN @@ -229,7 +229,7 @@ NaN -- !query 28 select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4) -- !query 28 schema -struct +struct -- !query 28 output NaN @@ -238,7 +238,7 @@ NaN SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) FROM (VALUES ('Infinity'), ('1')) v(x) -- !query 29 schema -struct +struct -- !query 29 output Infinity NaN @@ -247,7 +247,7 @@ Infinity NaN SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) FROM (VALUES ('Infinity'), ('Infinity')) v(x) -- !query 30 schema -struct +struct -- !query 30 output Infinity NaN @@ -256,43 +256,43 @@ Infinity NaN SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) FROM (VALUES ('-Infinity'), ('Infinity')) v(x) -- !query 31 schema -struct +struct -- !query 31 output NaN NaN -- !query 32 -SELECT CAST(avg(udf(CAST(x AS DOUBLE))) AS int), CAST(udf(var_pop(CAST(x AS DOUBLE))) AS decimal(10,3)) +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) -- !query 32 schema -struct +struct -- !query 32 output -100000005 2.5 +1.00000005E8 2.5 -- !query 33 -SELECT CAST(avg(udf(CAST(x AS DOUBLE))) AS long), CAST(udf(var_pop(CAST(x AS DOUBLE))) AS decimal(10,3)) +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) FROM (VALUES (7000000000005), (7000000000007)) v(x) -- !query 33 schema -struct +struct -- !query 33 output -7000000000006 1 +7.000000000006E12 1.0 -- !query 34 -SELECT CAST(udf(covar_pop(b, udf(a))) AS decimal(10,3)), CAST(covar_samp(udf(b), a) as decimal(10,3)) FROM aggtest +SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest -- !query 34 schema -struct +struct -- !query 34 output -653.629 871.505 +653.6289553875104 871.5052738500139 -- !query 35 -SELECT CAST(corr(b, udf(a)) AS decimal(10,3)) FROM aggtest +SELECT corr(b, udf(a)) FROM aggtest -- !query 35 schema -struct +struct -- !query 35 output -0.14 +0.1396345165178734 -- !query 36 @@ -306,16 +306,16 @@ struct -- !query 37 SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek -- !query 37 schema -struct +struct -- !query 37 output 4 -- !query 38 -select ten, udf(count(*)), CAST(sum(udf(four)) AS int) from onek +select ten, udf(count(*)), sum(udf(four)) from onek group by ten order by ten -- !query 38 schema -struct +struct -- !query 38 output 0 100 100 1 100 200 @@ -333,7 +333,7 @@ struct +struct -- !query 39 output 0 100 2 1 100 4 @@ -352,7 +352,7 @@ select ten, udf(sum(distinct four)) from onek a group by ten having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four) -- !query 40 schema -struct +struct -- !query 40 output 0 2 2 2 @@ -372,7 +372,7 @@ struct<> org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. -Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(udf(four) AS BIGINT))] +Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) = CAST(CAST(udf(cast(four as string)) AS INT) AS BIGINT))] Invalid expressions: [sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT))]; diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out index d90aa11fc6ef..9fe943874c3e 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out @@ -59,7 +59,7 @@ true false true false true true true true true -- !query 3 select min(udf(unique1)) from tenk1 -- !query 3 schema -struct +struct -- !query 3 output 0 @@ -67,7 +67,7 @@ struct -- !query 4 select udf(max(unique1)) from tenk1 -- !query 4 schema -struct +struct -- !query 4 output 9999 @@ -115,7 +115,7 @@ struct -- !query 10 select distinct max(udf(unique2)) from tenk1 -- !query 10 schema -struct +struct -- !query 10 output 9999 @@ -139,7 +139,7 @@ struct -- !query 13 select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1 -- !query 13 schema -struct +struct -- !query 13 output 9999 @@ -147,7 +147,7 @@ struct -- !query 14 select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc -- !query 14 schema -struct +struct -- !query 14 output 9999 3 9999 2 @@ -157,6 +157,6 @@ struct -- !query 15 select udf(max(100)) from tenk1 -- !query 15 schema -struct +struct -- !query 15 output 100 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out index 55bef64338f4..d9a8ca86361f 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out @@ -1,19 +1,22 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 37 +-- Number of queries: 35 -- !query 0 -set spark.sql.crossJoin.enabled=true +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet -- !query 0 schema -struct +struct<> -- !query 0 output -spark.sql.crossJoin.enabled true + -- !query 1 -CREATE TABLE CASE_TBL ( +CREATE TABLE CASE2_TBL ( i integer, - f double + j integer ) USING parquet -- !query 1 schema struct<> @@ -22,10 +25,7 @@ struct<> -- !query 2 -CREATE TABLE CASE2_TBL ( - i integer, - j integer -) USING parquet +INSERT INTO CASE_TBL VALUES (1, 10.1) -- !query 2 schema struct<> -- !query 2 output @@ -33,7 +33,7 @@ struct<> -- !query 3 -INSERT INTO CASE_TBL VALUES (1, 10.1) +INSERT INTO CASE_TBL VALUES (2, 20.2) -- !query 3 schema struct<> -- !query 3 output @@ -41,7 +41,7 @@ struct<> -- !query 4 -INSERT INTO CASE_TBL VALUES (2, 20.2) +INSERT INTO CASE_TBL VALUES (3, -30.3) -- !query 4 schema struct<> -- !query 4 output @@ -49,7 +49,7 @@ struct<> -- !query 5 -INSERT INTO CASE_TBL VALUES (3, -30.3) +INSERT INTO CASE_TBL VALUES (4, NULL) -- !query 5 schema struct<> -- !query 5 output @@ -57,7 +57,7 @@ struct<> -- !query 6 -INSERT INTO CASE_TBL VALUES (4, NULL) +INSERT INTO CASE2_TBL VALUES (1, -1) -- !query 6 schema struct<> -- !query 6 output @@ -65,7 +65,7 @@ struct<> -- !query 7 -INSERT INTO CASE2_TBL VALUES (1, -1) +INSERT INTO CASE2_TBL VALUES (2, -2) -- !query 7 schema struct<> -- !query 7 output @@ -73,7 +73,7 @@ struct<> -- !query 8 -INSERT INTO CASE2_TBL VALUES (2, -2) +INSERT INTO CASE2_TBL VALUES (3, -3) -- !query 8 schema struct<> -- !query 8 output @@ -81,7 +81,7 @@ struct<> -- !query 9 -INSERT INTO CASE2_TBL VALUES (3, -3) +INSERT INTO CASE2_TBL VALUES (2, -4) -- !query 9 schema struct<> -- !query 9 output @@ -89,7 +89,7 @@ struct<> -- !query 10 -INSERT INTO CASE2_TBL VALUES (2, -4) +INSERT INTO CASE2_TBL VALUES (1, NULL) -- !query 10 schema struct<> -- !query 10 output @@ -97,7 +97,7 @@ struct<> -- !query 11 -INSERT INTO CASE2_TBL VALUES (1, NULL) +INSERT INTO CASE2_TBL VALUES (NULL, -6) -- !query 11 schema struct<> -- !query 11 output @@ -105,148 +105,140 @@ struct<> -- !query 12 -INSERT INTO CASE2_TBL VALUES (NULL, -6) --- !query 12 schema -struct<> --- !query 12 output - - - --- !query 13 SELECT '3' AS `One`, CASE - WHEN CAST(udf(1 < 2) AS boolean) THEN 3 + WHEN udf(1 < 2) THEN 3 END AS `Simple WHEN` --- !query 13 schema +-- !query 12 schema struct --- !query 13 output +-- !query 12 output 3 3 --- !query 14 +-- !query 13 SELECT '' AS `One`, CASE WHEN 1 > 2 THEN udf(3) END AS `Simple default` --- !query 14 schema -struct --- !query 14 output +-- !query 13 schema +struct +-- !query 13 output NULL --- !query 15 +-- !query 14 SELECT '3' AS `One`, CASE WHEN udf(1) < 2 THEN udf(3) ELSE udf(4) END AS `Simple ELSE` --- !query 15 schema -struct --- !query 15 output +-- !query 14 schema +struct +-- !query 14 output 3 3 --- !query 16 +-- !query 15 SELECT udf('4') AS `One`, CASE WHEN 1 > 2 THEN 3 ELSE 4 END AS `ELSE default` --- !query 16 schema +-- !query 15 schema struct --- !query 16 output +-- !query 15 output 4 4 --- !query 17 +-- !query 16 SELECT udf('6') AS `One`, CASE - WHEN CAST(udf(1 > 2) AS boolean) THEN 3 + WHEN udf(1 > 2) THEN 3 WHEN udf(4) < 5 THEN 6 ELSE 7 END AS `Two WHEN with default` --- !query 17 schema +-- !query 16 schema struct --- !query 17 output +-- !query 16 output 6 6 --- !query 18 +-- !query 17 SELECT '7' AS `None`, CASE WHEN rand() < udf(0) THEN 1 END AS `NULL on no matches` --- !query 18 schema +-- !query 17 schema struct --- !query 18 output +-- !query 17 output 7 NULL +-- !query 18 +SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +-- !query 18 schema +struct +-- !query 18 output +1 + + -- !query 19 -SELECT CASE WHEN CAST(udf(1=0) AS boolean) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END -- !query 19 schema -struct +struct -- !query 19 output -1.0 +1 -- !query 20 -SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END +SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl -- !query 20 schema -struct +struct 100) THEN CAST(udf(cast((1 div 0) as string)) AS INT) ELSE CAST(udf(cast(0 as string)) AS INT) END:int> -- !query 20 output -1.0 - - --- !query 21 -SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl --- !query 21 schema -struct 100) THEN udf((cast(1 as double) / cast(0 as double))) ELSE udf(0) END:string> --- !query 21 output 0 0 0 0 --- !query 22 +-- !query 21 SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END --- !query 22 schema -struct --- !query 22 output +-- !query 21 schema +struct +-- !query 21 output 1 --- !query 23 +-- !query 22 SELECT '' AS `Five`, CASE WHEN i >= 3 THEN i END AS `>= 3 or Null` FROM CASE_TBL --- !query 23 schema +-- !query 22 schema struct= 3 or Null:int> --- !query 23 output +-- !query 22 output 3 4 NULL NULL --- !query 24 +-- !query 23 SELECT '' AS `Five`, CASE WHEN i >= 3 THEN (i + i) ELSE i END AS `Simplest Math` FROM CASE_TBL --- !query 24 schema +-- !query 23 schema struct --- !query 24 output +-- !query 23 output 1 2 6 8 --- !query 25 +-- !query 24 SELECT '' AS `Five`, i AS `Value`, CASE WHEN (i < 0) THEN 'small' WHEN (i = 0) THEN 'zero' @@ -255,16 +247,16 @@ SELECT '' AS `Five`, i AS `Value`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 25 schema +-- !query 24 schema struct --- !query 25 output +-- !query 24 output 1 one 2 two 3 big 4 big --- !query 26 +-- !query 25 SELECT '' AS `Five`, CASE WHEN ((i < 0) or (i < 0)) THEN 'small' WHEN ((i = 0) or (i = 0)) THEN 'zero' @@ -273,37 +265,37 @@ SELECT '' AS `Five`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 26 schema +-- !query 25 schema struct --- !query 26 output +-- !query 25 output big big one two --- !query 27 +-- !query 26 SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4 --- !query 27 schema +-- !query 26 schema struct --- !query 27 output +-- !query 26 output 4 NULL --- !query 28 +-- !query 27 SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2 --- !query 28 schema +-- !query 27 schema struct --- !query 28 output +-- !query 27 output --- !query 29 +-- !query 28 SELECT udf(COALESCE(a.f, b.i, b.j)) FROM CASE_TBL a, CASE2_TBL b --- !query 29 schema -struct --- !query 29 output +-- !query 28 schema +struct +-- !query 28 output -30.3 -30.3 -30.3 @@ -330,24 +322,24 @@ struct 3.0 --- !query 30 +-- !query 29 SELECT * FROM CASE_TBL a, CASE2_TBL b WHERE udf(COALESCE(a.f, b.i, b.j)) = 2 --- !query 30 schema +-- !query 29 schema struct --- !query 30 output +-- !query 29 output 4 NULL 2 -2 4 NULL 2 -4 --- !query 31 +-- !query 30 SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, NULLIF(b.i, 4) AS `NULLIF(b.i,4)` FROM CASE_TBL a, CASE2_TBL b --- !query 31 schema +-- !query 30 schema struct --- !query 31 output +-- !query 30 output 1 2 1 2 1 3 @@ -374,18 +366,18 @@ struct NULL 3 --- !query 32 +-- !query 31 SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b - WHERE CAST(udf(COALESCE(f,b.i) = 2) AS boolean) --- !query 32 schema + WHERE udf(COALESCE(f,b.i) = 2) +-- !query 31 schema struct --- !query 32 output +-- !query 31 output 4 NULL 2 -2 4 NULL 2 -4 --- !query 33 +-- !query 32 SELECT CASE (CASE vol('bar') WHEN udf('foo') THEN 'it was foo!' @@ -395,31 +387,23 @@ SELECT CASE WHEN udf('it was foo!') THEN 'foo recognized' WHEN 'it was bar!' THEN udf('bar recognized') ELSE 'unrecognized' END AS col --- !query 33 schema +-- !query 32 schema struct --- !query 33 output +-- !query 32 output bar recognized --- !query 34 +-- !query 33 DROP TABLE CASE_TBL --- !query 34 schema +-- !query 33 schema struct<> --- !query 34 output +-- !query 33 output --- !query 35 +-- !query 34 DROP TABLE CASE2_TBL --- !query 35 schema +-- !query 34 schema struct<> --- !query 35 output - - +-- !query 34 output --- !query 36 -set spark.sql.crossJoin.enabled=false --- !query 36 schema -struct --- !query 36 output -spark.sql.crossJoin.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out index 9476937abd9e..3d7c64054a6a 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out @@ -17,7 +17,7 @@ SELECT udf(count(*)), udf(count(1)), udf(count(null)), udf(count(a)), udf(count(b)), udf(count(a + b)), udf(count((a, b))) FROM testData -- !query 1 schema -struct +struct -- !query 1 output 7 7 0 5 5 4 7 @@ -32,7 +32,7 @@ SELECT udf(count(DISTINCT (a, b))) FROM testData -- !query 2 schema -struct +struct -- !query 2 output 1 0 2 2 2 6 @@ -40,7 +40,7 @@ struct +struct -- !query 3 output 4 4 4 @@ -50,6 +50,6 @@ SELECT udf(count(DISTINCT a, b)), udf(count(DISTINCT b, a)), udf(count(DISTINCT *)), udf(count(DISTINCT testData.*)) FROM testData -- !query 4 schema -struct +struct -- !query 4 output 3 3 3 3 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 000000000000..98d3ad37a8df --- /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 000000000000..b7bfad0e538a --- /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 000000000000..cb8a4e84a95c --- /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 000000000000..31e9e08e2ca8 --- /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 000000000000..789c91fa3870 --- /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-having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out index 7cea2e5128f8..1effcc8470e1 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out @@ -18,7 +18,7 @@ struct<> -- !query 1 SELECT udf(k) AS k, udf(sum(v)) FROM hav GROUP BY k HAVING udf(sum(v)) > 2 -- !query 1 schema -struct +struct -- !query 1 output one 6 three 3 @@ -27,7 +27,7 @@ three 3 -- !query 2 SELECT udf(count(udf(k))) FROM hav GROUP BY v + 1 HAVING v + 1 = udf(2) -- !query 2 schema -struct +struct -- !query 2 output 1 @@ -35,7 +35,7 @@ struct -- !query 3 SELECT udf(MIN(t.v)) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(udf(COUNT(udf(1))) > 0) -- !query 3 schema -struct +struct -- !query 3 output 1 @@ -43,7 +43,7 @@ struct -- !query 4 SELECT udf(a + b) FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > udf(1) -- !query 4 schema -struct +struct -- !query 4 output 3 7 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 000000000000..2cf24e50c80a --- /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-inner-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out index 10952cb21e4f..120f2d39f73d 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out @@ -59,7 +59,7 @@ struct<> -- !query 6 SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag -- !query 6 schema -struct +struct -- !query 6 output 1 a 1 a 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 000000000000..0cb82be2da22 --- /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 000000000000..e79d01fb14d6 --- /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-natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out index 53ef177db0bb..950809ddcaf2 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out @@ -59,6 +59,6 @@ two 2 22 -- !query 5 SELECT udf(count(*)) FROM nt1 natural full outer join nt2 -- !query 5 schema -struct +struct -- !query 5 output 4 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 000000000000..819f78607088 --- /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 000000000000..cb9e4d736c9a --- /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-special-values.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out new file mode 100644 index 000000000000..7b2b5dbe578c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out @@ -0,0 +1,62 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +SELECT udf(x) FROM (VALUES (1), (2), (NULL)) v(x) +-- !query 0 schema +struct +-- !query 0 output +1 +2 +NULL + + +-- !query 1 +SELECT udf(x) FROM (VALUES ('A'), ('B'), (NULL)) v(x) +-- !query 1 schema +struct +-- !query 1 output +A +B +NULL + + +-- !query 2 +SELECT udf(x) FROM (VALUES ('NaN'), ('1'), ('2')) v(x) +-- !query 2 schema +struct +-- !query 2 output +1 +2 +NaN + + +-- !query 3 +SELECT udf(x) FROM (VALUES ('Infinity'), ('1'), ('2')) v(x) +-- !query 3 schema +struct +-- !query 3 output +1 +2 +Infinity + + +-- !query 4 +SELECT udf(x) FROM (VALUES ('-Infinity'), ('1'), ('2')) v(x) +-- !query 4 schema +struct +-- !query 4 output +-Infinity +1 +2 + + +-- !query 5 +SELECT udf(x) FROM (VALUES 0.00000001, 0.00000002, 0.00000003) v(x) +-- !query 5 schema +struct +-- !query 5 output +0.00000001 +0.00000002 +0.00000003 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 000000000000..e1747f466779 --- /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/scala/org/apache/spark/sql/AggregateHashMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala index 938d76c9f083..b253c4a70bbf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala @@ -20,33 +20,34 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf +import org.apache.spark.sql.internal.SQLConf class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.codegen.fallback", "false") - .set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key, "false") // adding some checking after each test is run, assuring that the configs are not changed // in test code after { - assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + assert(sparkConf.get(SQLConf.CODEGEN_FALLBACK.key) == "false", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enabled") == "false", + assert(sparkConf.get(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key) == "false", "configuration parameter changed in test body") } } class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.codegen.fallback", "false") - .set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key, "true") // adding some checking after each test is run, assuring that the configs are not changed // in test code after { - assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + assert(sparkConf.get(SQLConf.CODEGEN_FALLBACK.key) == "false", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enabled") == "true", + assert(sparkConf.get(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key) == "true", "configuration parameter changed in test body") } } @@ -56,18 +57,18 @@ class TwoLevelAggregateHashMapWithVectorizedMapSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.codegen.fallback", "false") - .set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - .set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key, "true") + .set(SQLConf.ENABLE_VECTORIZED_HASH_MAP.key, "true") // adding some checking after each test is run, assuring that the configs are not changed // in test code after { - assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + assert(sparkConf.get(SQLConf.CODEGEN_FALLBACK.key) == "false", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enabled") == "true", + assert(sparkConf.get(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key) == "true", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.vectorized.enable") == "true", + assert(sparkConf.get(SQLConf.ENABLE_VECTORIZED_HASH_MAP.key) == "true", "configuration parameter changed in test body") } } 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 e005a3e9a258..e49ef012f5eb 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 98936702a013..8782f95fe517 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") { @@ -1672,7 +1664,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("reuse exchange") { - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "2") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2") { val df = spark.range(100).toDF() val join = df.join(df, "id") val plan = join.queryExecution.executedPlan 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 f4ba2f0673c0..a07553008fd7 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/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index efd5db1c5b6c..ff6143162ff2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -603,6 +603,70 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ("a", 30L, 32L, 2L, 15.0), ("b", 3L, 5L, 2L, 1.5), ("c", 1L, 2L, 1L, 1.0)) } + test("typed aggregation: expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long]), + ("a", 30L, 32L, 2L, 15.0, 2L), ("b", 3L, 5L, 2L, 1.5, 2L), ("c", 1L, 2L, 1L, 1.0, 1L)) + } + + test("typed aggregation: expr, expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long], + max("_2").as[Long]), + ("a", 30L, 32L, 2L, 15.0, 2L, 20L), + ("b", 3L, 5L, 2L, 1.5, 2L, 2L), + ("c", 1L, 2L, 1L, 1.0, 1L, 1L)) + } + + test("typed aggregation: expr, expr, expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long], + max("_2").as[Long], + min("_2").as[Long]), + ("a", 30L, 32L, 2L, 15.0, 2L, 20L, 10L), + ("b", 3L, 5L, 2L, 1.5, 2L, 2L, 1L), + ("c", 1L, 2L, 1L, 1.0, 1L, 1L, 1L)) + } + + test("typed aggregation: expr, expr, expr, expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long], + max("_2").as[Long], + min("_2").as[Long], + mean("_2").as[Double]), + ("a", 30L, 32L, 2L, 15.0, 2L, 20L, 10L, 15.0), + ("b", 3L, 5L, 2L, 1.5, 2L, 2L, 1L, 1.5), + ("c", 1L, 2L, 1L, 1.0, 1L, 1L, 1L, 1.0)) + } + test("cogroup") { val ds1 = Seq(1 -> "a", 3 -> "abc", 5 -> "hello", 3 -> "foo").toDS() val ds2 = Seq(2 -> "q", 3 -> "w", 5 -> "e", 5 -> "r").toDS() @@ -1365,7 +1429,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val agg = cp.groupBy('id % 2).agg(count('id)) agg.queryExecution.executedPlan.collectFirst { - case ShuffleExchangeExec(_, _: RDDScanExec) => + case ShuffleExchangeExec(_, _: RDDScanExec, _) => case BroadcastExchangeExec(_, _: RDDScanExec) => }.foreach { _ => fail( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 3f91b91850e8..ff48ac8d7a6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -301,11 +301,11 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( df.selectExpr(s"d - $i"), - Seq(Row(Date.valueOf("2015-07-30")), Row(Date.valueOf("2015-12-30")))) + Seq(Row(Date.valueOf("2015-07-29")), Row(Date.valueOf("2015-12-28")))) checkAnswer( df.selectExpr(s"t - $i"), Seq(Row(Timestamp.valueOf("2015-07-31 23:59:59")), - Row(Timestamp.valueOf("2015-12-31 00:00:00")))) + Row(Timestamp.valueOf("2015-12-29 00:00:00")))) } test("function add_months") { @@ -314,10 +314,10 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val df = Seq((1, d1), (2, d2)).toDF("n", "d") checkAnswer( df.select(add_months(col("d"), 1)), - Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2015-03-31")))) + Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2015-03-28")))) checkAnswer( df.selectExpr("add_months(d, -1)"), - Seq(Row(Date.valueOf("2015-07-31")), Row(Date.valueOf("2015-01-31")))) + Seq(Row(Date.valueOf("2015-07-31")), Row(Date.valueOf("2015-01-28")))) } test("function months_between") { 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 fffe52d52dec..51e26d42812c 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)) } @@ -671,8 +671,8 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo test("SPARK-22790,SPARK-27668: spark.sql.sources.compressionFactor takes effect") { Seq(1.0, 0.5).foreach { compressionFactor => - withSQLConf("spark.sql.sources.fileCompressionFactor" -> compressionFactor.toString, - "spark.sql.autoBroadcastJoinThreshold" -> "250") { + withSQLConf(SQLConf.FILE_COMRESSION_FACTOR.key -> compressionFactor.toString, + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "250") { withTempPath { workDir => // the file size is 486 bytes val workDirPath = workDir.getAbsolutePath diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index e379d6df867c..d62fe961117a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -26,6 +26,7 @@ import org.apache.spark.TestUtils import org.apache.spark.api.python.{PythonBroadcast, PythonEvalType, PythonFunction, PythonUtils} import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.config.Tests +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.execution.python.UserDefinedPythonFunction import org.apache.spark.sql.expressions.SparkUserDefinedFunction @@ -35,8 +36,12 @@ import org.apache.spark.sql.types.StringType * This object targets to integrate various UDF test cases so that Scalar UDF, Python UDF and * Scalar Pandas UDFs can be tested in SBT & Maven tests. * - * The available UDFs cast input to strings, which take one column as input and return a string - * type column as output. + * The available UDFs are special. It defines an UDF wrapped by cast. So, the input column is + * casted into string, UDF returns strings as are, and then output column is casted back to + * the input column. In this way, UDF is virtually no-op. + * + * Note that, due to this implementation limitation, complex types such as map, array and struct + * types do not work with this UDFs because they cannot be same after the cast roundtrip. * * To register Scala UDF in SQL: * {{{ @@ -59,8 +64,9 @@ import org.apache.spark.sql.types.StringType * To use it in Scala API and SQL: * {{{ * sql("SELECT udf_name(1)") - * spark.range(10).select(expr("udf_name(id)") - * spark.range(10).select(pandasTestUDF($"id")) + * val df = spark.range(10) + * df.select(expr("udf_name(id)") + * df.select(pandasTestUDF(df("id"))) * }}} */ object IntegratedUDFTestUtils extends SQLHelper { @@ -137,7 +143,8 @@ object IntegratedUDFTestUtils extends SQLHelper { "from pyspark.sql.types import StringType; " + "from pyspark.serializers import CloudPickleSerializer; " + s"f = open('$path', 'wb');" + - s"f.write(CloudPickleSerializer().dumps((lambda x: str(x), StringType())))"), + "f.write(CloudPickleSerializer().dumps((" + + "lambda x: None if x is None else str(x), StringType())))"), None, "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!! binaryPythonFunc = Files.readAllBytes(path.toPath) @@ -158,7 +165,9 @@ object IntegratedUDFTestUtils extends SQLHelper { "from pyspark.sql.types import StringType; " + "from pyspark.serializers import CloudPickleSerializer; " + s"f = open('$path', 'wb');" + - s"f.write(CloudPickleSerializer().dumps((lambda x: x.apply(str), StringType())))"), + "f.write(CloudPickleSerializer().dumps((" + + "lambda x: x.apply(" + + "lambda v: None if v is None else str(v)), StringType())))"), None, "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!! binaryPandasFunc = Files.readAllBytes(path.toPath) @@ -198,11 +207,22 @@ object IntegratedUDFTestUtils extends SQLHelper { } /** - * A Python UDF that takes one column and returns a string column. - * Equivalent to `udf(lambda x: str(x), "string")` + * A Python UDF that takes one column, casts into string, executes the Python native function, + * and casts back to the type of input column. + * + * Virtually equivalent to: + * + * {{{ + * from pyspark.sql.functions import udf + * + * df = spark.range(3).toDF("col") + * python_udf = udf(lambda x: str(x), "string") + * casted_col = python_udf(df.col.cast("string")) + * casted_col.cast(df.schema["col"].dataType) + * }}} */ case class TestPythonUDF(name: String) extends TestUDF { - private[IntegratedUDFTestUtils] lazy val udf = UserDefinedPythonFunction( + private[IntegratedUDFTestUtils] lazy val udf = new UserDefinedPythonFunction( name = name, func = PythonFunction( command = pythonFunc, @@ -214,7 +234,16 @@ object IntegratedUDFTestUtils extends SQLHelper { accumulator = null), dataType = StringType, pythonEvalType = PythonEvalType.SQL_BATCHED_UDF, - udfDeterministic = true) + udfDeterministic = true) { + + override def builder(e: Seq[Expression]): Expression = { + assert(e.length == 1, "Defined UDF only has one column") + val expr = e.head + assert(expr.resolved, "column should be resolved to use the same type " + + "as input. Try df(name) or df.col(name)") + Cast(super.builder(Cast(expr, StringType) :: Nil), expr.dataType) + } + } def apply(exprs: Column*): Column = udf(exprs: _*) @@ -222,11 +251,22 @@ object IntegratedUDFTestUtils extends SQLHelper { } /** - * A Scalar Pandas UDF that takes one column and returns a string column. - * Equivalent to `pandas_udf(lambda x: x.apply(str), "string", PandasUDFType.SCALAR)`. + * A Scalar Pandas UDF that takes one column, casts into string, executes the + * Python native function, and casts back to the type of input column. + * + * Virtually equivalent to: + * + * {{{ + * from pyspark.sql.functions import pandas_udf + * + * df = spark.range(3).toDF("col") + * scalar_udf = pandas_udf(lambda x: x.apply(lambda v: str(v)), "string") + * casted_col = scalar_udf(df.col.cast("string")) + * casted_col.cast(df.schema["col"].dataType) + * }}} */ case class TestScalarPandasUDF(name: String) extends TestUDF { - private[IntegratedUDFTestUtils] lazy val udf = UserDefinedPythonFunction( + private[IntegratedUDFTestUtils] lazy val udf = new UserDefinedPythonFunction( name = name, func = PythonFunction( command = pandasFunc, @@ -238,7 +278,16 @@ object IntegratedUDFTestUtils extends SQLHelper { accumulator = null), dataType = StringType, pythonEvalType = PythonEvalType.SQL_SCALAR_PANDAS_UDF, - udfDeterministic = true) + udfDeterministic = true) { + + override def builder(e: Seq[Expression]): Expression = { + assert(e.length == 1, "Defined UDF only has one column") + val expr = e.head + assert(expr.resolved, "column should be resolved to use the same type " + + "as input. Try df(name) or df.col(name)") + Cast(super.builder(Cast(expr, StringType) :: Nil), expr.dataType) + } + } def apply(exprs: Column*): Column = udf(exprs: _*) @@ -246,15 +295,39 @@ object IntegratedUDFTestUtils extends SQLHelper { } /** - * A Scala UDF that takes one column and returns a string column. - * Equivalent to `udf((input: Any) => String.valueOf(input)`. + * A Scala UDF that takes one column, casts into string, executes the + * Scala native function, and casts back to the type of input column. + * + * Virtually equivalent to: + * + * {{{ + * import org.apache.spark.sql.functions.udf + * + * val df = spark.range(3).toDF("col") + * val scala_udf = udf((input: Any) => input.toString) + * val casted_col = scala_udf(df.col("col").cast("string")) + * casted_col.cast(df.schema("col").dataType) + * }}} */ case class TestScalaUDF(name: String) extends TestUDF { - private[IntegratedUDFTestUtils] lazy val udf = SparkUserDefinedFunction( - (input: Any) => String.valueOf(input), + private[IntegratedUDFTestUtils] lazy val udf = new SparkUserDefinedFunction( + (input: Any) => if (input == null) { + null + } else { + input.toString + }, StringType, inputSchemas = Seq.fill(1)(None), - name = Some(name)) + name = Some(name)) { + + override def apply(exprs: Column*): Column = { + assert(exprs.length == 1, "Defined UDF only has one column") + val expr = exprs.head.expr + assert(expr.resolved, "column should be resolved to use the same type " + + "as input. Try df(name) or df.col(name)") + Column(Cast(createScalaUDF(Cast(expr, StringType) :: Nil), expr.dataType)) + } + } def apply(exprs: Column*): Column = udf(exprs: _*) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 32cddc94166b..059dbf892c65 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -26,7 +26,8 @@ import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder} -import org.apache.spark.sql.execution.{BinaryExecNode, SortExec} +import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec} import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.BatchEvalPythonExec import org.apache.spark.sql.internal.SQLConf @@ -72,7 +73,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("join operator selection") { spark.sharedState.cacheManager.clearCache() - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0", + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", @@ -651,7 +652,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("test SortMergeJoin (without spill)") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", - "spark.sql.sortMergeJoinExec.buffer.spill.threshold" -> Int.MaxValue.toString) { + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> Int.MaxValue.toString) { assertNotSpilled(sparkContext, "inner join") { checkAnswer( @@ -708,8 +709,8 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("test SortMergeJoin (with spill)") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", - "spark.sql.sortMergeJoinExec.buffer.in.memory.threshold" -> "0", - "spark.sql.sortMergeJoinExec.buffer.spill.threshold" -> "1") { + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "0", + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") { assertSpilled(sparkContext, "inner join") { checkAnswer( @@ -897,6 +898,26 @@ class JoinSuite extends QueryTest with SharedSQLContext { } } + test("SPARK-27485: EnsureRequirements should not fail join with duplicate keys") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val tbl_a = spark.range(40) + .select($"id" as "x", $"id" % 10 as "y") + .repartition(2, $"x", $"y", $"x") + .as("tbl_a") + + val tbl_b = spark.range(20) + .select($"id" as "x", $"id" % 2 as "y1", $"id" % 20 as "y2") + .as("tbl_b") + + val res = tbl_a + .join(tbl_b, + $"tbl_a.x" === $"tbl_b.x" && $"tbl_a.y" === $"tbl_b.y1" && $"tbl_a.y" === $"tbl_b.y2") + .select($"tbl_a.x") + checkAnswer(res, Row(0L) :: Row(1L) :: Nil) + } + } + test("SPARK-26352: join reordering should not change the order of columns") { withTable("tab1", "tab2", "tab3") { spark.sql("select 1 as x, 100 as y").write.saveAsTable("tab1") @@ -980,7 +1001,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { val left = Seq((1, 2), (2, 3)).toDF("a", "b") val right = Seq((1, 2), (3, 4)).toDF("c", "d") - val df = left.join(right, pythonTestUDF($"a") === pythonTestUDF($"c")) + val df = left.join(right, pythonTestUDF(left("a")) === pythonTestUDF(right.col("c"))) val joinNode = df.queryExecution.executedPlan.find(_.isInstanceOf[BroadcastHashJoinExec]) assert(joinNode.isDefined) @@ -994,4 +1015,26 @@ class JoinSuite extends QueryTest with SharedSQLContext { checkAnswer(df, Row(1, 2, 1, 2) :: Nil) } + + test("SPARK-28345: PythonUDF predicate should be able to pushdown to join") { + import IntegratedUDFTestUtils._ + + assume(shouldTestPythonUDFs) + + val pythonTestUDF = TestPythonUDF(name = "udf") + + val left = Seq((1, 2), (2, 3)).toDF("a", "b") + val right = Seq((1, 2), (3, 4)).toDF("c", "d") + val df = left.crossJoin(right).where(pythonTestUDF(left("a")) === right.col("c")) + + // Before optimization, there is a logical Filter operator. + val filterInAnalysis = df.queryExecution.analyzed.find(_.isInstanceOf[Filter]) + assert(filterInAnalysis.isDefined) + + // Filter predicate was pushdown as join condition. So there is no Filter exec operator. + val filterExec = df.queryExecution.executedPlan.find(_.isInstanceOf[FilterExec]) + assert(filterExec.isEmpty) + + checkAnswer(df, Row(1, 2, 1, 2) :: Nil) + } } 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 24e7564259c8..ae6e9037acd2 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/ProcessingTimeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala index 623a1b6f854c..e33870d4e1af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala @@ -22,12 +22,15 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.streaming.{ProcessingTime, Trigger} +import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger +import org.apache.spark.sql.streaming.Trigger class ProcessingTimeSuite extends SparkFunSuite { test("create") { - def getIntervalMs(trigger: Trigger): Long = trigger.asInstanceOf[ProcessingTime].intervalMs + def getIntervalMs(trigger: Trigger): Long = { + trigger.asInstanceOf[ProcessingTimeTrigger].intervalMs + } assert(getIntervalMs(Trigger.ProcessingTime(10.seconds)) === 10 * 1000) assert(getIntervalMs(Trigger.ProcessingTime(10, TimeUnit.SECONDS)) === 10 * 1000) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala index 328423160696..720d570ca838 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import org.apache.spark.SparkFunSuite import org.apache.spark.internal.config +import org.apache.spark.sql.internal.SQLConf.CHECKPOINT_LOCATION +import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD class RuntimeConfigSuite extends SparkFunSuite { @@ -60,8 +62,8 @@ class RuntimeConfigSuite extends SparkFunSuite { val conf = newConf() // SQL configs - assert(!conf.isModifiable("spark.sql.sources.schemaStringLengthThreshold")) - assert(conf.isModifiable("spark.sql.streaming.checkpointLocation")) + assert(!conf.isModifiable(SCHEMA_STRING_LENGTH_THRESHOLD.key)) + assert(conf.isModifiable(CHECKPOINT_LOCATION.key)) // Core configs assert(!conf.isModifiable(config.CPUS_PER_TASK.key)) assert(!conf.isModifiable("spark.executor.cores")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 2cc1be9fdda2..972950669198 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1896,7 +1896,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("Star Expansion - group by") { - withSQLConf("spark.sql.retainGroupColumns" -> "false") { + withSQLConf(SQLConf.DATAFRAME_RETAIN_GROUP_COLUMNS.key -> "false") { checkAnswer( testData2.groupBy($"a", $"b").agg($"*"), sql("SELECT * FROM testData2 group by a, b")) @@ -1936,7 +1936,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Common subexpression elimination") { // TODO: support subexpression elimination in whole stage codegen - withSQLConf("spark.sql.codegen.wholeStage" -> "false") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { // select from a table to prevent constant folding. val df = sql("SELECT a, b from testData2 limit 1") checkAnswer(df, Row(1, 1)) @@ -1985,9 +1985,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { df.selectExpr("testUdf(a + 1) + testUdf(1 + a)", "testUdf(a + 1)"), Row(4, 2), 1) // Try disabling it via configuration. - spark.conf.set("spark.sql.subexpressionElimination.enabled", "false") + spark.conf.set(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "false") verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 2) - spark.conf.set("spark.sql.subexpressionElimination.enabled", "true") + spark.conf.set(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "true") verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 1) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index bf77d8b4f4d0..8bd8430496d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -152,17 +152,37 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { val resultFile: String } + /** + * traits that indicate UDF or PgSQL to trigger the code path specific to each. For instance, + * PgSQL tests require to register some UDF functions. + */ + private trait PgSQLTest + + private trait UDFTest { + val udf: TestUDF + } + /** A regular test case. */ private case class RegularTestCase( name: String, inputFile: String, resultFile: String) extends TestCase /** A PostgreSQL test case. */ private case class PgSQLTestCase( - name: String, inputFile: String, resultFile: String) extends TestCase + name: String, inputFile: String, resultFile: String) extends TestCase with PgSQLTest /** A UDF test case. */ private case class UDFTestCase( - name: String, inputFile: String, resultFile: String, udf: TestUDF) extends TestCase + name: String, + inputFile: String, + resultFile: String, + udf: TestUDF) extends TestCase with UDFTest + + /** A UDF PostgreSQL test case. */ + private case class UDFPgSQLTestCase( + name: String, + inputFile: String, + resultFile: String, + udf: TestUDF) extends TestCase with UDFTest with PgSQLTest private def createScalaTestCase(testCase: TestCase): Unit = { if (blackList.exists(t => @@ -170,12 +190,14 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // Create a test case to ignore this case. ignore(testCase.name) { /* Do nothing */ } } else testCase match { - case UDFTestCase(_, _, _, udf: TestPythonUDF) if !shouldTestPythonUDFs => + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestPythonUDF] && !shouldTestPythonUDFs => ignore(s"${testCase.name} is skipped because " + s"[$pythonExec] and/or pyspark were not available.") { /* Do nothing */ } - case UDFTestCase(_, _, _, udf: TestScalarPandasUDF) if !shouldTestScalarPandasUDFs => + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestScalarPandasUDF] && !shouldTestScalarPandasUDFs => ignore(s"${testCase.name} is skipped because pyspark," + s"pandas and/or pyarrow were not available in [$pythonExec].") { /* Do nothing */ @@ -255,18 +277,25 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // This does not isolate catalog changes. val localSparkSession = spark.newSession() loadTestData(localSparkSession) + testCase match { - case udfTestCase: UDFTestCase => - // vol used by udf-case.sql. - localSparkSession.udf.register("vol", (s: String) => s) + case udfTestCase: UDFTest => registerTestUDF(udfTestCase.udf, localSparkSession) - case _: PgSQLTestCase => + case _ => + } + + testCase match { + case _: PgSQLTest => // booleq/boolne used by boolean.sql localSparkSession.udf.register("booleq", (b1: Boolean, b2: Boolean) => b1 == b2) localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) - case _ => // Don't add UDFs in Regular tests. + // PostgreSQL enabled cartesian product by default. + localSparkSession.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.ANSI_SQL_PARSER.key, true) + localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) + case _ => } if (configSet.isDefined) { @@ -389,13 +418,16 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { val absPath = file.getAbsolutePath val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator) - if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}udf")) { + if (file.getAbsolutePath.startsWith( + s"$inputFilePath${File.separator}udf${File.separator}pgSQL")) { + Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { udf => + UDFPgSQLTestCase( + s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) + } + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}udf")) { Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { udf => UDFTestCase( - s"$testCaseName - ${udf.prettyName}", - absPath, - resultFile, - udf) + s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) } } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}pgSQL")) { PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 2e2e61b43896..74341f93dd5b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.internal.StaticSQLConf.SPARK_SESSION_EXTENSIONS import org.apache.spark.sql.types.{DataType, Decimal, IntegerType, LongType, Metadata, StructType} import org.apache.spark.sql.vectorized.{ColumnarArray, ColumnarBatch, ColumnarMap, ColumnVector} import org.apache.spark.unsafe.types.UTF8String @@ -152,7 +153,7 @@ class SparkSessionExtensionSuite extends SparkFunSuite { test("use custom class for extensions") { val session = SparkSession.builder() .master("local[1]") - .config("spark.sql.extensions", classOf[MyExtensions].getCanonicalName) + .config(SPARK_SESSION_EXTENSIONS.key, classOf[MyExtensions].getCanonicalName) .getOrCreate() try { assert(session.sessionState.planner.strategies.contains(MySparkStrategy(session))) @@ -173,7 +174,7 @@ class SparkSessionExtensionSuite extends SparkFunSuite { test("use multiple custom class for extensions in the specified order") { val session = SparkSession.builder() .master("local[1]") - .config("spark.sql.extensions", Seq( + .config(SPARK_SESSION_EXTENSIONS.key, Seq( classOf[MyExtensions2].getCanonicalName, classOf[MyExtensions].getCanonicalName).mkString(",")) .getOrCreate() @@ -201,7 +202,7 @@ class SparkSessionExtensionSuite extends SparkFunSuite { test("allow an extension to be duplicated") { val session = SparkSession.builder() .master("local[1]") - .config("spark.sql.extensions", Seq( + .config(SPARK_SESSION_EXTENSIONS.key, Seq( classOf[MyExtensions].getCanonicalName, classOf[MyExtensions].getCanonicalName).mkString(",")) .getOrCreate() @@ -228,7 +229,7 @@ class SparkSessionExtensionSuite extends SparkFunSuite { test("use the last registered function name when there are duplicates") { val session = SparkSession.builder() .master("local[1]") - .config("spark.sql.extensions", Seq( + .config(SPARK_SESSION_EXTENSIONS.key, Seq( classOf[MyExtensions2].getCanonicalName, classOf[MyExtensions2Duplicate].getCanonicalName).mkString(",")) .getOrCreate() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index f155b5dc80cf..058c5ba7e50b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -514,4 +514,13 @@ class UDFSuite extends QueryTest with SharedSQLContext { assert(df.collect().toSeq === Seq(Row(expected))) } } + + test("SPARK-28321 0-args Java UDF should not be called only once") { + val nonDeterministicJavaUDF = udf( + new UDF0[Int] { + override def call(): Int = scala.util.Random.nextInt() + }, IntegerType).asNondeterministic() + + assert(spark.range(2).select(nonDeterministicJavaUDF()).distinct().count() == 2) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index c2d9e5498192..e30fb13d10df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -696,6 +696,32 @@ class PlannerSuite extends SharedSQLContext { } } + test("SPARK-27485: EnsureRequirements.reorder should handle duplicate expressions") { + val plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: exprA :: Nil, 5)) + val plan2 = DummySparkPlan() + val smjExec = SortMergeJoinExec( + leftKeys = exprA :: exprB :: exprB :: Nil, + rightKeys = exprA :: exprC :: exprC :: Nil, + joinType = Inner, + condition = None, + left = plan1, + right = plan2) + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(smjExec) + outputPlan match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, + ShuffleExchangeExec(HashPartitioning(leftPartitioningExpressions, _), _, _), _), + SortExec(_, _, + ShuffleExchangeExec(HashPartitioning(rightPartitioningExpressions, _), _, _), _)) => + assert(leftKeys === smjExec.leftKeys) + assert(rightKeys === smjExec.rightKeys) + assert(leftKeys === leftPartitioningExpressions) + assert(rightKeys === rightPartitioningExpressions) + case _ => fail(outputPlan.toString) + } + } + test("SPARK-24500: create union with stream of children") { val df = Union(Stream( Range(1, 1, 1, 1), 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 7de5e826f667..39c87c9eeb47 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/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 20fed07d3872..35c33a7157d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -574,22 +574,17 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA withSparkSession(test, 4, None) } - test("Union two datasets with different pre-shuffle partition number") { + test("Do not reduce the number of shuffle partition for repartition") { val test: SparkSession => Unit = { spark: SparkSession => - val dataset1 = spark.range(3) - val dataset2 = spark.range(3) - - val resultDf = dataset1.repartition(2, dataset1.col("id")) - .union(dataset2.repartition(3, dataset2.col("id"))).toDF() + val ds = spark.range(3) + val resultDf = ds.repartition(2, ds.col("id")).toDF() checkAnswer(resultDf, - Seq((0), (0), (1), (1), (2), (2)).map(i => Row(i))) + Seq(0, 1, 2).map(i => Row(i))) val finalPlan = resultDf.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan - // As the pre-shuffle partition number are different, we will skip reducing - // the shuffle partition numbers. assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 0) } - withSparkSession(test, 100, None) + withSparkSession(test, 200, None) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 8269d4d3a285..64e305cd5c37 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.internal.SQLConf.MAX_NESTED_VIEW_DEPTH import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} class SimpleSQLViewSuite extends SQLViewSuite with SharedSQLContext @@ -665,17 +666,17 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { sql(s"CREATE VIEW view${idx + 1} AS SELECT * FROM view$idx") } - withSQLConf("spark.sql.view.maxNestedViewDepth" -> "10") { + withSQLConf(MAX_NESTED_VIEW_DEPTH.key -> "10") { val e = intercept[AnalysisException] { sql("SELECT * FROM view10") }.getMessage assert(e.contains("The depth of view `default`.`view0` exceeds the maximum view " + "resolution depth (10). Analysis is aborted to avoid errors. Increase the value " + - "of spark.sql.view.maxNestedViewDepth to work around this.")) + s"of ${MAX_NESTED_VIEW_DEPTH.key} to work around this.")) } val e = intercept[IllegalArgumentException] { - withSQLConf("spark.sql.view.maxNestedViewDepth" -> "0") {} + withSQLConf(MAX_NESTED_VIEW_DEPTH.key -> "0") {} }.getMessage assert(e.contains("The maximum depth of a view reference in a nested view must be " + "positive.")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala index 1c6fc3530cbe..971fd842f046 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.TestUtils.assertSpilled import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD} import org.apache.spark.sql.test.SharedSQLContext case class WindowData(month: Int, area: String, product: Int) @@ -477,8 +478,8 @@ class SQLWindowFunctionSuite extends QueryTest with SharedSQLContext { |WINDOW w1 AS (ORDER BY x ROWS BETWEEN UNBOUNDED PRECEDiNG AND CURRENT RoW) """.stripMargin) - withSQLConf("spark.sql.windowExec.buffer.in.memory.threshold" -> "1", - "spark.sql.windowExec.buffer.spill.threshold" -> "2") { + withSQLConf(WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "1", + WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "2") { assertSpilled(sparkContext, "test with low buffer spill threshold") { checkAnswer(actual, expected) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index a276e477c83c..483a04610338 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -17,11 +17,9 @@ package org.apache.spark.sql.execution -import org.apache.spark.metrics.source.CodegenMetrics -import org.apache.spark.sql.{QueryTest, Row, SaveMode} +import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeGenerator} import org.apache.spark.sql.execution.aggregate.HashAggregateExec -import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.expressions.scalalang.typed @@ -145,10 +143,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { .select("int") val plan = df.queryExecution.executedPlan - assert(!plan.find(p => + assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && p.asInstanceOf[WholeStageCodegenExec].child.children(0) - .isInstanceOf[SortMergeJoinExec]).isDefined) + .isInstanceOf[SortMergeJoinExec]).isEmpty) assert(df.collect() === Array(Row(1), Row(2))) } } @@ -181,6 +179,13 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { wholeStageCodeGenExec.get.asInstanceOf[WholeStageCodegenExec].doCodeGen()._2 } + def genCode(ds: Dataset[_]): Seq[CodeAndComment] = { + val plan = ds.queryExecution.executedPlan + val wholeStageCodeGenExecs = plan.collect { case p: WholeStageCodegenExec => p } + assert(wholeStageCodeGenExecs.nonEmpty, "WholeStageCodegenExec is expected") + wholeStageCodeGenExecs.map(_.doCodeGen()._2) + } + ignore("SPARK-21871 check if we can get large code size when compiling too long functions") { val codeWithShortFunctions = genGroupByCode(3) val (_, maxCodeSize1) = CodeGenerator.compile(codeWithShortFunctions) @@ -241,9 +246,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { val df = spark.range(100) val join = df.join(df, "id") val plan = join.queryExecution.executedPlan - assert(!plan.find(p => + assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].codegenStageId == 0).isDefined, + p.asInstanceOf[WholeStageCodegenExec].codegenStageId == 0).isEmpty, "codegen stage IDs should be preserved through ReuseExchange") checkAnswer(join, df.toDF) } @@ -253,18 +258,13 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { import testImplicits._ withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_USE_ID_IN_CLASS_NAME.key -> "true") { - val bytecodeSizeHisto = CodegenMetrics.METRIC_COMPILATION_TIME - - // the same query run twice should hit the codegen cache - spark.range(3).select('id + 2).collect - val after1 = bytecodeSizeHisto.getCount - spark.range(3).select('id + 2).collect - val after2 = bytecodeSizeHisto.getCount // same query shape as above, deliberately - // bytecodeSizeHisto's count is always monotonically increasing if new compilation to - // bytecode had occurred. If the count stayed the same that means we've got a cache hit. - assert(after1 == after2, "Should hit codegen cache. No new compilation to bytecode expected") - - // a different query can result in codegen cache miss, that's by design + // the same query run twice should produce identical code, which would imply a hit in + // the generated code cache. + val ds1 = spark.range(3).select('id + 2) + val code1 = genCode(ds1) + val ds2 = spark.range(3).select('id + 2) + val code2 = genCode(ds2) // same query shape as above, deliberately + assert(code1 == code2, "Should produce same code") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala index 86874b9817c2..67c3fa0d3bf5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala @@ -1191,7 +1191,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { test("max records in batch conf") { val totalRecords = 10 val maxRecordsPerBatch = 3 - spark.conf.set("spark.sql.execution.arrow.maxRecordsPerBatch", maxRecordsPerBatch) + spark.conf.set(SQLConf.ARROW_EXECUTION_MAX_RECORDS_PER_BATCH.key, maxRecordsPerBatch) val df = spark.sparkContext.parallelize(1 to totalRecords, 2).toDF("i") val arrowBatches = df.toArrowBatchRdd.collect() assert(arrowBatches.length >= 4) @@ -1206,7 +1206,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { } assert(recordCount == totalRecords) allocator.close() - spark.conf.unset("spark.sql.execution.arrow.maxRecordsPerBatch") + spark.conf.unset(SQLConf.ARROW_EXECUTION_MAX_RECORDS_PER_BATCH.key) } testQuietly("unsupported types") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 81158d9e5424..2776bc310fef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -83,7 +83,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -92,7 +92,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -119,7 +119,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -128,7 +128,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -154,7 +154,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -163,7 +163,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -189,7 +189,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -198,7 +198,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -234,7 +234,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -243,7 +243,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala index cd97324c997f..6925bdd72674 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.benchmark +import org.apache.spark.sql.internal.SQLConf + /** * Benchmark to measure built-in data sources write performance. * To run this benchmark: @@ -45,8 +47,8 @@ object BuiltInDataSourceWriteBenchmark extends DataSourceWriteBenchmark { mainArgs } - spark.conf.set("spark.sql.parquet.compression.codec", "snappy") - spark.conf.set("spark.sql.orc.compression.codec", "snappy") + spark.conf.set(SQLConf.PARQUET_COMPRESSION.key, "snappy") + spark.conf.set(SQLConf.ORC_COMPRESSION.key, "snappy") formats.foreach { format => runBenchmark(s"$format writer benchmark") { 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 c4662c8999e4..bafc0337bdc0 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/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 466baf208abc..711ecf1738ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -437,7 +437,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-20356: pruned InMemoryTableScanExec should have correct ordering and partitioning") { - withSQLConf("spark.sql.shuffle.partitions" -> "200") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "200") { val df1 = Seq(("a", 1), ("b", 1), ("c", 2)).toDF("item", "group") val df2 = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("item", "id") val df3 = df1.join(df2, Seq("item")).select($"id", $"group".as("item")).distinct() 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 b3a5c687f775..7b2ccca403bb 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 @@ -50,7 +50,7 @@ class PartitionBatchPruningSuite // Enable in-memory partition pruning spark.conf.set(SQLConf.IN_MEMORY_PARTITION_PRUNING.key, true) // Enable in-memory table scan accumulators - spark.conf.set("spark.sql.inMemoryTableScanStatistics.enable", "true") + spark.conf.set(SQLConf.IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED.key, "true") } override protected def afterAll(): Unit = { @@ -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/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 7df0dabd67f8..ce209666024d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSourceResolution} import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 +import org.apache.spark.sql.internal.SQLConf.DEFAULT_V2_CATALOG import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -77,7 +78,7 @@ class PlanResolutionSuite extends AnalysisTest { def parseAndResolve(query: String, withDefault: Boolean = false): LogicalPlan = { val newConf = conf.copy() - newConf.setConfString("spark.sql.default.catalog", "testcat") + newConf.setConfString(DEFAULT_V2_CATALOG.key, "testcat") DataSourceResolution(newConf, if (withDefault) lookupWithDefault else lookupWithoutDefault) .apply(parsePlan(query)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index af524c7ca025..eaff5a2352a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -201,7 +201,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } test("partitioned table - case insensitive") { - withSQLConf("spark.sql.caseSensitive" -> "false") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { val table = createTable( files = Seq( @@ -437,7 +437,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } test("[SPARK-16818] exchange reuse respects differences in partition pruning") { - spark.conf.set("spark.sql.exchange.reuse", true) + spark.conf.set(SQLConf.EXCHANGE_REUSE_ENABLED.key, true) withTempPath { path => val tempDir = path.getCanonicalPath spark.range(10) 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 2e7d682a3bbc..cf17025980ee 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 @@ -1399,8 +1400,8 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te // that whole test file is mapped to only one partition. This will guarantee // reliable sampling of the input file. withSQLConf( - "spark.sql.files.maxPartitionBytes" -> (128 * 1024 * 1024).toString, - "spark.sql.files.openCostInBytes" -> (4 * 1024 * 1024).toString + SQLConf.FILES_MAX_PARTITION_BYTES.key -> (128 * 1024 * 1024).toString, + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> (4 * 1024 * 1024).toString )(withTempPath { path => val ds = sampledTestData.coalesce(1) ds.write.text(path.getAbsolutePath) @@ -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/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 6316e89537ca..34b44be57689 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2041,8 +2041,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // that whole test file is mapped to only one partition. This will guarantee // reliable sampling of the input file. withSQLConf( - "spark.sql.files.maxPartitionBytes" -> (128 * 1024 * 1024).toString, - "spark.sql.files.openCostInBytes" -> (4 * 1024 * 1024).toString + SQLConf.FILES_MAX_PARTITION_BYTES.key -> (128 * 1024 * 1024).toString, + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> (4 * 1024 * 1024).toString )(withTempPath { path => val ds = sampledTestData.coalesce(1) ds.write.text(path.getAbsolutePath) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index c5d12d618e05..577d1bc8d6a4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1208,6 +1208,14 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } + // SPARK-28371: make sure filter is null-safe. + withParquetDataFrame(Seq(Tuple1[String](null))) { implicit df => + checkFilterPredicate( + '_1.startsWith("blah").asInstanceOf[Predicate], + classOf[UserDefinedByInstance[_, _]], + Seq.empty[Row]) + } + import testImplicits._ // Test canDrop() has taken effect testStringStartsWith(spark.range(1024).map(_.toString).toDF(), "value like 'a%'") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 6b05b9c0f720..6f2218ba82dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -475,7 +475,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { val extraOptions = Map( SQLConf.OUTPUT_COMMITTER_CLASS.key -> classOf[ParquetOutputCommitter].getCanonicalName, - "spark.sql.parquet.output.committer.class" -> + SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key -> classOf[JobCommitFailureParquetOutputCommitter].getCanonicalName ) withTempPath { dir => @@ -505,7 +505,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { // Using a output committer that always fail when committing a task, so that both // `commitTask()` and `abortTask()` are invoked. val extraOptions = Map[String, String]( - "spark.sql.parquet.output.committer.class" -> + SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key -> classOf[TaskCommitFailureParquetOutputCommitter].getCanonicalName ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 6f3ed3d85e93..04ace0a236e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -953,7 +953,7 @@ abstract class ParquetPartitionDiscoverySuite withSQLConf( ParquetOutputFormat.JOB_SUMMARY_LEVEL -> "ALL", - "spark.sql.sources.commitProtocolClass" -> + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { spark.range(3).write.parquet(s"$path/p0=0/p1=0") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index b260f5d3f016..dc4a2998a908 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -277,9 +277,9 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared } test("ShuffledHashJoin metrics") { - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "40", - "spark.sql.shuffle.partitions" -> "2", - "spark.sql.join.preferSortMergeJoin" -> "false") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "40", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") val df2 = (1 to 10).map(i => (i, i.toString)).toSeq.toDF("key", "value") // Assume the execution plan is diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala index f12eeaa58064..8f26c04307ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SQLAppStatusStore} +import org.apache.spark.sql.internal.SQLConf.WHOLESTAGE_CODEGEN_ENABLED import org.apache.spark.sql.test.SQLTestUtils @@ -154,7 +155,7 @@ trait SQLMetricsTestUtils extends SQLTestUtils { expectedNodeIds: Set[Long], enableWholeStage: Boolean = false): Option[Map[Long, (String, Map[String, Any])]] = { val previousExecutionIds = currentExecutionIds() - withSQLConf("spark.sql.codegen.wholeStage" -> enableWholeStage.toString) { + withSQLConf(WHOLESTAGE_CODEGEN_ENABLED.key -> enableWholeStage.toString) { df.collect() } sparkContext.listenerBus.waitUntilEmpty(10000) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala index 723764c77727..c0fd3fe3ef7b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala @@ -24,7 +24,6 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.streaming.ProcessingTime import org.apache.spark.sql.streaming.util.StreamManualClock class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { @@ -35,7 +34,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { val timeout = 10.seconds test("nextBatchTime") { - val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(100)) + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTimeTrigger(100)) assert(processingTimeExecutor.nextBatchTime(0) === 100) assert(processingTimeExecutor.nextBatchTime(1) === 100) assert(processingTimeExecutor.nextBatchTime(99) === 100) @@ -49,7 +48,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { val clock = new StreamManualClock() @volatile var continueExecuting = true @volatile var clockIncrementInTrigger = 0L - val executor = ProcessingTimeExecutor(ProcessingTime("1000 milliseconds"), clock) + val executor = ProcessingTimeExecutor(ProcessingTimeTrigger("1000 milliseconds"), clock) val executorThread = new Thread() { override def run(): Unit = { executor.execute(() => { @@ -97,7 +96,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { test("calling nextBatchTime with the result of a previous call should return the next interval") { val intervalMS = 100 - val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMS)) + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTimeTrigger(intervalMS)) val ITERATION = 10 var nextBatchTime: Long = 0 @@ -111,7 +110,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { private def testBatchTermination(intervalMs: Long): Unit = { var batchCounts = 0 - val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMs)) + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTimeTrigger(intervalMs)) processingTimeExecutor.execute(() => { batchCounts += 1 // If the batch termination works correctly, batchCounts should be 3 after `execute` @@ -130,7 +129,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { @volatile var batchFallingBehindCalled = false val t = new Thread() { override def run(): Unit = { - val processingTimeExecutor = new ProcessingTimeExecutor(ProcessingTime(100), clock) { + val processingTimeExecutor = new ProcessingTimeExecutor(ProcessingTimeTrigger(100), clock) { override def notifyBatchFallingBehind(realElapsedTimeMs: Long): Unit = { batchFallingBehindCalled = true } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala index 2a1e7d615e5e..7bca225dfdd8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} import org.apache.spark.sql.functions.count +import org.apache.spark.sql.internal.SQLConf.SHUFFLE_PARTITIONS import org.apache.spark.util.Utils class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { @@ -124,7 +125,7 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { import spark.implicits._ coordRef = spark.streams.stateStoreCoordinator implicit val sqlContext = spark.sqlContext - spark.conf.set("spark.sql.shuffle.partitions", "1") + spark.conf.set(SHUFFLE_PARTITIONS.key, "1") // Start a query and run a batch to load state stores val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index af4369de800e..a84d107f2cbc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -569,7 +569,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] val spark = SparkSession.builder().master("local[2]").getOrCreate() SparkSession.setActiveSession(spark) implicit val sqlContext = spark.sqlContext - spark.conf.set("spark.sql.shuffle.partitions", "1") + spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "1") import spark.implicits._ val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index e3e5ddff9637..8edbb8770671 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -647,7 +647,7 @@ class SQLAppStatusListenerMemoryLeakSuite extends SparkFunSuite { .setMaster("local") .setAppName("test") .set(config.TASK_MAX_FAILURES, 1) // Don't retry the tasks to run this test quickly - .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly + .set(UI_RETAINED_EXECUTIONS.key, "50") // Set it to 50 to run this test quickly .set(ASYNC_TRACKING_ENABLED, false) withSpark(new SparkContext(conf)) { sc => quietly { 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 5f27e75addcf..89eaac8e5927 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") { @@ -895,6 +892,17 @@ class JDBCSuite extends QueryTest "BIT") assert(msSqlServerDialect.getJDBCType(BinaryType).map(_.databaseTypeDefinition).get == "VARBINARY(MAX)") + assert(msSqlServerDialect.getJDBCType(ShortType).map(_.databaseTypeDefinition).get == + "SMALLINT") + } + + test("SPARK-28152 MsSqlServerDialect catalyst type mapping") { + val msSqlServerDialect = JdbcDialects.get("jdbc:sqlserver") + val metadata = new MetadataBuilder().putLong("scale", 1) + assert(msSqlServerDialect.getCatalystType(java.sql.Types.SMALLINT, "SMALLINT", 1, + metadata).get == ShortType) + assert(msSqlServerDialect.getCatalystType(java.sql.Types.REAL, "REAL", 1, + metadata).get == FloatType) } test("table exists query by jdbc dialect") { @@ -1322,7 +1330,7 @@ class JDBCSuite extends QueryTest testJdbcParitionColumn("THEID", "THEID") testJdbcParitionColumn("\"THEID\"", "THEID") - withSQLConf("spark.sql.caseSensitive" -> "false") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { testJdbcParitionColumn("ThEiD", "THEID") } testJdbcParitionColumn("THE ID", "THE ID") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index fc61050dc745..75f68dea96bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -63,7 +63,7 @@ abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { val maxNrBuckets: Int = 200000 val catalog = spark.sessionState.catalog - withSQLConf("spark.sql.sources.bucketing.maxBuckets" -> maxNrBuckets.toString) { + withSQLConf(SQLConf.BUCKETING_MAX_BUCKETS.key -> maxNrBuckets.toString) { // within the new limit Seq(100001, maxNrBuckets).foreach(numBuckets => { withTable("t") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index d46029e84433..5f9856656ac3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.internal.SQLConf.BUCKETING_MAX_BUCKETS import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils @@ -252,7 +253,7 @@ class CreateTableAsSelectSuite val maxNrBuckets: Int = 200000 val catalog = spark.sessionState.catalog - withSQLConf("spark.sql.sources.bucketing.maxBuckets" -> maxNrBuckets.toString) { + withSQLConf(BUCKETING_MAX_BUCKETS.key -> maxNrBuckets.toString) { // Within the new limit Seq(100001, maxNrBuckets).foreach(numBuckets => { 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 1916e1174efe..b11d46903f04 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 @@ -21,13 +21,27 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter +<<<<<<< HEAD import org.apache.spark.sql.{QueryTest, Row} +||||||| merged common ancestors +import org.apache.spark.sql.QueryTest +======= +import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, QueryTest} +>>>>>>> origin/master 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 import org.apache.spark.sql.test.SharedSQLContext +<<<<<<< HEAD import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} +||||||| merged common ancestors +import org.apache.spark.sql.types.{LongType, StringType, StructType} +======= +import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampType} +>>>>>>> origin/master class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { @@ -37,8 +51,10 @@ 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("spark.sql.catalog.session", classOf[TestInMemoryTableCatalog].getName) + spark.conf.set(V2_SESSION_CATALOG.key, classOf[TestInMemoryTableCatalog].getName) val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") @@ -48,6 +64,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() } @@ -201,20 +218,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") { @@ -324,7 +489,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn test("CreateTableAsSelect: v2 session catalog can load v1 source table") { val sparkSession = spark.newSession() - sparkSession.conf.set("spark.sql.catalog.session", classOf[V2SessionCatalog].getName) + sparkSession.conf.set(V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName) val df = sparkSession.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") @@ -410,4 +575,834 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn df_joined) } } + + test("AlterTable: table does not exist") { + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE testcat.ns1.table_name DROP COLUMN id") + } + + assert(exc.getMessage.contains("testcat.ns1.table_name")) + assert(exc.getMessage.contains("Table or view not found")) + } + + test("AlterTable: change rejected by implementation") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[SparkException] { + sql(s"ALTER TABLE $t DROP COLUMN id") + } + + assert(exc.getMessage.contains("Unsupported table change")) + assert(exc.getMessage.contains("Cannot drop all fields")) // from the implementation + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", IntegerType)) + } + } + + test("AlterTable: add top-level column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN data string") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", IntegerType).add("data", StringType)) + } + } + + test("AlterTable: add column with comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN data string COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq( + StructField("id", IntegerType), + StructField("data", StringType).withComment("doc")))) + } + } + + test("AlterTable: add multiple columns") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ADD COLUMNS data string COMMENT 'doc', ts timestamp") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq( + StructField("id", IntegerType), + StructField("data", StringType).withComment("doc"), + StructField("ts", TimestampType)))) + } + } + + test("AlterTable: add nested column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN point.z double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType))))) + } + } + + test("AlterTable: add nested column to map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN points.key.z double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType))), LongType))) + } + } + + test("AlterTable: add nested column to map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map>) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN points.value.z double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType)))))) + } + } + + test("AlterTable: add nested column to array element") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN points.element.z double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType)))))) + } + } + + test("AlterTable: add complex column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN points array>") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType)))))) + } + } + + test("AlterTable: add nested column with comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN points.element.z double COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType).withComment("doc")))))) + } + } + + test("AlterTable: add nested column parent must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ADD COLUMN point.z double") + } + + assert(exc.getMessage.contains("point")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: update column type int -> long") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN id TYPE bigint") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", LongType)) + } + } + + test("AlterTable: update nested type float -> double") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN point.x TYPE double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))))) + } + } + + test("AlterTable: update column with struct type fails") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN point TYPE struct") + } + + assert(exc.getMessage.contains("point")) + assert(exc.getMessage.contains("update a struct by adding, deleting, or updating its fields")) + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))))) + } + } + + test("AlterTable: update column with array type fails") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN points TYPE array") + } + + assert(exc.getMessage.contains("update the element by updating points.element")) + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(IntegerType))) + } + } + + test("AlterTable: update column array element type") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.element TYPE long") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(LongType))) + } + } + + test("AlterTable: update column with map type fails") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, m map) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN m TYPE map") + } + + assert(exc.getMessage.contains("update a map by updating m.key or m.value")) + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("m", MapType(StringType, IntegerType))) + } + } + + test("AlterTable: update column map value type") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, m map) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN m.value TYPE long") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("m", MapType(StringType, LongType))) + } + } + + test("AlterTable: update nested type in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.key.x TYPE double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))), LongType))) + } + } + + test("AlterTable: update nested type in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.value.x TYPE double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType)))))) + } + } + + test("AlterTable: update nested type in array") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.element.x TYPE double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType)))))) + } + } + + test("AlterTable: update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN data TYPE string") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN point.x TYPE double") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: update column type must be compatible") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN id TYPE boolean") + } + + assert(exc.getMessage.contains("id")) + assert(exc.getMessage.contains("int cannot be cast to boolean")) + } + } + + test("AlterTable: update column comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN id COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq(StructField("id", IntegerType).withComment("doc")))) + } + } + + test("AlterTable: update column type and comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN id TYPE bigint COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq(StructField("id", LongType).withComment("doc")))) + } + } + + test("AlterTable: update nested column comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN point.y COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc"))))) + } + } + + test("AlterTable: update nested column comment in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.key.y COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc"))), LongType))) + } + } + + test("AlterTable: update nested column comment in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.value.y COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc")))))) + } + } + + test("AlterTable: update nested column comment in array") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.element.y COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc")))))) + } + } + + test("AlterTable: comment update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN data COMMENT 'doc'") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested comment update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN point.x COMMENT 'doc'") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: rename column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t RENAME COLUMN id TO user_id") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("user_id", IntegerType)) + } + } + + test("AlterTable: rename nested column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + sql(s"ALTER TABLE $t RENAME COLUMN point.y TO t") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType))))) + } + } + + test("AlterTable: rename nested column in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point map, bigint>) USING foo") + sql(s"ALTER TABLE $t RENAME COLUMN point.key.y TO t") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType))), LongType))) + } + } + + test("AlterTable: rename nested column in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map>) USING foo") + sql(s"ALTER TABLE $t RENAME COLUMN points.value.y TO t") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType)))))) + } + } + + test("AlterTable: rename nested column in array element") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t RENAME COLUMN points.element.y TO t") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType)))))) + } + } + + test("AlterTable: rename column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t RENAME COLUMN data TO some_string") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested rename column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t RENAME COLUMN point.x TO z") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: drop column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, data string) USING foo") + sql(s"ALTER TABLE $t DROP COLUMN data") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", IntegerType)) + } + } + + test("AlterTable: drop nested column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + sql(s"ALTER TABLE $t DROP COLUMN point.t") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))))) + } + } + + test("AlterTable: drop nested column in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point map, bigint>) USING foo") + sql(s"ALTER TABLE $t DROP COLUMN point.key.y") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", MapType(StructType(Seq( + StructField("x", DoubleType))), LongType))) + } + } + + test("AlterTable: drop nested column in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map>) USING foo") + sql(s"ALTER TABLE $t DROP COLUMN points.value.y") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType)))))) + } + } + + test("AlterTable: drop nested column in array element") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t DROP COLUMN points.element.y") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType)))))) + } + } + + test("AlterTable: drop column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t DROP COLUMN data") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested drop column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t DROP COLUMN point.x") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: set location") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t SET LOCATION 's3://bucket/path'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.properties == Map("provider" -> "foo", "location" -> "s3://bucket/path").asJava) + } + } + + test("AlterTable: set table property") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t SET TBLPROPERTIES ('test'='34')") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.properties == Map("provider" -> "foo", "test" -> "34").asJava) + } + } + + test("AlterTable: remove table property") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo TBLPROPERTIES('test' = '34')") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.properties == Map("provider" -> "foo", "test" -> "34").asJava) + + sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('test')") + + val updated = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(updated.name == "testcat.ns1.table_name") + assert(updated.properties == Map("provider" -> "foo").asJava) + } + } } 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 35de96f7b142..15c46531edb0 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,12 +66,29 @@ class TestInMemoryTableCatalog extends TableCatalog { schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): Table = { - if (tables.containsKey(ident)) { throw new TableAlreadyExistsException(ident) } +<<<<<<< HEAD val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions) +||||||| merged common ancestors + + if (partitions.nonEmpty) { + throw new UnsupportedOperationException( + s"Catalog $name: Partitioned tables are not supported") + } + + val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties) +======= + TestInMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) + if (partitions.nonEmpty) { + throw new UnsupportedOperationException( + s"Catalog $name: Partitioned tables are not supported") + } + + val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties) +>>>>>>> origin/master tables.put(ident, table) @@ -83,8 +100,20 @@ class TestInMemoryTableCatalog extends TableCatalog { case Some(table) => val properties = CatalogV2Util.applyPropertiesChanges(table.properties, changes) val schema = CatalogV2Util.applySchemaChanges(table.schema, changes) +<<<<<<< HEAD val newTable = new InMemoryTable( table.name, schema, properties, table.partitioning, table.data) +||||||| merged common ancestors + val newTable = new InMemoryTable(table.name, schema, properties, table.data) +======= + + // fail if the last column in the schema was dropped + if (schema.fields.isEmpty) { + throw new IllegalArgumentException(s"Cannot drop all fields") + } + + val newTable = new InMemoryTable(table.name, schema, properties, table.data) +>>>>>>> origin/master tables.put(ident, newTable) @@ -94,7 +123,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() @@ -104,7 +135,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], @@ -147,6 +178,7 @@ private class InMemoryTable( } override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + TestInMemoryTableCatalog.maybeSimulateFailedTableWrite(options) new WriteBuilder with SupportsTruncate { private var shouldTruncate: Boolean = false @@ -188,7 +220,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/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 4bf49ff4d5c6..92ec2a0c172e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -305,7 +305,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("update mode") { val inputData = MemoryStream[Int] - spark.conf.set("spark.sql.shuffle.partitions", "10") + spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "10") val windowedAggregation = inputData.toDF() .withColumn("eventTime", $"value".cast("timestamp")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 2b8d77386925..72f893845172 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -1310,7 +1310,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { val start = startId.map(new FileStreamSourceOffset(_)) val end = FileStreamSourceOffset(endId) - withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") { + withSQLConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false") { assert(fileSource.getBatch(start, end).as[String].collect().toSeq === expected) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index f2f5fad59eb2..1ed2599444c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -871,7 +871,7 @@ class StreamSuite extends StreamTest { testQuietly("specify custom state store provider") { val providerClassName = classOf[TestStateStoreProvider].getCanonicalName - withSQLConf("spark.sql.streaming.stateStore.providerClass" -> providerClassName) { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> providerClassName) { val input = MemoryStream[Int] val df = input.toDS().groupBy().count() val query = df.writeStream.outputMode("complete").format("memory").queryName("name").start() @@ -888,9 +888,9 @@ class StreamSuite extends StreamTest { testQuietly("custom state store provider read from offset log") { val input = MemoryStream[Int] val df = input.toDS().groupBy().count() - val providerConf1 = "spark.sql.streaming.stateStore.providerClass" -> + val providerConf1 = SQLConf.STATE_STORE_PROVIDER_CLASS.key -> "org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider" - val providerConf2 = "spark.sql.streaming.stateStore.providerClass" -> + val providerConf2 = SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[TestStateStoreProvider].getCanonicalName def runQuery(queryName: String, checkpointLoc: String): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala index 88f510c726fa..da2f221aaf10 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS import org.apache.spark.sql.streaming.StreamingQueryListener._ @@ -29,7 +30,7 @@ class StreamingQueryListenersConfSuite extends StreamTest with BeforeAndAfter { import testImplicits._ override protected def sparkConf: SparkConf = - super.sparkConf.set("spark.sql.streaming.streamingQueryListeners", + super.sparkConf.set(STREAMING_QUERY_LISTENERS.key, "org.apache.spark.sql.streaming.TestListener") test("test if the configured query lister is loaded") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index a5cb25c49b86..e6b56e5f46f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -413,9 +413,9 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi sources.nonEmpty } // Disabled by default - assert(spark.conf.get("spark.sql.streaming.metricsEnabled").toBoolean === false) + assert(spark.conf.get(SQLConf.STREAMING_METRICS_ENABLED.key).toBoolean === false) - withSQLConf("spark.sql.streaming.metricsEnabled" -> "false") { + withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "false") { testStream(inputData.toDF)( AssertOnQuery { q => !isMetricsRegistered(q) }, StopStream, @@ -424,7 +424,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } // Registered when enabled - withSQLConf("spark.sql.streaming.metricsEnabled" -> "true") { + withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "true") { testStream(inputData.toDF)( AssertOnQuery { q => isMetricsRegistered(q) }, StopStream, @@ -434,7 +434,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } test("SPARK-22975: MetricsReporter defaults when there was no progress reported") { - withSQLConf("spark.sql.streaming.metricsEnabled" -> "true") { + withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "true") { BlockingSource.latch = new CountDownLatch(1) withTempDir { tempDir => val sq = spark.readStream diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala index c5b95fa9b64a..3ec4750c59fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.streaming.continuous import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED import org.apache.spark.sql.streaming.OutputMode class ContinuousAggregationSuite extends ContinuousSuiteBase { @@ -36,7 +37,7 @@ class ContinuousAggregationSuite extends ContinuousSuiteBase { } test("basic") { - withSQLConf(("spark.sql.streaming.unsupportedOperationCheck", "false")) { + withSQLConf((UNSUPPORTED_OPERATION_CHECK_ENABLED.key, "false")) { val input = ContinuousMemoryStream.singlePartition[Int] testStream(input.toDF().agg(max('value)), OutputMode.Complete)( @@ -112,7 +113,7 @@ class ContinuousAggregationSuite extends ContinuousSuiteBase { } test("repeated restart") { - withSQLConf(("spark.sql.streaming.unsupportedOperationCheck", "false")) { + withSQLConf((UNSUPPORTED_OPERATION_CHECK_ENABLED.key, "false")) { val input = ContinuousMemoryStream.singlePartition[Int] testStream(input.toDF().agg(max('value)), OutputMode.Complete)( 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 9840c7f06678..c6921010a002 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 @@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf.CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE +import org.apache.spark.sql.internal.SQLConf.{CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE, MIN_BATCHES_TO_RETAIN} import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.test.TestSparkSession @@ -37,18 +37,43 @@ class ContinuousSuiteBase extends StreamTest { "continuous-stream-test-sql-context", sparkConf.set("spark.sql.testkey", "true"))) - protected def waitForRateSourceTriggers(query: StreamExecution, numTriggers: Int): Unit = { - query match { - case s: ContinuousExecution => - assert(numTriggers >= 2, "must wait for at least 2 triggers to ensure query is initialized") - val reader = s.lastExecution.executedPlan.collectFirst { - case ContinuousScanExec(_, _, r: RateStreamContinuousStream, _) => r - }.get - - val deltaMs = numTriggers * 1000 + 300 - while (System.currentTimeMillis < reader.creationTime + deltaMs) { - Thread.sleep(reader.creationTime + deltaMs - System.currentTimeMillis) + protected def waitForRateSourceTriggers(query: ContinuousExecution, numTriggers: Int): Unit = { + query.awaitEpoch(0) + + // This is called after waiting first epoch to be committed, so we can just treat + // it as partition readers for rate source are already initialized. + val firstCommittedTime = System.nanoTime() + val deltaNs = (numTriggers * 1000 + 300) * 1000000L + var toWaitNs = firstCommittedTime + deltaNs - System.nanoTime() + while (toWaitNs > 0) { + Thread.sleep(toWaitNs / 1000000) + toWaitNs = firstCommittedTime + deltaNs - System.nanoTime() + } + } + + protected def waitForRateSourceCommittedValue( + query: ContinuousExecution, + desiredValue: Long, + maxWaitTimeMs: Long): Unit = { + def readHighestCommittedValue(c: ContinuousExecution): Option[Long] = { + c.committedOffsets.lastOption.map { case (_, offset) => + offset match { + case o: RateStreamOffset => + o.partitionToValueAndRunTimeMs.map { + case (_, ValueRunTimeMsPair(value, _)) => value + }.max } + } + } + + val maxWait = System.currentTimeMillis() + maxWaitTimeMs + while (System.currentTimeMillis() < maxWait && + readHighestCommittedValue(query).getOrElse(Long.MinValue) < desiredValue) { + Thread.sleep(100) + } + if (System.currentTimeMillis() > maxWait) { + logWarning(s"Couldn't reach desired value in $maxWaitTimeMs milliseconds!" + + s"Current highest committed value is ${readHighestCommittedValue(query)}") } } @@ -216,14 +241,16 @@ class ContinuousSuite extends ContinuousSuiteBase { .queryName("noharness") .trigger(Trigger.Continuous(100)) .start() + + val expected = Set(0, 1, 2, 3) val continuousExecution = query.asInstanceOf[StreamingQueryWrapper].streamingQuery.asInstanceOf[ContinuousExecution] - continuousExecution.awaitEpoch(0) - waitForRateSourceTriggers(continuousExecution, 2) + waitForRateSourceCommittedValue(continuousExecution, expected.max, 20 * 1000) query.stop() val results = spark.read.table("noharness").collect() - assert(Set(0, 1, 2, 3).map(Row(_)).subsetOf(results.toSet)) + assert(expected.map(Row(_)).subsetOf(results.toSet), + s"Result set ${results.toSet} are not a superset of $expected!") } } @@ -241,7 +268,9 @@ class ContinuousStressSuite extends ContinuousSuiteBase { testStream(df)( StartStream(longContinuousTrigger), AwaitEpoch(0), - Execute(waitForRateSourceTriggers(_, 10)), + Execute { exec => + waitForRateSourceTriggers(exec.asInstanceOf[ContinuousExecution], 5) + }, IncrementEpoch(), StopStream, CheckAnswerRowsContains(scala.Range(0, 2500).map(Row(_))) @@ -259,7 +288,9 @@ class ContinuousStressSuite extends ContinuousSuiteBase { testStream(df)( StartStream(Trigger.Continuous(2012)), AwaitEpoch(0), - Execute(waitForRateSourceTriggers(_, 10)), + Execute { exec => + waitForRateSourceTriggers(exec.asInstanceOf[ContinuousExecution], 5) + }, IncrementEpoch(), StopStream, CheckAnswerRowsContains(scala.Range(0, 2500).map(Row(_)))) @@ -307,7 +338,7 @@ class ContinuousMetaSuite extends ContinuousSuiteBase { "local[10]", "continuous-stream-test-sql-context", sparkConf.set("spark.sql.testkey", "true") - .set("spark.sql.streaming.minBatchesToRetain", "2"))) + .set(MIN_BATCHES_TO_RETAIN.key, "2"))) test("SPARK-24351: check offsetLog/commitLog retained in the checkpoint directory") { withTempDir { checkpointDir => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index 7b2c1a56e8ba..4db605ee1b23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -24,8 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.execution.streaming.{RateStreamOffset, Sink, StreamingQueryWrapper} -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger +import org.apache.spark.sql.execution.streaming.{ContinuousTrigger, RateStreamOffset, Sink, StreamingQueryWrapper} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} import org.apache.spark.sql.sources.v2._ @@ -242,7 +241,7 @@ class StreamingDataSourceV2Suite extends StreamTest { override def beforeAll(): Unit = { super.beforeAll() val fakeCheckpoint = Utils.createTempDir() - spark.conf.set("spark.sql.streaming.checkpointLocation", fakeCheckpoint.getCanonicalPath) + spark.conf.set(SQLConf.CHECKPOINT_LOCATION.key, fakeCheckpoint.getCanonicalPath) } override def afterEach(): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 8fb1400a9b5a..c630f1497a17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -203,7 +203,7 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { .stop() assert(LastOptions.partitionColumns == Seq("a")) - withSQLConf("spark.sql.caseSensitive" -> "false") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { df.writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index e9ab62800f84..126e23e6e592 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -409,7 +409,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be test("write path implements onTaskCommit API correctly") { withSQLConf( - "spark.sql.sources.commitProtocolClass" -> + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[MessageCapturingCommitProtocol].getCanonicalName) { withTempDir { dir => val path = dir.getCanonicalPath diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index d1de9f037992..b4d1d0d58aad 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -137,7 +137,7 @@ object HiveThriftServer2 extends Logging { } private[thriftserver] object ExecutionState extends Enumeration { - val STARTED, COMPILED, FAILED, FINISHED = Value + val STARTED, COMPILED, FAILED, FINISHED, CLOSED = Value type ExecutionState = Value } @@ -147,16 +147,17 @@ object HiveThriftServer2 extends Logging { val startTimestamp: Long, val userName: String) { var finishTimestamp: Long = 0L + var closeTimestamp: Long = 0L var executePlan: String = "" var detail: String = "" var state: ExecutionState.Value = ExecutionState.STARTED val jobId: ArrayBuffer[String] = ArrayBuffer[String]() var groupId: String = "" - def totalTime: Long = { - if (finishTimestamp == 0L) { + def totalTime(endTime: Long): Long = { + if (endTime == 0L) { System.currentTimeMillis - startTimestamp } else { - finishTimestamp - startTimestamp + endTime - startTimestamp } } } @@ -254,6 +255,11 @@ object HiveThriftServer2 extends Logging { trimExecutionIfNecessary() } + def onOperationClosed(id: String): Unit = synchronized { + executionList(id).closeTimestamp = System.currentTimeMillis + executionList(id).state = ExecutionState.CLOSED + } + private def trimExecutionIfNecessary() = { if (executionList.size > retainedStatements) { val toRemove = math.max(retainedStatements / 10, 1) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 820f76db6db3..2f011c25fe2c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -70,11 +70,12 @@ private[hive] class SparkExecuteStatementOperation( } } - def close(): Unit = { + override def close(): Unit = { // RDDs will be cleaned automatically upon garbage collection. logDebug(s"CLOSING $statementId") cleanup(OperationState.CLOSED) sqlContext.sparkContext.clearJobGroup() + HiveThriftServer2.listener.onOperationClosed(statementId) } def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 99ba968e1ae8..89faff2f6f91 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -58,8 +58,15 @@ private[hive] class SparkGetColumnsOperation( val catalog: SessionCatalog = sqlContext.sessionState.catalog + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + override def runInternal(): Unit = { - val statementId = UUID.randomUUID().toString + statementId = UUID.randomUUID().toString // Do not change cmdStr. It's used for Hive auditing and authorization. val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName, tablePattern : $tableName" val logMsg = s"Listing columns '$cmdStr, columnName : $columnName'" diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 3ecbbd036c87..87ef154bcc8a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -45,8 +45,15 @@ private[hive] class SparkGetSchemasOperation( schemaName: String) extends GetSchemasOperation(parentSession, catalogName, schemaName) with Logging { + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + override def runInternal(): Unit = { - val statementId = UUID.randomUUID().toString + statementId = UUID.randomUUID().toString // Do not change cmdStr. It's used for Hive auditing and authorization. val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName" val logMsg = s"Listing databases '$cmdStr'" 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 878683692fb6..952de42083c4 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 @@ -55,8 +55,15 @@ private[hive] class SparkGetTablesOperation( extends GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes) with Logging{ + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + override def runInternal(): Unit = { - val statementId = UUID.randomUUID().toString + statementId = UUID.randomUUID().toString // Do not change cmdStr. It's used for Hive auditing and authorization. val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName" val tableTypesStr = if (tableTypes == null) "null" else tableTypes.asScala.mkString(",") diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 27d2c997ca3e..1747b5bafc93 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -70,8 +70,8 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" private def generateSQLStatsTable(request: HttpServletRequest): Seq[Node] = { val numStatement = listener.getExecutionList.size val table = if (numStatement > 0) { - val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Duration", - "Statement", "State", "Detail") + val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Close Time", + "Execution Time", "Duration", "Statement", "State", "Detail") val dataRows = listener.getExecutionList.sortBy(_.startTimestamp).reverse def generateDataRow(info: ExecutionInfo): Seq[Node] = { @@ -90,7 +90,9 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" {info.groupId} {formatDate(info.startTimestamp)} {if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)} - {formatDurationOption(Some(info.totalTime))} + {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} + {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} + {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} {info.statement} {info.state} {errorMessageCell(detail)} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index fdc9bee5ed05..a45c6e363cbf 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -79,8 +79,8 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) .filter(_.sessionId == sessionID) val numStatement = executionList.size val table = if (numStatement > 0) { - val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Duration", - "Statement", "State", "Detail") + val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Close Time", + "Execution Time", "Duration", "Statement", "State", "Detail") val dataRows = executionList.sortBy(_.startTimestamp).reverse def generateDataRow(info: ExecutionInfo): Seq[Node] = { @@ -99,7 +99,9 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) {info.groupId} {formatDate(info.startTimestamp)} {formatDate(info.finishTimestamp)} - {formatDurationOption(Some(info.totalTime))} + {formatDate(info.closeTimestamp)} + {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} + {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} {info.statement} {info.state} {errorMessageCell(detail)} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index b06856b05479..dd18add53fde 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -44,6 +44,7 @@ import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.HiveTestUtils +import org.apache.spark.sql.internal.StaticSQLConf.HIVE_THRIFT_SERVER_SINGLESESSION import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.{ThreadUtils, Utils} @@ -536,9 +537,9 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } if (HiveUtils.isHive23) { - assert(conf.get("spark.sql.hive.version") === Some("2.3.5")) + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("2.3.5")) } else { - assert(conf.get("spark.sql.hive.version") === Some("1.2.1")) + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("1.2.1")) } } } @@ -553,9 +554,9 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } if (HiveUtils.isHive23) { - assert(conf.get("spark.sql.hive.version") === Some("2.3.5")) + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("2.3.5")) } else { - assert(conf.get("spark.sql.hive.version") === Some("1.2.1")) + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("1.2.1")) } } } @@ -659,7 +660,7 @@ class SingleSessionSuite extends HiveThriftJdbcTest { override def mode: ServerMode.Value = ServerMode.binary override protected def extraConf: Seq[String] = - "--conf spark.sql.hive.thriftServer.singleSession=true" :: Nil + s"--conf ${HIVE_THRIFT_SERVER_SINGLESESSION.key}=true" :: Nil test("share the temporary functions across JDBC connections") { withMultipleConnectionJdbcStatement()( diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index ad7a9a238f8a..8fce9d938343 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -344,6 +344,7 @@ SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) String ipAddress = getIpAddress(); TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, req.getClient_protocol()); + res.setServerProtocolVersion(protocol); SessionHandle sessionHandle; if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && (userName != null)) { @@ -354,7 +355,6 @@ SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) sessionHandle = cliService.openSession(protocol, userName, req.getPassword(), ipAddress, req.getConfiguration()); } - res.setServerProtocolVersion(protocol); return sessionHandle; } diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 9552d9bd68cd..d41c3b493bb4 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -345,6 +345,7 @@ SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) String ipAddress = getIpAddress(); TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, req.getClient_protocol()); + res.setServerProtocolVersion(protocol); SessionHandle sessionHandle; if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && (userName != null)) { @@ -355,7 +356,6 @@ SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) sessionHandle = cliService.openSession(protocol, userName, req.getPassword(), ipAddress, req.getConfiguration()); } - res.setServerProtocolVersion(protocol); return sessionHandle; } 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 0ed464dad91b..ee1734b1f232 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 62d3bad63bf5..4be3cd45454c 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/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index 4351dc703684..9bc0be87be5a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -27,9 +27,12 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.spark.{SecurityManager, SparkConf, TestUtils} +import org.apache.spark.internal.config.MASTER_REST_SERVER_ENABLED +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.util.Utils @@ -184,11 +187,11 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { val args = Seq( "--name", "prepare testing tables", "--master", "local[2]", - "--conf", "spark.ui.enabled=false", - "--conf", "spark.master.rest.enabled=false", - "--conf", "spark.sql.hive.metastore.version=1.2.1", - "--conf", "spark.sql.hive.metastore.jars=maven", - "--conf", s"spark.sql.warehouse.dir=${wareHousePath.getCanonicalPath}", + "--conf", s"${UI_ENABLED.key}=false", + "--conf", s"${MASTER_REST_SERVER_ENABLED.key}=false", + "--conf", s"${HiveUtils.HIVE_METASTORE_VERSION.key}=1.2.1", + "--conf", s"${HiveUtils.HIVE_METASTORE_JARS.key}=maven", + "--conf", s"${WAREHOUSE_PATH.key}=${wareHousePath.getCanonicalPath}", "--conf", s"spark.sql.test.version.index=$index", "--driver-java-options", s"-Dderby.system.home=${wareHousePath.getCanonicalPath}", tempPyFile.getCanonicalPath) @@ -203,11 +206,11 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { "--class", PROCESS_TABLES.getClass.getName.stripSuffix("$"), "--name", "HiveExternalCatalog backward compatibility test", "--master", "local[2]", - "--conf", "spark.ui.enabled=false", - "--conf", "spark.master.rest.enabled=false", - "--conf", "spark.sql.hive.metastore.version=1.2.1", - "--conf", "spark.sql.hive.metastore.jars=maven", - "--conf", s"spark.sql.warehouse.dir=${wareHousePath.getCanonicalPath}", + "--conf", s"${UI_ENABLED.key}=false", + "--conf", s"${MASTER_REST_SERVER_ENABLED.key}=false", + "--conf", s"${HiveUtils.HIVE_METASTORE_VERSION.key}=1.2.1", + "--conf", s"${HiveUtils.HIVE_METASTORE_JARS.key}=maven", + "--conf", s"${WAREHOUSE_PATH.key}=${wareHousePath.getCanonicalPath}", "--driver-java-options", s"-Dderby.system.home=${wareHousePath.getCanonicalPath}", unusedJar.toString) runSparkSubmit(args) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 0ff22150658b..e2ddec342766 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -33,6 +33,8 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.hive.test.{HiveTestUtils, TestHiveContext} +import org.apache.spark.sql.internal.SQLConf.SHUFFLE_PARTITIONS +import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.types.{DecimalType, StructType} import org.apache.spark.tags.ExtendedHiveTest import org.apache.spark.util.{ResetSystemProperties, Utils} @@ -338,10 +340,10 @@ object SetMetastoreURLTest extends Logging { val builder = SparkSession.builder() .config(sparkConf) .config(UI_ENABLED.key, "false") - .config("spark.sql.hive.metastore.version", "0.13.1") + .config(HiveUtils.HIVE_METASTORE_VERSION.key, "0.13.1") // The issue described in SPARK-16901 only appear when // spark.sql.hive.metastore.jars is not set to builtin. - .config("spark.sql.hive.metastore.jars", "maven") + .config(HiveUtils.HIVE_METASTORE_JARS.key, "maven") .enableHiveSupport() val spark = builder.getOrCreate() @@ -392,16 +394,16 @@ object SetWarehouseLocationTest extends Logging { // We are expecting that the value of spark.sql.warehouse.dir will override the // value of hive.metastore.warehouse.dir. val session = new TestHiveContext(new SparkContext(sparkConf - .set("spark.sql.warehouse.dir", warehouseLocation.toString) + .set(WAREHOUSE_PATH.key, warehouseLocation.toString) .set("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString))) .sparkSession (session, warehouseLocation.toString) } - if (sparkSession.conf.get("spark.sql.warehouse.dir") != expectedWarehouseLocation) { + if (sparkSession.conf.get(WAREHOUSE_PATH.key) != expectedWarehouseLocation) { throw new Exception( - "spark.sql.warehouse.dir is not set to the expected warehouse location " + + s"${WAREHOUSE_PATH.key} is not set to the expected warehouse location " + s"$expectedWarehouseLocation.") } @@ -564,7 +566,7 @@ object SparkSubmitClassLoaderTest extends Logging { val conf = new SparkConf() val hiveWarehouseLocation = Utils.createTempDir() conf.set(UI_ENABLED, false) - conf.set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString) + conf.set(WAREHOUSE_PATH.key, hiveWarehouseLocation.toString) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) val df = hiveContext.createDataFrame((1 to 100).map(i => (i, i))).toDF("i", "j") @@ -642,14 +644,14 @@ object SparkSQLConfTest extends Logging { val conf = new SparkConf() { override def getAll: Array[(String, String)] = { def isMetastoreSetting(conf: String): Boolean = { - conf == "spark.sql.hive.metastore.version" || conf == "spark.sql.hive.metastore.jars" + conf == HiveUtils.HIVE_METASTORE_VERSION.key || conf == HiveUtils.HIVE_METASTORE_JARS.key } // If there is any metastore settings, remove them. val filteredSettings = super.getAll.filterNot(e => isMetastoreSetting(e._1)) // Always add these two metastore settings at the beginning. - ("spark.sql.hive.metastore.version" -> "0.12") +: - ("spark.sql.hive.metastore.jars" -> "maven") +: + (HiveUtils.HIVE_METASTORE_VERSION.key -> "0.12") +: + (HiveUtils.HIVE_METASTORE_JARS.key -> "maven") +: filteredSettings } @@ -676,10 +678,10 @@ object SPARK_9757 extends QueryTest { val hiveWarehouseLocation = Utils.createTempDir() val sparkContext = new SparkContext( new SparkConf() - .set("spark.sql.hive.metastore.version", "0.13.1") - .set("spark.sql.hive.metastore.jars", "maven") + .set(HiveUtils.HIVE_METASTORE_VERSION.key, "0.13.1") + .set(HiveUtils.HIVE_METASTORE_JARS.key, "maven") .set(UI_ENABLED, false) - .set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString)) + .set(WAREHOUSE_PATH.key, hiveWarehouseLocation.toString)) val hiveContext = new TestHiveContext(sparkContext) spark = hiveContext.sparkSession @@ -725,7 +727,7 @@ object SPARK_11009 extends QueryTest { val sparkContext = new SparkContext( new SparkConf() .set(UI_ENABLED, false) - .set("spark.sql.shuffle.partitions", "100")) + .set(SHUFFLE_PARTITIONS.key, "100")) val hiveContext = new TestHiveContext(sparkContext) spark = hiveContext.sparkSession @@ -756,7 +758,7 @@ object SPARK_14244 extends QueryTest { val sparkContext = new SparkContext( new SparkConf() .set(UI_ENABLED, false) - .set("spark.sql.shuffle.partitions", "100")) + .set(SHUFFLE_PARTITIONS.key, "100")) val hiveContext = new TestHiveContext(sparkContext) spark = hiveContext.sparkSession diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index ecd428780c67..d06cc1c0a88a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -1028,7 +1028,7 @@ class HashAggregationQueryWithControlledFallbackSuite extends AggregationQuerySu override protected def checkAnswer(actual: => DataFrame, expectedAnswer: Seq[Row]): Unit = { Seq("true", "false").foreach { enableTwoLevelMaps => - withSQLConf("spark.sql.codegen.aggregate.map.twolevel.enabled" -> + withSQLConf(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> enableTwoLevelMaps) { (1 to 3).foreach { fallbackStartsAt => withSQLConf("spark.sql.TungstenAggregate.testFallbackStartsAt" -> 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 3104fb4d8173..0ea941c8e0d8 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.")) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 8cdb8dd84fb2..d68a47053f18 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.execution.command.CacheTableCommand import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf, WithTestConf} -import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.internal.StaticSQLConf.{CATALOG_IMPLEMENTATION, WAREHOUSE_PATH} import org.apache.spark.util.{ShutdownHookManager, Utils} // SPARK-3729: Test key required to check for initialization errors with config. @@ -57,9 +57,9 @@ object TestHive new SparkConf() .set("spark.sql.test", "") .set(SQLConf.CODEGEN_FALLBACK.key, "false") - .set("spark.sql.hive.metastore.barrierPrefixes", + .set(HiveUtils.HIVE_METASTORE_BARRIER_PREFIXES.key, "org.apache.spark.sql.hive.execution.PairSerDe") - .set("spark.sql.warehouse.dir", TestHiveContext.makeWarehouseDir().toURI.getPath) + .set(WAREHOUSE_PATH.key, TestHiveContext.makeWarehouseDir().toURI.getPath) // SPARK-8910 .set(UI_ENABLED, false) .set(config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) @@ -534,7 +534,7 @@ private[hive] class TestHiveSparkSession( } // Clean out the Hive warehouse between each suite - val warehouseDir = new File(new URI(sparkContext.conf.get("spark.sql.warehouse.dir")).getPath) + val warehouseDir = new File(new URI(sparkContext.conf.get(WAREHOUSE_PATH.key)).getPath) Utils.deleteRecursively(warehouseDir) warehouseDir.mkdir() From c396d32d9bea1a8dfb0659849b7f2042281f935e Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 23 Jul 2019 15:32:59 -0700 Subject: [PATCH 10/15] Resolve conflicts, address comments --- .../sql/catalyst/analysis/Analyzer.scala | 10 +++++- .../sql/catalyst/parser/AstBuilder.scala | 8 +---- .../catalyst/plans/DescribeTableSchemas.scala | 8 ++--- .../plans/logical/basicLogicalOperators.scala | 8 +++-- .../logical/sql/DescribeTableStatement.scala | 2 +- .../sql/catalyst/parser/DDLParserSuite.scala | 18 +--------- .../spark/sql/execution/command/tables.scala | 2 +- .../datasources/DataSourceResolution.scala | 21 +---------- .../datasources/v2/DataSourceV2Strategy.scala | 11 ++---- .../datasources/v2/DescribeTableExec.scala | 18 +++++----- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 34 ++++++------------ .../sources/v2/TestInMemoryTableCatalog.scala | 36 ++++--------------- 12 files changed, 48 insertions(+), 128 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e55cdfedd323..0c1308aac7b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects._ 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} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, DescribeTableStatement} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL @@ -834,6 +834,14 @@ class Analyzer( v2Catalog.asTableCatalog, ident, UnresolvedRelation(alter.tableName), Seq(TableChange.setProperty("location", newLoc))) + + case describe @ DescribeTableStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), partitionSpec, isExtended) => + DescribeTable( + v2Catalog.asTableCatalog, + ident, + UnresolvedRelation(describe.tableName), + isExtended) } } 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 3164ab7d5376..58cbea025be4 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,13 +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._ -<<<<<<< HEAD -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} -||||||| merged common ancestors -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} ->>>>>>> origin/master +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, 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._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala index d23888d9189a..260fc58253fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala @@ -21,15 +21,11 @@ import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.types.{MetadataBuilder, StringType, StructField, StructType} private[sql] object DescribeTableSchemas { - val DESCRIBE_TABLE_ATTRIBUTES = Seq( + def describeTableAttributes(): Seq[AttributeReference] = Seq( AttributeReference("col_name", StringType, nullable = false, new MetadataBuilder().putString("comment", "name of the column").build())(), AttributeReference("data_type", StringType, nullable = false, new MetadataBuilder().putString("comment", "data type of the column").build())(), AttributeReference("comment", StringType, nullable = true, - new MetadataBuilder().putString("comment", "comment of the column").build())() - ) - - val DESCRIBE_TABLE_SCHEMA = StructType( - DESCRIBE_TABLE_ATTRIBUTES.map(attr => StructField(attr.name, attr.dataType, attr.nullable))) + new MetadataBuilder().putString("comment", "comment of the column").build())()) } 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 30f3b8336a56..0b741f44acf4 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 @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction} import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.sql.DescribeTableStatement import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.types._ @@ -546,9 +545,12 @@ object OverwritePartitionsDynamic { case class DescribeTable( catalog: TableCatalog, ident: Identifier, + table: NamedRelation, isExtended: Boolean) extends Command { - override lazy val output = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES - override lazy val schema = DescribeTableSchemas.DESCRIBE_TABLE_SCHEMA + + override def children: Seq[LogicalPlan] = Seq(table) + + override val output = DescribeTableSchemas.describeTableAttributes() } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala index 070b1416711e..62743ddadd93 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala @@ -25,5 +25,5 @@ case class DescribeTableStatement( tableName: Seq[String], partitionSpec: TablePartitionSpec, isExtended: Boolean) extends ParsedStatement { - override def output: Seq[Attribute] = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES + override def output: Seq[Attribute] = DescribeTableSchemas.describeTableAttributes() } 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 931123cf5451..925c4d2d3072 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,24 +19,12 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale -<<<<<<< HEAD import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} -||||||| merged common ancestors -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} ->>>>>>> origin/master 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 -<<<<<<< HEAD -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} -||||||| merged common ancestors -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} ->>>>>>> origin/master +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -628,7 +616,6 @@ class DDLParserSuite extends AnalysisTest { Seq(Seq("x"), Seq("y"), Seq("a", "b", "c")))) } } -<<<<<<< HEAD test("describe table column") { comparePlans(parsePlan("DESCRIBE t col"), @@ -670,8 +657,6 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsePlan("describe table formatted t"), DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) } -||||||| merged common ancestors -======= private case class TableSpec( name: Seq[String], @@ -737,5 +722,4 @@ class DDLParserSuite extends AnalysisTest { } } } ->>>>>>> origin/master } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 14f46f8a66df..f55ec2777ef2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -500,7 +500,7 @@ case class TruncateTableCommand( } abstract class DescribeCommandBase extends RunnableCommand { - override val output = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES + override val output = DescribeTableSchemas.describeTableAttributes() protected def describeSchema( schema: 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 0a43a0138b57..d00a6957c220 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,16 +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} -<<<<<<< HEAD -import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} -||||||| merged common ancestors -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} ->>>>>>> origin/master +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand} import org.apache.spark.sql.execution.datasources.v2.{CatalogTableAsV2, DataSourceV2Relation} @@ -102,7 +94,6 @@ case class DataSourceResolution( convertCTAS(v2SessionCatalog.asTableCatalog, identifier, create) } -<<<<<<< HEAD case DescribeColumnStatement( AsTableIdentifier(tableName), colName, isExtended) => DescribeColumnCommand(tableName, colName, isExtended) @@ -115,15 +106,6 @@ case class DataSourceResolution( AsTableIdentifier(tableName), partitionSpec, isExtended) => DescribeTableCommand(tableName, partitionSpec, isExtended) - case DescribeTableStatement( - CatalogObjectIdentifier(Some(catalog), ident), partitionSpec, isExtended) => - if (partitionSpec.nonEmpty) { - throw new AnalysisException("DESCRIBE TABLE... PARTITION is not supported for v2 tables.") - } - DescribeTable(catalog.asTableCatalog, ident, isExtended) - -||||||| merged common ancestors -======= case ReplaceTableStatement( AsTableIdentifier(table), schema, partitionCols, bucketSpec, properties, V1WriteProvider(provider), options, location, comment, orCreate) => @@ -156,7 +138,6 @@ case class DataSourceResolution( .asTableCatalog convertRTAS(catalog, identifier, rtas) ->>>>>>> origin/master case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) => DropTable(catalog.asTableCatalog, ident, ifExists) 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 ec64a9ff7391..de0a9202fa42 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 @@ -24,15 +24,8 @@ 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 -<<<<<<< HEAD -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} -||||||| merged common ancestors -import org.apache.spark.sql.catalyst.plans.logical.{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} ->>>>>>> origin/master +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} -import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} import org.apache.spark.sql.sources @@ -244,7 +237,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { Nil } - case DescribeTable(catalog, ident, isExtended) => + case DescribeTable(catalog, ident, _, isExtended) => DescribeTableExec(catalog, ident, isExtended) :: Nil case DropTable(catalog, ident, ifExists) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index c92bd9c72165..4c88ca741192 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -28,15 +28,17 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow import org.apache.spark.sql.catalyst.plans.DescribeTableSchemas import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.types.StructType case class DescribeTableExec( catalog: TableCatalog, ident: Identifier, isExtended: Boolean) extends LeafExecNode { - import DescribeTableExec._ + override val output: Seq[AttributeReference] = + DescribeTableSchemas.describeTableAttributes() - override def output: Seq[AttributeReference] = DescribeTableSchemas.DESCRIBE_TABLE_ATTRIBUTES + private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind() override protected def doExecute(): RDD[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() @@ -63,7 +65,7 @@ case class DescribeTableExec( } private def addPartitioning(rows: ArrayBuffer[InternalRow], table: Table): Unit = { - rows += EMPTY_ROW + rows += emptyRow() rows += toCatalystRow(" Partitioning", "", "") rows += toCatalystRow("--------------", "", "") if (table.partitioning.isEmpty) { @@ -76,21 +78,17 @@ case class DescribeTableExec( } private def addProperties(rows: ArrayBuffer[InternalRow], table: Table): Unit = { - rows += EMPTY_ROW + rows += emptyRow() rows += toCatalystRow(" Table Property", " Value", "") rows += toCatalystRow("----------------", "-------", "") rows ++= table.properties.asScala.toList.sortBy(_._1).map { case (key, value) => toCatalystRow(key, value, "") } } -} -private object DescribeTableExec { - private val ENCODER = RowEncoder(DescribeTableSchemas.DESCRIBE_TABLE_SCHEMA) - private val EMPTY_ROW = toCatalystRow("", "", "") + private def emptyRow(): InternalRow = toCatalystRow("", "", "") private def toCatalystRow(strs: String*): InternalRow = { - ENCODER.resolveAndBind().toRow( - new GenericRowWithSchema(strs.toArray, DescribeTableSchemas.DESCRIBE_TABLE_SCHEMA)) + encoder.toRow(new GenericRowWithSchema(strs.toArray, schema)).copy() } } 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 b11d46903f04..1e245d55919c 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 @@ -21,27 +21,15 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter -<<<<<<< HEAD -import org.apache.spark.sql.{QueryTest, Row} -||||||| merged common ancestors -import org.apache.spark.sql.QueryTest -======= import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, QueryTest} ->>>>>>> origin/master +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalog.v2.Identifier 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 import org.apache.spark.sql.test.SharedSQLContext -<<<<<<< HEAD -import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} -||||||| merged common ancestors -import org.apache.spark.sql.types.{LongType, StringType, StructType} -======= -import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampType} ->>>>>>> origin/master +import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, Metadata, StringType, StructField, StructType, TimestampType} class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { @@ -88,11 +76,11 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn " USING foo" + " PARTITIONED BY (id)") val descriptionDf = spark.sql("DESCRIBE TABLE testcat.table_name") - assert(descriptionDf.schema === StructType( + assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === Seq( - StructField("col_name", StringType, nullable = false), - StructField("data_type", StringType, nullable = false), - StructField("comment", StringType)))) + ("col_name", StringType), + ("data_type", StringType), + ("comment", StringType))) val description = descriptionDf.collect() assert(description === Seq( Row("id", "bigint", ""), @@ -105,11 +93,11 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn " PARTITIONED BY (id)" + " TBLPROPERTIES ('bar'='baz')") val descriptionDf = spark.sql("DESCRIBE TABLE EXTENDED testcat.table_name") - assert(descriptionDf.schema === StructType( - Seq( - StructField("col_name", StringType, nullable = false), - StructField("data_type", StringType, nullable = false), - StructField("comment", StringType)))) + assert(descriptionDf.schema.map(field => (field.name, field.dataType)) + === Seq( + ("col_name", StringType), + ("data_type", StringType), + ("comment", StringType))) assert(descriptionDf.collect() .map(_.toSeq) .map(_.toArray.map(_.toString.trim)) === Array( 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 15c46531edb0..c5eebceaa8a3 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 @@ -69,26 +69,8 @@ class TestInMemoryTableCatalog extends TableCatalog { if (tables.containsKey(ident)) { throw new TableAlreadyExistsException(ident) } -<<<<<<< HEAD - - val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions) -||||||| merged common ancestors - - if (partitions.nonEmpty) { - throw new UnsupportedOperationException( - s"Catalog $name: Partitioned tables are not supported") - } - - val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties) -======= TestInMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) - if (partitions.nonEmpty) { - throw new UnsupportedOperationException( - s"Catalog $name: Partitioned tables are not supported") - } - - val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties) ->>>>>>> origin/master + val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions) tables.put(ident, table) @@ -100,20 +82,14 @@ class TestInMemoryTableCatalog extends TableCatalog { case Some(table) => val properties = CatalogV2Util.applyPropertiesChanges(table.properties, changes) val schema = CatalogV2Util.applySchemaChanges(table.schema, changes) -<<<<<<< HEAD - val newTable = new InMemoryTable( - table.name, schema, properties, table.partitioning, table.data) -||||||| merged common ancestors - val newTable = new InMemoryTable(table.name, schema, properties, table.data) -======= // fail if the last column in the schema was dropped if (schema.fields.isEmpty) { throw new IllegalArgumentException(s"Cannot drop all fields") } - val newTable = new InMemoryTable(table.name, schema, properties, table.data) ->>>>>>> origin/master + val newTable = new InMemoryTable( + table.name, schema, properties, table.partitioning, table.data) tables.put(ident, newTable) @@ -253,7 +229,7 @@ class TestStagingInMemoryCatalog validateStagedTable(partitions, properties) new TestStagedCreateTable( ident, - new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) + new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions)) } override def stageReplace( @@ -264,7 +240,7 @@ class TestStagingInMemoryCatalog validateStagedTable(partitions, properties) new TestStagedReplaceTable( ident, - new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) + new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions)) } override def stageCreateOrReplace( @@ -275,7 +251,7 @@ class TestStagingInMemoryCatalog validateStagedTable(partitions, properties) new TestStagedCreateOrReplaceTable( ident, - new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) + new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions)) } private def validateStagedTable( From 18d688f17b77987f59664d86011b801b17d867b8 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 23 Jul 2019 15:45:03 -0700 Subject: [PATCH 11/15] Throw exception if table doesn't exist --- .../sql/execution/datasources/v2/DescribeTableExec.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 4c88ca741192..51f7b2c10ba9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRowWithSchema} import org.apache.spark.sql.catalyst.plans.DescribeTableSchemas @@ -52,7 +53,9 @@ case class DescribeTableExec( } } else { - rows += toCatalystRow(s"Table $ident does not exist.", "", "") + // Should rarely happen because the Analyzer should catch this, but the table could have been + // dropped between the analysis phase and the execution phase. + throw new NoSuchTableException(ident) } sparkContext.parallelize(rows) } From 2b77c801b8e5de79b3bacdc1ae4425934c5cb90a Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 29 Jul 2019 18:42:13 -0700 Subject: [PATCH 12/15] Address comments, resolve conflicts --- .../sql/catalyst/analysis/Analyzer.scala | 29 ++++++--- .../sql/catalyst/parser/AstBuilder.scala | 8 +-- ...chemas.scala => DescribeTableSchema.scala} | 2 +- .../plans/logical/basicLogicalOperators.scala | 2 +- .../logical/sql/DescribeTableStatement.scala | 4 +- .../sql/catalyst/parser/DDLParserSuite.scala | 14 +---- .../spark/sql/execution/command/tables.scala | 4 +- .../datasources/v2/DescribeTableExec.scala | 4 +- .../sources/v2/TestInMemoryTableCatalog.scala | 61 ++----------------- 9 files changed, 36 insertions(+), 92 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/{DescribeTableSchemas.scala => DescribeTableSchema.scala} (97%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 3fb4b8ded765..721ae19b236f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -36,13 +36,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -<<<<<<< HEAD -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, DescribeTableStatement} -||||||| merged common ancestors -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement} -======= -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, InsertIntoStatement} ->>>>>>> origin/master +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, DescribeTableStatement, InsertIntoStatement} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL @@ -177,6 +171,7 @@ class Analyzer( Batch("Resolution", fixedPoint, ResolveTableValuedFunctions :: ResolveAlterTable :: + ResolveDescribeTable :: ResolveInsertInto :: ResolveTables :: ResolveRelations :: @@ -985,6 +980,26 @@ class Analyzer( isExtended) } } + /** + * Resolve DESCRIBE TABLE statements that use a DSv2 catalog. + * + * This rule converts unresolved DESCRIBE TABLE statements to v2 when a v2 catalog is responsible + * for the table identifier. A v2 catalog is responsible for an identifier when the identifier + * has a catalog specified, like prod_catalog.db.table, or when a default v2 catalog is set and + * the table identifier does not include a catalog. + */ + object ResolveDescribeTable extends Rule[LogicalPlan] { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case describe @ DescribeTableStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), partitionSpec, isExtended) => + DescribeTable( + v2Catalog.asTableCatalog, + ident, + UnresolvedRelation(describe.tableName), + isExtended) + } + } /** * Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s from 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 acc2628393f3..2f3576ad6522 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,13 +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._ -<<<<<<< HEAD -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} -||||||| merged common ancestors -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.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} ->>>>>>> origin/master +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, 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._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchema.scala similarity index 97% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchema.scala index 260fc58253fe..ff35972b901f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchemas.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/DescribeTableSchema.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.types.{MetadataBuilder, StringType, StructField, StructType} -private[sql] object DescribeTableSchemas { +private[sql] object DescribeTableSchema { def describeTableAttributes(): Seq[AttributeReference] = Seq( AttributeReference("col_name", StringType, nullable = false, new MetadataBuilder().putString("comment", "name of the column").build())(), 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 0b741f44acf4..8e4b3b5e7aca 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 @@ -550,7 +550,7 @@ case class DescribeTable( override def children: Seq[LogicalPlan] = Seq(table) - override val output = DescribeTableSchemas.describeTableAttributes() + override val output = DescribeTableSchema.describeTableAttributes() } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala index 62743ddadd93..bf837441bb50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.catalyst.plans.logical.sql import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.DescribeTableSchemas +import org.apache.spark.sql.catalyst.plans.DescribeTableSchema case class DescribeTableStatement( tableName: Seq[String], partitionSpec: TablePartitionSpec, isExtended: Boolean) extends ParsedStatement { - override def output: Seq[Attribute] = DescribeTableSchemas.describeTableAttributes() + override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes() } 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 4c3af01825a7..d8c3ebc5b966 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 @@ -23,16 +23,8 @@ import org.apache.spark.sql.AnalysisException 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, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.BucketSpec -<<<<<<< HEAD -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, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} -||||||| merged common ancestors -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, ReplaceTableAsSelectStatement, ReplaceTableStatement} -======= import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} ->>>>>>> origin/master +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -625,7 +617,6 @@ class DDLParserSuite extends AnalysisTest { } } -<<<<<<< HEAD test("describe table column") { comparePlans(parsePlan("DESCRIBE t col"), DescribeColumnStatement( @@ -667,8 +658,6 @@ class DDLParserSuite extends AnalysisTest { DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) } -||||||| merged common ancestors -======= test("insert table: basic append") { Seq( "INSERT INTO TABLE testcat.ns1.ns2.tbl SELECT * FROM source", @@ -775,7 +764,6 @@ class DDLParserSuite extends AnalysisTest { assert(exc.getMessage.contains("INSERT INTO ... IF NOT EXISTS")) } ->>>>>>> origin/master private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index f55ec2777ef2..ca42de39db86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.DescribeTableSchemas +import org.apache.spark.sql.catalyst.plans.DescribeTableSchema import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} @@ -500,7 +500,7 @@ case class TruncateTableCommand( } abstract class DescribeCommandBase extends RunnableCommand { - override val output = DescribeTableSchemas.describeTableAttributes() + override val output = DescribeTableSchema.describeTableAttributes() protected def describeSchema( schema: StructType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 51f7b2c10ba9..fac86278f34a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRowWithSchema} -import org.apache.spark.sql.catalyst.plans.DescribeTableSchemas +import org.apache.spark.sql.catalyst.plans.DescribeTableSchema import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType @@ -37,7 +37,7 @@ case class DescribeTableExec( isExtended: Boolean) extends LeafExecNode { override val output: Seq[AttributeReference] = - DescribeTableSchemas.describeTableAttributes() + DescribeTableSchema.describeTableAttributes() private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind() 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 8a1c8e97df38..a1d310a0f3ab 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 @@ -71,19 +71,7 @@ class TestInMemoryTableCatalog extends TableCatalog { throw new TableAlreadyExistsException(ident) } TestInMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) -<<<<<<< HEAD - val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions) -||||||| merged common ancestors - if (partitions.nonEmpty) { - throw new UnsupportedOperationException( - s"Catalog $name: Partitioned tables are not supported") - } - - val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties) -======= - val table = new InMemoryTable(s"$name.${ident.quoted}", schema, partitions, properties) ->>>>>>> origin/master tables.put(ident, table) @@ -101,15 +89,8 @@ class TestInMemoryTableCatalog extends TableCatalog { throw new IllegalArgumentException(s"Cannot drop all fields") } -<<<<<<< HEAD - val newTable = new InMemoryTable( - table.name, schema, properties, table.partitioning, table.data) -||||||| merged common ancestors - val newTable = new InMemoryTable(table.name, schema, properties, table.data) -======= val newTable = new InMemoryTable(table.name, schema, table.partitioning, properties) .withData(table.data) ->>>>>>> origin/master tables.put(ident, newTable) @@ -134,40 +115,24 @@ class TestInMemoryTableCatalog extends TableCatalog { class InMemoryTable( val name: String, val schema: StructType, -<<<<<<< HEAD - override val properties: util.Map[String, String], - override val partitioning: Array[Transform]) -||||||| merged common ancestors - override val properties: util.Map[String, String]) -======= override val partitioning: Array[Transform], override val properties: util.Map[String, String]) ->>>>>>> origin/master extends Table with SupportsRead with SupportsWrite { -<<<<<<< HEAD def this( name: String, schema: StructType, properties: util.Map[String, String], partitioning: Array[Transform], data: Array[BufferedRows]) = { - this(name, schema, properties, partitioning) - replaceData(data) -||||||| merged common ancestors - def this( - name: String, - schema: StructType, - properties: util.Map[String, String], - data: Array[BufferedRows]) = { - this(name, schema, properties) - replaceData(data) -======= + this(name, schema, partitioning, properties) + withData(data) + } + partitioning.foreach { t => if (!t.isInstanceOf[IdentityTransform]) { throw new IllegalArgumentException(s"Transform $t must be IdentityTransform") } ->>>>>>> origin/master } @volatile var dataMap: mutable.Map[Seq[Any], BufferedRows] = mutable.Map.empty @@ -331,13 +296,7 @@ class TestStagingInMemoryCatalog validateStagedTable(partitions, properties) new TestStagedCreateTable( ident, -<<<<<<< HEAD - new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions)) -||||||| merged common ancestors - new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) -======= new InMemoryTable(s"$name.${ident.quoted}", schema, partitions, properties)) ->>>>>>> origin/master } override def stageReplace( @@ -348,13 +307,7 @@ class TestStagingInMemoryCatalog validateStagedTable(partitions, properties) new TestStagedReplaceTable( ident, -<<<<<<< HEAD - new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions)) -||||||| merged common ancestors - new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) -======= new InMemoryTable(s"$name.${ident.quoted}", schema, partitions, properties)) ->>>>>>> origin/master } override def stageCreateOrReplace( @@ -365,13 +318,7 @@ class TestStagingInMemoryCatalog validateStagedTable(partitions, properties) new TestStagedCreateOrReplaceTable( ident, -<<<<<<< HEAD - new InMemoryTable(s"$name.${ident.quoted}", schema, properties, partitions)) -||||||| merged common ancestors - new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) -======= new InMemoryTable(s"$name.${ident.quoted}", schema, partitions, properties)) ->>>>>>> origin/master } private def validateStagedTable( From f701daaf1bd22ff55fcadb3cfd488c324e2f6711 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 29 Jul 2019 18:54:16 -0700 Subject: [PATCH 13/15] Fix some redundant code and address another comment --- .../sql/catalyst/analysis/Analyzer.scala | 16 ++-------- .../plans/logical/basicLogicalOperators.scala | 6 +--- .../datasources/v2/DataSourceV2Strategy.scala | 4 +-- .../datasources/v2/DescribeTableExec.scala | 29 +++++-------------- 4 files changed, 13 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 721ae19b236f..83092cd5545d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -970,14 +970,6 @@ class Analyzer( v2Catalog.asTableCatalog, ident, UnresolvedRelation(alter.tableName), Seq(TableChange.setProperty("location", newLoc))) - - case describe @ DescribeTableStatement( - CatalogObjectIdentifier(Some(v2Catalog), ident), partitionSpec, isExtended) => - DescribeTable( - v2Catalog.asTableCatalog, - ident, - UnresolvedRelation(describe.tableName), - isExtended) } } /** @@ -992,12 +984,8 @@ class Analyzer( import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case describe @ DescribeTableStatement( - CatalogObjectIdentifier(Some(v2Catalog), ident), partitionSpec, isExtended) => - DescribeTable( - v2Catalog.asTableCatalog, - ident, - UnresolvedRelation(describe.tableName), - isExtended) + CatalogObjectIdentifier(Some(v2Catalog), ident), _, isExtended) => + DescribeTable(UnresolvedRelation(describe.tableName), isExtended) } } 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 8e4b3b5e7aca..6f33944fc1cf 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 @@ -542,11 +542,7 @@ object OverwritePartitionsDynamic { } } -case class DescribeTable( - catalog: TableCatalog, - ident: Identifier, - table: NamedRelation, - isExtended: Boolean) extends Command { +case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { override def children: Seq[LogicalPlan] = Seq(table) 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 de0a9202fa42..91fc2e068af7 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 @@ -237,8 +237,8 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { Nil } - case DescribeTable(catalog, ident, _, isExtended) => - DescribeTableExec(catalog, ident, isExtended) :: Nil + case DescribeTable(r: DataSourceV2Relation, isExtended) => + DescribeTableExec(r.table, isExtended) :: Nil case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index fac86278f34a..5db79c84a395 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -21,9 +21,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRowWithSchema} import org.apache.spark.sql.catalyst.plans.DescribeTableSchema @@ -31,10 +29,7 @@ import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType -case class DescribeTableExec( - catalog: TableCatalog, - ident: Identifier, - isExtended: Boolean) extends LeafExecNode { +case class DescribeTableExec(table: Table, isExtended: Boolean) extends LeafExecNode { override val output: Seq[AttributeReference] = DescribeTableSchema.describeTableAttributes() @@ -43,31 +38,23 @@ case class DescribeTableExec( override protected def doExecute(): RDD[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() - if (catalog.tableExists(ident)) { - val table = catalog.loadTable(ident) - addSchema(rows, table) + addSchema(rows) - if (isExtended) { - addPartitioning(rows, table) - addProperties(rows, table) - } - - } else { - // Should rarely happen because the Analyzer should catch this, but the table could have been - // dropped between the analysis phase and the execution phase. - throw new NoSuchTableException(ident) + if (isExtended) { + addPartitioning(rows) + addProperties(rows) } sparkContext.parallelize(rows) } - private def addSchema(rows: ArrayBuffer[InternalRow], table: Table): Unit = { + private def addSchema(rows: ArrayBuffer[InternalRow]): Unit = { rows ++= table.schema.map{ column => toCatalystRow( column.name, column.dataType.simpleString, column.getComment().getOrElse("")) } } - private def addPartitioning(rows: ArrayBuffer[InternalRow], table: Table): Unit = { + private def addPartitioning(rows: ArrayBuffer[InternalRow]): Unit = { rows += emptyRow() rows += toCatalystRow(" Partitioning", "", "") rows += toCatalystRow("--------------", "", "") @@ -80,7 +67,7 @@ case class DescribeTableExec( } } - private def addProperties(rows: ArrayBuffer[InternalRow], table: Table): Unit = { + private def addProperties(rows: ArrayBuffer[InternalRow]): Unit = { rows += emptyRow() rows += toCatalystRow(" Table Property", " Value", "") rows += toCatalystRow("----------------", "-------", "") From 9e04c6ee93ed77620c7e621c3ff4ef45b3f07e18 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 29 Jul 2019 18:57:48 -0700 Subject: [PATCH 14/15] Add test for non-existent table. --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 1 - .../apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala | 6 ++++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 83092cd5545d..a2f2f12cc4d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -981,7 +981,6 @@ class Analyzer( * the table identifier does not include a catalog. */ object ResolveDescribeTable extends Rule[LogicalPlan] { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case describe @ DescribeTableStatement( CatalogObjectIdentifier(Some(v2Catalog), ident), _, isExtended) => 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 05b977d4d176..d95021077f9c 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 @@ -87,6 +87,12 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn Row("data", "string", ""))) } + test("DescribeTable with v2 catalog when table does not exist.") { + intercept[AnalysisException] { + spark.sql("DESCRIBE TABLE testcat.table_name") + } + } + test("DescribeTable extended using v2 catalog") { spark.sql("CREATE TABLE testcat.table_name (id bigint, data string)" + " USING foo" + From cff78a16e691917e812b4cd63bf7544a54af4742 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 6 Aug 2019 11:57:27 -0700 Subject: [PATCH 15/15] Remove unused constructor --- .../sql/sources/v2/TestInMemoryTableCatalog.scala | 10 ---------- 1 file changed, 10 deletions(-) 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 a1d310a0f3ab..7c51a29bde90 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 @@ -119,16 +119,6 @@ class InMemoryTable( override val properties: util.Map[String, String]) extends Table with SupportsRead with SupportsWrite { - def this( - name: String, - schema: StructType, - properties: util.Map[String, String], - partitioning: Array[Transform], - data: Array[BufferedRows]) = { - this(name, schema, partitioning, properties) - withData(data) - } - partitioning.foreach { t => if (!t.isInstanceOf[IdentityTransform]) { throw new IllegalArgumentException(s"Transform $t must be IdentityTransform")