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 572fccf3b211..acbe67a974bf 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 @@ -2192,19 +2192,6 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsed, expected) } - test("SHOW TBLPROPERTIES table") { - comparePlans( - parsePlan("SHOW TBLPROPERTIES a.b.c"), - ShowTableProperties( - UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES", true), - None)) - - comparePlans( - parsePlan("SHOW TBLPROPERTIES a.b.c('propKey1')"), - ShowTableProperties( - UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES", true), Some("propKey1"))) - } - test("DESCRIBE FUNCTION") { comparePlans( parsePlan("DESC FUNCTION a"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala index 44fbc639a580..95624f3f61c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.connector -import org.apache.spark.sql.{DataFrame, Row, SaveMode} +import org.apache.spark.sql.{DataFrame, SaveMode} import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTable, Table, TableCatalog} class DataSourceV2SQLSessionCatalogSuite @@ -64,22 +64,6 @@ class DataSourceV2SQLSessionCatalogSuite } } - test("SPARK-31624: SHOW TBLPROPERTIES working with V2 tables and the session catalog") { - val t1 = "tbl" - withTable(t1) { - sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format TBLPROPERTIES " + - "(key='v', key2='v2')") - - checkAnswer(sql(s"SHOW TBLPROPERTIES $t1"), Seq(Row("key", "v"), Row("key2", "v2"))) - - checkAnswer(sql(s"SHOW TBLPROPERTIES $t1('key')"), Row("key", "v")) - - checkAnswer( - sql(s"SHOW TBLPROPERTIES $t1('keyX')"), - Row("keyX", s"Table default.$t1 does not have property: keyX")) - } - } - test("SPARK-33651: allow CREATE EXTERNAL TABLE without LOCATION") { withTable("t") { val prop = TestV2SessionCatalogBase.SIMULATE_ALLOW_EXTERNAL_PROPERTY + "=true" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index d2d46d620b31..8e5d1026e43a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2119,62 +2119,6 @@ class DataSourceV2SQLSuite assert(e.message.contains("CREATE VIEW is only supported with v1 tables")) } - test("SHOW TBLPROPERTIES: v2 table") { - val t = "testcat.ns1.ns2.tbl" - withTable(t) { - val user = "andrew" - val status = "new" - val provider = "foo" - spark.sql(s"CREATE TABLE $t (id bigint, data string) USING $provider " + - s"TBLPROPERTIES ('user'='$user', 'status'='$status')") - - val properties = sql(s"SHOW TBLPROPERTIES $t") - - val schema = new StructType() - .add("key", StringType, nullable = false) - .add("value", StringType, nullable = false) - - val expected = Seq( - Row("status", status), - Row("user", user)) - - assert(properties.schema === schema) - assert(expected === properties.collect()) - } - } - - test("SHOW TBLPROPERTIES(key): v2 table") { - val t = "testcat.ns1.ns2.tbl" - withTable(t) { - val user = "andrew" - val status = "new" - val provider = "foo" - spark.sql(s"CREATE TABLE $t (id bigint, data string) USING $provider " + - s"TBLPROPERTIES ('user'='$user', 'status'='$status')") - - val properties = sql(s"SHOW TBLPROPERTIES $t ('status')") - - val expected = Seq(Row("status", status)) - - assert(expected === properties.collect()) - } - } - - test("SHOW TBLPROPERTIES(key): v2 table, key not found") { - val t = "testcat.ns1.ns2.tbl" - withTable(t) { - val nonExistingKey = "nonExistingKey" - spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo " + - s"TBLPROPERTIES ('user'='andrew', 'status'='new')") - - val properties = sql(s"SHOW TBLPROPERTIES $t ('$nonExistingKey')") - - val expected = Seq(Row(nonExistingKey, s"Table $t does not have property: $nonExistingKey")) - - assert(expected === properties.collect()) - } - } - test("DESCRIBE FUNCTION: only support session catalog") { val e = intercept[AnalysisException] { sql("DESCRIBE FUNCTION testcat.ns1.ns2.fun") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTblPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTblPropertiesParserSuite.scala new file mode 100644 index 000000000000..532705641d9d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTblPropertiesParserSuite.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.command + +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedTableOrView} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan +import org.apache.spark.sql.catalyst.plans.logical.ShowTableProperties + +class ShowTblPropertiesParserSuite extends AnalysisTest { + test("show tblproperties") { + comparePlans( + parsePlan("SHOW TBLPROPERTIES a.b.c"), + ShowTableProperties( + UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES", true), + None)) + + comparePlans( + parsePlan("SHOW TBLPROPERTIES a.b.c('propKey1')"), + ShowTableProperties( + UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES", true), Some("propKey1"))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTblPropertiesSuiteBase.scala new file mode 100644 index 000000000000..28f58d67f236 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTblPropertiesSuiteBase.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.types.{StringType, StructType} + +/** + * This base suite contains unified tests for the `SHOW TBLPROPERTIES` command that check V1 and V2 + * table catalogs. The tests that cannot run for all supported catalogs are located in more + * specific test suites: + * + * - V2 table catalog tests: `org.apache.spark.sql.execution.command.v2.ShowTblPropertiesSuite` + * - V1 table catalog tests: + * `org.apache.spark.sql.execution.command.v1.ShowTblPropertiesSuiteBase` + * - V1 In-Memory catalog: `org.apache.spark.sql.execution.command.v1.ShowTblPropertiesSuite` + * - V1 Hive External catalog: +* `org.apache.spark.sql.hive.execution.command.ShowTblPropertiesSuite` + */ +trait ShowTblPropertiesSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command = "SHOW TBLPROPERTIES" + + test("SHOW TBLPROPERTIES BASIC") { + withNamespaceAndTable("ns1", "tbl") { tbl => + val user = "andrew" + val status = "new" + spark.sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing " + + s"TBLPROPERTIES ('user'='$user', 'status'='$status')") + + val properties = sql(s"SHOW TBLPROPERTIES $tbl").filter("key != 'transient_lastDdlTime'") + val schema = new StructType() + .add("key", StringType, nullable = false) + .add("value", StringType, nullable = false) + val expected = Seq( + Row("status", status), + Row("user", user)) + + assert(properties.schema === schema) + checkAnswer(properties.sort("key"), expected) + } + } + + test("SHOW TBLPROPERTIES(KEY)") { + withNamespaceAndTable("ns1", "tbl") { tbl => + val user = "andrew" + val status = "new" + spark.sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing " + + s"TBLPROPERTIES ('user'='$user', 'status'='$status')") + + val properties = sql(s"SHOW TBLPROPERTIES $tbl ('status')") + val expected = Seq(Row("status", status)) + checkAnswer(properties, expected) + } + } + + test("SHOW TBLPROPERTIES WITH TABLE NOT EXIST") { + val message = intercept[AnalysisException] { + sql("SHOW TBLPROPERTIES BADTABLE") + }.getMessage + assert(message.contains("Table or view not found: BADTABLE")) + } + + test("SHOW TBLPROPERTIES(KEY) KEY NOT FOUND") { + withNamespaceAndTable("ns1", "tbl") { tbl => + val nonExistingKey = "nonExistingKey" + spark.sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing " + + s"TBLPROPERTIES ('user'='andrew', 'status'='new')") + + val res = sql(s"SHOW TBLPROPERTIES $tbl ('$nonExistingKey')").collect() + assert(res.length == 1) + assert(res.head.getString(0) == nonExistingKey) + assert(res.head.getString(1).contains(s"does not have property: $nonExistingKey")) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTblPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTblPropertiesSuite.scala new file mode 100644 index 000000000000..190b2701221d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTblPropertiesSuite.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.internal.SQLConf + +/** + * This base suite contains unified tests for the `SHOW TBLPROPERTIES` command that checks V1 + * table catalogs. The tests that cannot run for all V1 catalogs are located in more + * specific test suites: + * + * - V1 In-Memory catalog: `org.apache.spark.sql.execution.command.v1.ShowTblPropertiesSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.ShowTblPropertiesSuite` + */ +trait ShowTblPropertiesSuiteBase extends command.ShowTblPropertiesSuiteBase + with command.TestsV1AndV2Commands { + + test("SHOW TBLPROPERTIES WITH LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA") { + withNamespaceAndTable("ns1", "tbl") { tbl => + spark.sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing " + + s"TBLPROPERTIES ('user'='andrew', 'status'='new')") + withSQLConf(SQLConf.LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA.key -> "true") { + checkAnswer(sql(s"SHOW TBLPROPERTIES $tbl('user')"), Row("andrew")) + checkAnswer(sql(s"SHOW TBLPROPERTIES $tbl('status')"), Row("new")) + } + } + } + + test("SHOW TBLPROPERTIES FOR VIEW") { + val v = "testview" + withView(v) { + spark.sql(s"CREATE VIEW $v TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1") + checkAnswer(sql(s"SHOW TBLPROPERTIES $v").filter("key != 'transient_lastDdlTime'"), + Seq(Row("p1", "v1"), Row("p2", "v2"))) + checkAnswer(sql(s"SHOW TBLPROPERTIES $v('p1')"), Row("p1", "v1")) + checkAnswer(sql(s"SHOW TBLPROPERTIES $v('p3')"), + Row("p3", s"Table default.$v does not have property: p3")) + } + } + + test("SHOW TBLPROPERTIES FOR TEMPORARY IEW") { + val v = "testview" + withView(v) { + spark.sql(s"CREATE TEMPORARY VIEW $v AS SELECT 1 AS c1;") + checkAnswer(sql(s"SHOW TBLPROPERTIES $v"), Seq.empty) + } + } +} + +/** + * The class contains tests for the `SHOW TBLPROPERTIES` command to check V1 In-Memory + * table catalog. + */ +class ShowTblPropertiesSuite extends ShowTblPropertiesSuiteBase with CommandSuiteBase { + override def commandVersion: String = super[ShowTblPropertiesSuiteBase].commandVersion +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTblPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTblPropertiesSuite.scala new file mode 100644 index 000000000000..ace1e30a26b2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTblPropertiesSuite.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.sql.execution.command + +/** + * The class contains tests for the `SHOW TBLPROPERTIES` command to check V2 table catalogs. + */ +class ShowTblPropertiesSuite extends command.ShowTblPropertiesSuiteBase with CommandSuiteBase { +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 883400688857..eb2dd4640e5d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -121,46 +121,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } - test("show tblproperties of data source tables - basic") { - checkAnswer( - sql("SHOW TBLPROPERTIES parquet_tab1").filter(s"key = 'my_key1'"), - Row("my_key1", "v1") :: Nil - ) - - checkAnswer( - sql(s"SHOW TBLPROPERTIES parquet_tab1('my_key1')"), - Row("my_key1", "v1") :: Nil - ) - } - - test("show tblproperties for datasource table - errors") { - val message = intercept[AnalysisException] { - sql("SHOW TBLPROPERTIES badtable") - }.getMessage - assert(message.contains("Table or view not found: badtable")) - - // When key is not found, a row containing the error is returned. - checkAnswer( - sql("SHOW TBLPROPERTIES parquet_tab1('invalid.prop.key')"), - Row("invalid.prop.key", - "Table default.parquet_tab1 does not have property: invalid.prop.key") :: Nil - ) - } - - test("SPARK-34240 Unify output of SHOW TBLPROPERTIES and pass output attributes properly") { - checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2").filter("key != 'transient_lastDdlTime'"), - Row("prop1Key", "prop1Val") :: Row("`prop2Key`", "prop2Val") :: Nil) - checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('prop1Key')"), Row("prop1Key", "prop1Val")) - checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('`prop2Key`')"), Row("`prop2Key`", "prop2Val")) - withSQLConf(SQLConf.LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA.key -> "true") { - checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2").filter("key != 'transient_lastDdlTime'"), - Row("prop1Key", "prop1Val") :: Row("`prop2Key`", "prop2Val") :: Nil) - checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('prop1Key')"), Row("prop1Val")) - checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('`prop2Key`')"), - Row("prop2Val")) - } - } - Seq(true, false).foreach { local => val loadQuery = if (local) "LOAD DATA LOCAL" else "LOAD DATA" test(loadQuery) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTblPropertiesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTblPropertiesSuite.scala new file mode 100644 index 000000000000..f84eee5c9494 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTblPropertiesSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution.command + +import org.apache.spark.sql.execution.command.v1 + +/** + * The class contains tests for the `SHOW TBLPROPERTIES` command to check V1 Hive external + * table catalog. + */ +class ShowTblPropertiesSuite extends v1.ShowTblPropertiesSuiteBase with CommandSuiteBase { + override def commandVersion: String = super[ShowTblPropertiesSuiteBase].commandVersion +}