You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/11/07 18:40:56 UTC

[GitHub] [spark] MaxGekk opened a new pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

MaxGekk opened a new pull request #30287:
URL: https://github.com/apache/spark/pull/30287


   ### What changes were proposed in this pull request?
   In the PR, I propose to gather common `SHOW TABLES` tests into one trait `org.apache.spark.sql.execution.command.ShowTablesSuite`, and put datasource specific tests to the `v1.ShowTablesSuite` and `v2.ShowTablesSuite`.
   
   ### Why are the changes needed?
   - The unification will allow to run common `SHOW TABLES` tests for both DSv1 and DSv2
   - We can detect missing features and differences between DSv1 and DSv2 implementations.
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   ### How was this patch tested?
   By running new test suites:
   - `org.apache.spark.sql.execution.command.v1.ShowTablesSuite`
   - `org.apache.spark.sql.execution.command.v2.ShowTablesSuite`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723506423






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724727884






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723486613


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35357/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519938633



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuite.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{QueryTest, Row}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+trait ShowTablesSuite extends QueryTest with SharedSparkSession {
+  protected def catalog: String
+  protected def defaultUsing: String
+  case class ShowRow(namespace: String, table: String, isTemporary: Boolean)
+  protected def getRows(showRows: Seq[ShowRow]): Seq[Row]
+  // Gets the schema of `SHOW TABLES`
+  protected def showSchema: StructType
+
+  protected def runShowTablesSql(sqlText: String, expected: Seq[ShowRow]): Unit = {
+    val df = spark.sql(sqlText)
+    assert(df.schema === showSchema)
+    assert(df.collect() === getRows(expected))
+  }
+
+  protected def withSourceViews(f: => Unit): Unit = {
+    withTable("source", "source2") {
+      val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
+      df.createOrReplaceTempView("source")
+      val df2 = spark.createDataFrame(Seq((4L, "d"), (5L, "e"), (6L, "f"))).toDF("id", "data")
+      df2.createOrReplaceTempView("source2")
+      f
+    }
+  }
+
+  test("show an existing table") {
+    val namespace = "test"
+    val table = "people"
+    withDatabase(s"$catalog.$namespace") {
+      sql(s"CREATE DATABASE $catalog.$namespace")
+      withTable(s"$catalog.$namespace.$table") {
+        sql(s"CREATE TABLE $catalog.$namespace.$table (name STRING, id INT) $defaultUsing")
+        runShowTablesSql(s"SHOW TABLES IN $catalog.test", Seq(ShowRow(namespace, table, false)))
+      }
+    }
+  }
+
+  test("show tables with a pattern") {
+    withDatabase(s"$catalog.db", s"$catalog.db2") {
+      sql(s"CREATE DATABASE $catalog.db")
+      sql(s"CREATE DATABASE $catalog.db2")
+      withTable(
+        s"$catalog.db.table",
+        s"$catalog.db.table_name_1",
+        s"$catalog.db.table_name_2",
+        s"$catalog.db2.table_name_2") {
+        sql(s"CREATE TABLE $catalog.db.table (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db.table_name_1 (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db.table_name_2 (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db2.table_name_2 (id bigint, data string) $defaultUsing")
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db",
+          Seq(
+            ShowRow("db", "table", false),
+            ShowRow("db", "table_name_1", false),
+            ShowRow("db", "table_name_2", false)))
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db LIKE '*name*'",
+          Seq(
+            ShowRow("db", "table_name_1", false),
+            ShowRow("db", "table_name_2", false)))
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db LIKE '*2'",
+          Seq(ShowRow("db", "table_name_2", false)))
+      }
+    }
+  }
+
+  // TODO(SPARK-33393): Support SHOW TABLE EXTENDED in DSv2
+  test("SHOW TABLE EXTENDED for default") {

Review comment:
       DSv2 doesn't support `SHOW TABLE EXTENDED`. After https://issues.apache.org/jira/browse/SPARK-33393, the test can work for DSv2 too with minimal changes:
   - Abstract the column name `database` - in DSv2 we should show `namespace` probably
   - DSv2 will not show views (at least at the moment).
   
   I would split the test to 2 tests:
   - SHOW TABLE EXTENDED for tables
   - SHOW TABLE EXTENDED for views.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519612360



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v1
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+class ShowTablesSuite extends CommonShowTablesSuite {
+  override def catalog: String = "spark_catalog"
+  override protected def defaultUsing: String = "USING parquet"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("database", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+      .add("isTemporary", BooleanType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, isTemporary) => Row(namespace, table, isTemporary)
+    }
+  }
+
+  // `SHOW TABLES` returns empty result in V2 catalog instead of throwing the exception.

Review comment:
       I will open an JIRA for this, and add `TODO` here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724875855






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723961526


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35389/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723947009


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35389/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r520317546



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v2
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+import org.apache.spark.sql.connector.InMemoryTableCatalog
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{StringType, StructType}
+
+class ShowTablesSuite extends QueryTest with SharedSparkSession with CommonShowTablesSuite {
+  override def catalog: String = "test_catalog_v2"
+  override protected def defaultUsing: String = "USING _"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("namespace", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, _) => Row(namespace, table)
+    }
+  }
+
+  override def sparkConf: SparkConf = super.sparkConf
+    .set(s"spark.sql.catalog.$catalog", classOf[InMemoryTableCatalog].getName)
+
+  // The test fails with the exception `NoSuchDatabaseException` in V1 catalog.
+  // TODO(SPARK-33394): Throw `NoSuchDatabaseException` for not existing namespace
+  test("show table in a not existing namespace") {
+    runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.n1.n2.db
+  test("show tables in nested namespaces") {
+    withTable(s"$catalog.n1.n2.db") {
+      spark.sql(s"CREATE TABLE $catalog.n1.n2.db.table_name (id bigint, data string) $defaultUsing")
+      runShowTablesSql(
+        s"SHOW TABLES FROM $catalog.n1.n2.db",
+        Seq(ShowRow("n1.n2.db", "table_name", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table
+  test("using v2 catalog with empty namespace") {
+    withTable(s"$catalog.table") {
+      spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
+      runShowTablesSql(s"SHOW TABLES FROM $catalog", Seq(ShowRow("", "table", false)))
+    }
+  }
+
+  test("namespace is not specified and the default catalog is set") {
+    withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) {
+      withTable("table") {
+        spark.sql(s"CREATE TABLE table (id bigint, data string) $defaultUsing")
+        // TODO(SPARK-33403): DSv2 SHOW TABLES doesn't show `default`

Review comment:
       The code above returns the same in DSv2 but fails in DSv1:
   ```
     test("namespace is not specified and the default catalog is set") {
       withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) {
         withTable("table") {
           spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
           runShowTablesSql("SHOW TABLES", Seq(ShowRow("", "table", false)))
         }
       }
     }
   ```
   ```
   The namespace in session catalog must have exactly one name part: spark_catalog.table;
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519564586



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v1
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+class ShowTablesSuite extends CommonShowTablesSuite {
+  override def catalog: String = "spark_catalog"
+  override protected def defaultUsing: String = "USING parquet"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("database", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+      .add("isTemporary", BooleanType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, isTemporary) => Row(namespace, table, isTemporary)
+    }
+  }
+
+  // `SHOW TABLES` returns empty result in V2 catalog instead of throwing the exception.

Review comment:
       this is something we should fix later.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724728756






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519929207



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v2
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
+import org.apache.spark.sql.connector.InMemoryTableCatalog
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{StringType, StructType}
+
+class ShowTablesSuite extends QueryTest with SharedSparkSession with CommonShowTablesSuite {
+  override def catalog: String = "test_catalog_v2"
+  override protected def defaultUsing: String = "USING _"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("namespace", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, _) => Row(namespace, table)
+    }
+  }
+
+  override def sparkConf: SparkConf = super.sparkConf
+    .set(s"spark.sql.catalog.$catalog", classOf[InMemoryTableCatalog].getName)
+
+  // The test fails with the exception `NoSuchDatabaseException` in V1 catalog.
+  // TODO(SPARK-33394): Throw `NoSuchDatabaseException` for not existing namespace
+  test("show table in a not existing namespace") {
+    runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.n1.n2.db
+  test("show tables in nested namespaces") {
+    withTable(s"$catalog.n1.n2.db") {
+      spark.sql(s"CREATE TABLE $catalog.n1.n2.db.table_name (id bigint, data string) $defaultUsing")
+      runShowTablesSql(
+        s"SHOW TABLES FROM $catalog.n1.n2.db",
+        Seq(ShowRow("n1.n2.db", "table_name", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table
+  test("using v2 catalog with empty namespace") {
+    withTable(s"$catalog.table") {
+      spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
+      runShowTablesSql(s"SHOW TABLES FROM $catalog", Seq(ShowRow("", "table", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table

Review comment:
       How about to change the condition here: https://github.com/apache/spark/blob/45b7212fd3d60f537aa034a6255188e0283d05e7/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala#L64 and don't throw the exception if `ident.namespace.length == 2 && ident.namespace(0) == "spark_catalog"`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724688190


   **[Test build #130870 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130870/testReport)** for PR 30287 at commit [`8108cb1`](https://github.com/apache/spark/commit/8108cb1b5879b7873e3bfa43058ec3db7feba148).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519563666



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuite.scala
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{QueryTest, Row}
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
+import org.apache.spark.sql.catalyst.plans.logical.{ShowTables, ShowTableStatement}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+trait ShowTablesSuite extends QueryTest with SharedSparkSession with AnalysisTest {
+  protected def catalog: String
+  protected def defaultUsing: String
+  case class ShowRow(namespace: String, table: String, isTemporary: Boolean)
+  protected def getRows(showRows: Seq[ShowRow]): Seq[Row]
+  // Gets the schema of `SHOW TABLES`
+  protected def showSchema: StructType
+
+  protected def runShowTablesSql(sqlText: String, expected: Seq[ShowRow]): Unit = {
+    val df = spark.sql(sqlText)
+    assert(df.schema === showSchema)
+    assert(df.collect() === getRows(expected))
+  }
+
+  protected def withSourceViews(f: => Unit): Unit = {
+    withTable("source", "source2") {
+      val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
+      df.createOrReplaceTempView("source")
+      val df2 = spark.createDataFrame(Seq((4L, "d"), (5L, "e"), (6L, "f"))).toDF("id", "data")
+      df2.createOrReplaceTempView("source2")
+      f
+    }
+  }
+
+  test("show tables") {

Review comment:
       Can we put the parser tests in `ShowTablesParserSuite`? It doesn't involve data source and doesn't need to be run twice with DS v1 and v2.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724614063


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35463/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724688190


   **[Test build #130870 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130870/testReport)** for PR 30287 at commit [`8108cb1`](https://github.com/apache/spark/commit/8108cb1b5879b7873e3bfa43058ec3db7feba148).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724875855






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723480390


   **[Test build #130748 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130748/testReport)** for PR 30287 at commit [`947c8bc`](https://github.com/apache/spark/commit/947c8bc847b2eed260199f153fd8cd829d4a8fc3).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724070332


   **[Test build #130780 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130780/testReport)** for PR 30287 at commit [`b608f8b`](https://github.com/apache/spark/commit/b608f8b3c593373319eae5e68a4bb6f09c99d7dc).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class ShowTablesParserSuite extends AnalysisTest with SharedSparkSession `
     * `trait ShowTablesSuite extends QueryTest with SharedSparkSession `


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519582669



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuite.scala
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{QueryTest, Row}
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
+import org.apache.spark.sql.catalyst.plans.logical.{ShowTables, ShowTableStatement}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+trait ShowTablesSuite extends QueryTest with SharedSparkSession with AnalysisTest {
+  protected def catalog: String
+  protected def defaultUsing: String
+  case class ShowRow(namespace: String, table: String, isTemporary: Boolean)
+  protected def getRows(showRows: Seq[ShowRow]): Seq[Row]
+  // Gets the schema of `SHOW TABLES`
+  protected def showSchema: StructType
+
+  protected def runShowTablesSql(sqlText: String, expected: Seq[ShowRow]): Unit = {
+    val df = spark.sql(sqlText)
+    assert(df.schema === showSchema)
+    assert(df.collect() === getRows(expected))
+  }
+
+  protected def withSourceViews(f: => Unit): Unit = {
+    withTable("source", "source2") {
+      val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
+      df.createOrReplaceTempView("source")
+      val df2 = spark.createDataFrame(Seq((4L, "d"), (5L, "e"), (6L, "f"))).toDF("id", "data")
+      df2.createOrReplaceTempView("source2")
+      f
+    }
+  }
+
+  test("show tables") {

Review comment:
       In a separate file is better.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724727884






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724071669






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519954776



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v2
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
+import org.apache.spark.sql.connector.InMemoryTableCatalog
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{StringType, StructType}
+
+class ShowTablesSuite extends QueryTest with SharedSparkSession with CommonShowTablesSuite {
+  override def catalog: String = "test_catalog_v2"
+  override protected def defaultUsing: String = "USING _"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("namespace", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, _) => Row(namespace, table)
+    }
+  }
+
+  override def sparkConf: SparkConf = super.sparkConf
+    .set(s"spark.sql.catalog.$catalog", classOf[InMemoryTableCatalog].getName)
+
+  // The test fails with the exception `NoSuchDatabaseException` in V1 catalog.
+  // TODO(SPARK-33394): Throw `NoSuchDatabaseException` for not existing namespace
+  test("show table in a not existing namespace") {
+    runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.n1.n2.db
+  test("show tables in nested namespaces") {
+    withTable(s"$catalog.n1.n2.db") {
+      spark.sql(s"CREATE TABLE $catalog.n1.n2.db.table_name (id bigint, data string) $defaultUsing")
+      runShowTablesSql(
+        s"SHOW TABLES FROM $catalog.n1.n2.db",
+        Seq(ShowRow("n1.n2.db", "table_name", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table
+  test("using v2 catalog with empty namespace") {
+    withTable(s"$catalog.table") {
+      spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
+      runShowTablesSql(s"SHOW TABLES FROM $catalog", Seq(ShowRow("", "table", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table

Review comment:
       We can't do that. v1 catalog can't support 2 level namespace, even if the first part name is "spark_catalog". It's still the namespace, not a catalog name, in that context.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723961541






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724285064


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35414/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723928102


   **[Test build #130780 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130780/testReport)** for PR 30287 at commit [`b608f8b`](https://github.com/apache/spark/commit/b608f8b3c593373319eae5e68a4bb6f09c99d7dc).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724391004






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724295377






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519951946



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuite.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{QueryTest, Row}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+trait ShowTablesSuite extends QueryTest with SharedSparkSession {
+  protected def catalog: String
+  protected def defaultUsing: String
+  case class ShowRow(namespace: String, table: String, isTemporary: Boolean)
+  protected def getRows(showRows: Seq[ShowRow]): Seq[Row]
+  // Gets the schema of `SHOW TABLES`
+  protected def showSchema: StructType
+
+  protected def runShowTablesSql(sqlText: String, expected: Seq[ShowRow]): Unit = {
+    val df = spark.sql(sqlText)
+    assert(df.schema === showSchema)
+    assert(df.collect() === getRows(expected))
+  }
+
+  protected def withSourceViews(f: => Unit): Unit = {
+    withTable("source", "source2") {
+      val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
+      df.createOrReplaceTempView("source")
+      val df2 = spark.createDataFrame(Seq((4L, "d"), (5L, "e"), (6L, "f"))).toDF("id", "data")
+      df2.createOrReplaceTempView("source2")
+      f
+    }
+  }
+
+  test("show an existing table") {
+    val namespace = "test"
+    val table = "people"
+    withDatabase(s"$catalog.$namespace") {
+      sql(s"CREATE DATABASE $catalog.$namespace")
+      withTable(s"$catalog.$namespace.$table") {
+        sql(s"CREATE TABLE $catalog.$namespace.$table (name STRING, id INT) $defaultUsing")
+        runShowTablesSql(s"SHOW TABLES IN $catalog.test", Seq(ShowRow(namespace, table, false)))
+      }
+    }
+  }
+
+  test("show tables with a pattern") {
+    withDatabase(s"$catalog.db", s"$catalog.db2") {
+      sql(s"CREATE DATABASE $catalog.db")
+      sql(s"CREATE DATABASE $catalog.db2")
+      withTable(
+        s"$catalog.db.table",
+        s"$catalog.db.table_name_1",
+        s"$catalog.db.table_name_2",
+        s"$catalog.db2.table_name_2") {
+        sql(s"CREATE TABLE $catalog.db.table (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db.table_name_1 (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db.table_name_2 (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db2.table_name_2 (id bigint, data string) $defaultUsing")
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db",
+          Seq(
+            ShowRow("db", "table", false),
+            ShowRow("db", "table_name_1", false),
+            ShowRow("db", "table_name_2", false)))
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db LIKE '*name*'",
+          Seq(
+            ShowRow("db", "table_name_1", false),
+            ShowRow("db", "table_name_2", false)))
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db LIKE '*2'",
+          Seq(ShowRow("db", "table_name_2", false)))
+      }
+    }
+  }
+
+  // TODO(SPARK-33393): Support SHOW TABLE EXTENDED in DSv2
+  test("SHOW TABLE EXTENDED for default") {

Review comment:
       My point is that: the tests in the base trait, should work for both v1 and v2 catalogs, and will be run with both v1 and v2 catalog.
   
   The test now just run twice repeatedly in `v1.ShowTablesSuite` and `v2.ShowTablesSuite`, which doesn't make sense.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724626474






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724264458


   **[Test build #130804 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130804/testReport)** for PR 30287 at commit [`d264962`](https://github.com/apache/spark/commit/d264962f6d6c9134e07b6ef442ebb9cdd64ea1f1).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724626474


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r520102213



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v2
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
+import org.apache.spark.sql.connector.InMemoryTableCatalog
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{StringType, StructType}
+
+class ShowTablesSuite extends QueryTest with SharedSparkSession with CommonShowTablesSuite {
+  override def catalog: String = "test_catalog_v2"
+  override protected def defaultUsing: String = "USING _"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("namespace", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, _) => Row(namespace, table)
+    }
+  }
+
+  override def sparkConf: SparkConf = super.sparkConf
+    .set(s"spark.sql.catalog.$catalog", classOf[InMemoryTableCatalog].getName)
+
+  // The test fails with the exception `NoSuchDatabaseException` in V1 catalog.
+  // TODO(SPARK-33394): Throw `NoSuchDatabaseException` for not existing namespace
+  test("show table in a not existing namespace") {
+    runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.n1.n2.db
+  test("show tables in nested namespaces") {
+    withTable(s"$catalog.n1.n2.db") {
+      spark.sql(s"CREATE TABLE $catalog.n1.n2.db.table_name (id bigint, data string) $defaultUsing")
+      runShowTablesSql(
+        s"SHOW TABLES FROM $catalog.n1.n2.db",
+        Seq(ShowRow("n1.n2.db", "table_name", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table
+  test("using v2 catalog with empty namespace") {
+    withTable(s"$catalog.table") {
+      spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
+      runShowTablesSql(s"SHOW TABLES FROM $catalog", Seq(ShowRow("", "table", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table

Review comment:
       I split the test, and opened the JIRA: https://issues.apache.org/jira/browse/SPARK-33403




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724295357


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35414/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723480192


   @cloud-fan @HyukjinKwon Please, take a look at this PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519582006



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuite.scala
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{QueryTest, Row}
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
+import org.apache.spark.sql.catalyst.plans.logical.{ShowTables, ShowTableStatement}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+trait ShowTablesSuite extends QueryTest with SharedSparkSession with AnalysisTest {
+  protected def catalog: String

Review comment:
       A simpler way to test v1 vs v2 is: In v2 test, register a v2 catalog and set it as the default. Then in the test cases we just need to write `$namespace.$table`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #30287:
URL: https://github.com/apache/spark/pull/30287


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519933552



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command
+
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
+import org.apache.spark.sql.catalyst.plans.logical.{ShowTables, ShowTableStatement}
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ShowTablesParserSuite extends AnalysisTest with SharedSparkSession {
+  private val catalog = "test_catalog"
+
+  test("show tables") {
+    comparePlans(
+      parsePlan("SHOW TABLES"),
+      ShowTables(UnresolvedNamespace(Seq.empty[String]), None))
+    comparePlans(
+      parsePlan("SHOW TABLES '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq.empty[String]), Some("*test*")))
+    comparePlans(
+      parsePlan("SHOW TABLES LIKE '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq.empty[String]), Some("*test*")))
+    comparePlans(
+      parsePlan(s"SHOW TABLES FROM $catalog.ns1.ns2.tbl"),
+      ShowTables(UnresolvedNamespace(Seq(catalog, "ns1", "ns2", "tbl")), None))
+    comparePlans(
+      parsePlan(s"SHOW TABLES IN $catalog.ns1.ns2.tbl"),
+      ShowTables(UnresolvedNamespace(Seq(catalog, "ns1", "ns2", "tbl")), None))
+    comparePlans(
+      parsePlan("SHOW TABLES IN ns1 '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq("ns1")), Some("*test*")))
+    comparePlans(
+      parsePlan("SHOW TABLES IN ns1 LIKE '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq("ns1")), Some("*test*")))
+  }
+
+  test("show table extended") {
+    comparePlans(
+      parsePlan("SHOW TABLE EXTENDED LIKE '*test*'"),
+      ShowTableStatement(None, "*test*", None))

Review comment:
       We parse `SHOW TABLE EXTENDED`:
   https://github.com/apache/spark/blob/ccc0250a08fa9519a6dc3c1153e51c1e110f1d7d/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4#L211-L212
   and convert it to `ShowTableStatement `:
   https://github.com/apache/spark/blob/0ad35ba5f8bd6413669b568de659334bb9a3fb44/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala#L2929-L2934




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724727863


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35477/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723506197


   **[Test build #130748 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130748/testReport)** for PR 30287 at commit [`947c8bc`](https://github.com/apache/spark/commit/947c8bc847b2eed260199f153fd8cd829d4a8fc3).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723961541






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723486616






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724563939


   **[Test build #130855 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130855/testReport)** for PR 30287 at commit [`6286152`](https://github.com/apache/spark/commit/6286152eb9aa4a253864592b815e94e3c88970fb).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r520101711



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v2
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
+import org.apache.spark.sql.connector.InMemoryTableCatalog
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{StringType, StructType}
+
+class ShowTablesSuite extends QueryTest with SharedSparkSession with CommonShowTablesSuite {
+  override def catalog: String = "test_catalog_v2"
+  override protected def defaultUsing: String = "USING _"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("namespace", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, _) => Row(namespace, table)
+    }
+  }
+
+  override def sparkConf: SparkConf = super.sparkConf
+    .set(s"spark.sql.catalog.$catalog", classOf[InMemoryTableCatalog].getName)
+
+  // The test fails with the exception `NoSuchDatabaseException` in V1 catalog.
+  // TODO(SPARK-33394): Throw `NoSuchDatabaseException` for not existing namespace
+  test("show table in a not existing namespace") {
+    runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.n1.n2.db
+  test("show tables in nested namespaces") {
+    withTable(s"$catalog.n1.n2.db") {
+      spark.sql(s"CREATE TABLE $catalog.n1.n2.db.table_name (id bigint, data string) $defaultUsing")
+      runShowTablesSql(
+        s"SHOW TABLES FROM $catalog.n1.n2.db",
+        Seq(ShowRow("n1.n2.db", "table_name", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table
+  test("using v2 catalog with empty namespace") {
+    withTable(s"$catalog.table") {
+      spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
+      runShowTablesSql(s"SHOW TABLES FROM $catalog", Seq(ShowRow("", "table", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table
+  test("namespace is not specified and default v2 catalog is set") {
+    withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) {
+      withTable(s"$catalog.table") {
+        spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
+        // v2 catalog is used where default namespace is empty for TestInMemoryTableCatalog.
+        runShowTablesSql("SHOW TABLES", Seq(ShowRow("", "table", false)))
+      }
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.ns1.ns2.tbl
+  test("SHOW TABLE EXTENDED not valid v1 database") {
+    def testV1CommandNamespace(sqlCommand: String, namespace: String): Unit = {
+      val e = intercept[AnalysisException] {
+        sql(sqlCommand)
+      }
+      assert(e.message.contains(s"The database name is not valid: ${namespace}"))
+    }
+
+    val namespace = s"$catalog.ns1.ns2"
+    val table = "tbl"
+    withTable(s"$namespace.$table") {
+      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
+        s"$defaultUsing PARTITIONED BY (id)")
+
+      testV1CommandNamespace(s"SHOW TABLE EXTENDED FROM $namespace LIKE 'tb*'",
+        namespace)
+      testV1CommandNamespace(s"SHOW TABLE EXTENDED IN $namespace LIKE 'tb*'",
+        namespace)
+      testV1CommandNamespace("SHOW TABLE EXTENDED " +
+        s"FROM $namespace LIKE 'tb*' PARTITION(id=1)",
+        namespace)
+      testV1CommandNamespace("SHOW TABLE EXTENDED " +
+        s"IN $namespace LIKE 'tb*' PARTITION(id=1)",
+        namespace)
+    }
+  }
+
+  // The test fails with the error in V1 session catalog:

Review comment:
       What do you propose here? The original test has nested namespaces, BTW.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724385816






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724563939


   **[Test build #130855 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130855/testReport)** for PR 30287 at commit [`6286152`](https://github.com/apache/spark/commit/6286152eb9aa4a253864592b815e94e3c88970fb).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519888921



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v1
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+class ShowTablesSuite extends CommonShowTablesSuite {
+  override def catalog: String = CatalogManager.SESSION_CATALOG_NAME
+  override protected def defaultUsing: String = "USING parquet"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("database", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+      .add("isTemporary", BooleanType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, isTemporary) => Row(namespace, table, isTemporary)
+    }
+  }
+
+  // `SHOW TABLES` returns empty result in V2 catalog instead of throwing the exception.
+  test("show table in a not existing namespace") {
+    val msg = intercept[NoSuchDatabaseException] {
+      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+    }.getMessage
+    assert(msg.contains("Database 'unknown' not found"))
+  }
+
+  // `SHOW TABLES` from v2 catalog returns empty result.

Review comment:
       How is this related to v2? This test checks the default catalog, which by default is v1.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724874556


   **[Test build #130870 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130870/testReport)** for PR 30287 at commit [`8108cb1`](https://github.com/apache/spark/commit/8108cb1b5879b7873e3bfa43058ec3db7feba148).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519886690



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuite.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{QueryTest, Row}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+trait ShowTablesSuite extends QueryTest with SharedSparkSession {
+  protected def catalog: String
+  protected def defaultUsing: String
+  case class ShowRow(namespace: String, table: String, isTemporary: Boolean)
+  protected def getRows(showRows: Seq[ShowRow]): Seq[Row]
+  // Gets the schema of `SHOW TABLES`
+  protected def showSchema: StructType
+
+  protected def runShowTablesSql(sqlText: String, expected: Seq[ShowRow]): Unit = {
+    val df = spark.sql(sqlText)
+    assert(df.schema === showSchema)
+    assert(df.collect() === getRows(expected))
+  }
+
+  protected def withSourceViews(f: => Unit): Unit = {
+    withTable("source", "source2") {
+      val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
+      df.createOrReplaceTempView("source")
+      val df2 = spark.createDataFrame(Seq((4L, "d"), (5L, "e"), (6L, "f"))).toDF("id", "data")
+      df2.createOrReplaceTempView("source2")
+      f
+    }
+  }
+
+  test("show an existing table") {
+    val namespace = "test"
+    val table = "people"
+    withDatabase(s"$catalog.$namespace") {
+      sql(s"CREATE DATABASE $catalog.$namespace")
+      withTable(s"$catalog.$namespace.$table") {
+        sql(s"CREATE TABLE $catalog.$namespace.$table (name STRING, id INT) $defaultUsing")
+        runShowTablesSql(s"SHOW TABLES IN $catalog.test", Seq(ShowRow(namespace, table, false)))
+      }
+    }
+  }
+
+  test("show tables with a pattern") {
+    withDatabase(s"$catalog.db", s"$catalog.db2") {
+      sql(s"CREATE DATABASE $catalog.db")
+      sql(s"CREATE DATABASE $catalog.db2")
+      withTable(
+        s"$catalog.db.table",
+        s"$catalog.db.table_name_1",
+        s"$catalog.db.table_name_2",
+        s"$catalog.db2.table_name_2") {
+        sql(s"CREATE TABLE $catalog.db.table (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db.table_name_1 (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db.table_name_2 (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db2.table_name_2 (id bigint, data string) $defaultUsing")
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db",
+          Seq(
+            ShowRow("db", "table", false),
+            ShowRow("db", "table_name_1", false),
+            ShowRow("db", "table_name_2", false)))
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db LIKE '*name*'",
+          Seq(
+            ShowRow("db", "table_name_1", false),
+            ShowRow("db", "table_name_2", false)))
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db LIKE '*2'",
+          Seq(ShowRow("db", "table_name_2", false)))
+      }
+    }
+  }
+
+  // TODO(SPARK-33393): Support SHOW TABLE EXTENDED in DSv2
+  test("SHOW TABLE EXTENDED for default") {

Review comment:
       this test assumes that the default catalog is v1 catalog. Shall we move it to `v1.ShowTablesSuite`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519953488



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v1
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+class ShowTablesSuite extends CommonShowTablesSuite {
+  override def catalog: String = CatalogManager.SESSION_CATALOG_NAME
+  override protected def defaultUsing: String = "USING parquet"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("database", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+      .add("isTemporary", BooleanType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, isTemporary) => Row(namespace, table, isTemporary)
+    }
+  }
+
+  // `SHOW TABLES` returns empty result in V2 catalog instead of throwing the exception.
+  test("show table in a not existing namespace") {
+    val msg = intercept[NoSuchDatabaseException] {
+      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+    }.getMessage
+    assert(msg.contains("Database 'unknown' not found"))
+  }
+
+  // `SHOW TABLES` from v2 catalog returns empty result.

Review comment:
       The confusion is: temp views don't belong to catalog, but belong to the spark session. Can we create permanent views in this test? Or change the test name to `v1  SHOW TABLES` list the temp views.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724626481


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35463/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723506423






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519953488



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v1
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+class ShowTablesSuite extends CommonShowTablesSuite {
+  override def catalog: String = CatalogManager.SESSION_CATALOG_NAME
+  override protected def defaultUsing: String = "USING parquet"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("database", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+      .add("isTemporary", BooleanType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, isTemporary) => Row(namespace, table, isTemporary)
+    }
+  }
+
+  // `SHOW TABLES` returns empty result in V2 catalog instead of throwing the exception.
+  test("show table in a not existing namespace") {
+    val msg = intercept[NoSuchDatabaseException] {
+      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+    }.getMessage
+    assert(msg.contains("Database 'unknown' not found"))
+  }
+
+  // `SHOW TABLES` from v2 catalog returns empty result.

Review comment:
       The confusion is: temp views don't belong to catalog, but belong to the spark session. Can we create permanent views in this test? Or change the test name to "v1  SHOW TABLES list the temp views".




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724303537






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724071669






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724268112


   **[Test build #130805 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130805/testReport)** for PR 30287 at commit [`286e11a`](https://github.com/apache/spark/commit/286e11ad657a74b1b69f7caac6a1bfcdeb3a65fd).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723928102


   **[Test build #130780 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130780/testReport)** for PR 30287 at commit [`b608f8b`](https://github.com/apache/spark/commit/b608f8b3c593373319eae5e68a4bb6f09c99d7dc).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519564343



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v1
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+class ShowTablesSuite extends CommonShowTablesSuite {
+  override def catalog: String = "spark_catalog"

Review comment:
       let's not hardcode it. `CatalogManager.SESSION_CATALOG_NAME`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519569708



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuite.scala
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{QueryTest, Row}
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
+import org.apache.spark.sql.catalyst.plans.logical.{ShowTables, ShowTableStatement}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+trait ShowTablesSuite extends QueryTest with SharedSparkSession with AnalysisTest {
+  protected def catalog: String
+  protected def defaultUsing: String
+  case class ShowRow(namespace: String, table: String, isTemporary: Boolean)
+  protected def getRows(showRows: Seq[ShowRow]): Seq[Row]
+  // Gets the schema of `SHOW TABLES`
+  protected def showSchema: StructType
+
+  protected def runShowTablesSql(sqlText: String, expected: Seq[ShowRow]): Unit = {
+    val df = spark.sql(sqlText)
+    assert(df.schema === showSchema)
+    assert(df.collect() === getRows(expected))
+  }
+
+  protected def withSourceViews(f: => Unit): Unit = {
+    withTable("source", "source2") {
+      val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
+      df.createOrReplaceTempView("source")
+      val df2 = spark.createDataFrame(Seq((4L, "d"), (5L, "e"), (6L, "f"))).toDF("id", "data")
+      df2.createOrReplaceTempView("source2")
+      f
+    }
+  }
+
+  test("show tables") {

Review comment:
       Do you mean in a separate file or a class in the same file?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519941901



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v1
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+class ShowTablesSuite extends CommonShowTablesSuite {
+  override def catalog: String = CatalogManager.SESSION_CATALOG_NAME
+  override protected def defaultUsing: String = "USING parquet"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("database", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+      .add("isTemporary", BooleanType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, isTemporary) => Row(namespace, table, isTemporary)
+    }
+  }
+
+  // `SHOW TABLES` returns empty result in V2 catalog instead of throwing the exception.
+  test("show table in a not existing namespace") {
+    val msg = intercept[NoSuchDatabaseException] {
+      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+    }.getMessage
+    assert(msg.contains("Database 'unknown' not found"))
+  }
+
+  // `SHOW TABLES` from v2 catalog returns empty result.

Review comment:
       I described the reason why the test is placed to V1 specific test suite - because V2 doesn't show views otherwise we could move it to the common trait and set `default` to a V2 catalog.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724264458


   **[Test build #130804 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130804/testReport)** for PR 30287 at commit [`d264962`](https://github.com/apache/spark/commit/d264962f6d6c9134e07b6ef442ebb9cdd64ea1f1).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724389927


   **[Test build #130804 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130804/testReport)** for PR 30287 at commit [`d264962`](https://github.com/apache/spark/commit/d264962f6d6c9134e07b6ef442ebb9cdd64ea1f1).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724626448


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35463/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724714682


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35477/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724728756






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r520254619



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v2
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+import org.apache.spark.sql.connector.InMemoryTableCatalog
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{StringType, StructType}
+
+class ShowTablesSuite extends QueryTest with SharedSparkSession with CommonShowTablesSuite {
+  override def catalog: String = "test_catalog_v2"
+  override protected def defaultUsing: String = "USING _"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("namespace", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, _) => Row(namespace, table)
+    }
+  }
+
+  override def sparkConf: SparkConf = super.sparkConf
+    .set(s"spark.sql.catalog.$catalog", classOf[InMemoryTableCatalog].getName)
+
+  // The test fails with the exception `NoSuchDatabaseException` in V1 catalog.
+  // TODO(SPARK-33394): Throw `NoSuchDatabaseException` for not existing namespace
+  test("show table in a not existing namespace") {
+    runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.n1.n2.db
+  test("show tables in nested namespaces") {
+    withTable(s"$catalog.n1.n2.db") {
+      spark.sql(s"CREATE TABLE $catalog.n1.n2.db.table_name (id bigint, data string) $defaultUsing")
+      runShowTablesSql(
+        s"SHOW TABLES FROM $catalog.n1.n2.db",
+        Seq(ShowRow("n1.n2.db", "table_name", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table
+  test("using v2 catalog with empty namespace") {
+    withTable(s"$catalog.table") {
+      spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
+      runShowTablesSql(s"SHOW TABLES FROM $catalog", Seq(ShowRow("", "table", false)))
+    }
+  }
+
+  test("namespace is not specified and the default catalog is set") {
+    withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) {
+      withTable("table") {
+        spark.sql(s"CREATE TABLE table (id bigint, data string) $defaultUsing")
+        // TODO(SPARK-33403): DSv2 SHOW TABLES doesn't show `default`

Review comment:
       This doesn't need TODO as it's a corrected behavior.
   
   `CREATE TABLE table` creates a table in the current catalog and current namespace. The current namespace, if hasn't been set by users via USE yet, is decided by the current catalog. For the v1 catalog, the default current  namespace is `["default"]`. For the testing v2 catalog, it's `[]`.
   
   I think a better way to write this test is:
   ```
   spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
   runShowTablesSql("SHOW TABLES", Seq(ShowRow("", "table", false)))
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724385816






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519698787



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala
##########
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v1
+
+import org.apache.spark.sql.{AnalysisException, Row}
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+class ShowTablesSuite extends CommonShowTablesSuite {
+  override def catalog: String = "spark_catalog"
+  override protected def defaultUsing: String = "USING parquet"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("database", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+      .add("isTemporary", BooleanType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, isTemporary) => Row(namespace, table, isTemporary)
+    }
+  }
+
+  // `SHOW TABLES` returns empty result in V2 catalog instead of throwing the exception.

Review comment:
       Here is the umbrella ticket for TODOs: https://issues.apache.org/jira/browse/SPARK-33392




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519890466



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v2
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
+import org.apache.spark.sql.connector.InMemoryTableCatalog
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{StringType, StructType}
+
+class ShowTablesSuite extends QueryTest with SharedSparkSession with CommonShowTablesSuite {
+  override def catalog: String = "test_catalog_v2"
+  override protected def defaultUsing: String = "USING _"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("namespace", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, _) => Row(namespace, table)
+    }
+  }
+
+  override def sparkConf: SparkConf = super.sparkConf
+    .set(s"spark.sql.catalog.$catalog", classOf[InMemoryTableCatalog].getName)
+
+  // The test fails with the exception `NoSuchDatabaseException` in V1 catalog.
+  // TODO(SPARK-33394): Throw `NoSuchDatabaseException` for not existing namespace
+  test("show table in a not existing namespace") {
+    runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.n1.n2.db
+  test("show tables in nested namespaces") {
+    withTable(s"$catalog.n1.n2.db") {
+      spark.sql(s"CREATE TABLE $catalog.n1.n2.db.table_name (id bigint, data string) $defaultUsing")
+      runShowTablesSql(
+        s"SHOW TABLES FROM $catalog.n1.n2.db",
+        Seq(ShowRow("n1.n2.db", "table_name", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table
+  test("using v2 catalog with empty namespace") {
+    withTable(s"$catalog.table") {
+      spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
+      runShowTablesSql(s"SHOW TABLES FROM $catalog", Seq(ShowRow("", "table", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table

Review comment:
       If we want to focus on the default catalog, let's use one-level namespace to have a fair comparison




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724293531


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35415/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519883829



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command
+
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
+import org.apache.spark.sql.catalyst.plans.logical.{ShowTables, ShowTableStatement}
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ShowTablesParserSuite extends AnalysisTest with SharedSparkSession {
+  private val catalog = "test_catalog"
+
+  test("show tables") {
+    comparePlans(
+      parsePlan("SHOW TABLES"),
+      ShowTables(UnresolvedNamespace(Seq.empty[String]), None))
+    comparePlans(
+      parsePlan("SHOW TABLES '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq.empty[String]), Some("*test*")))
+    comparePlans(
+      parsePlan("SHOW TABLES LIKE '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq.empty[String]), Some("*test*")))
+    comparePlans(
+      parsePlan(s"SHOW TABLES FROM $catalog.ns1.ns2.tbl"),
+      ShowTables(UnresolvedNamespace(Seq(catalog, "ns1", "ns2", "tbl")), None))
+    comparePlans(
+      parsePlan(s"SHOW TABLES IN $catalog.ns1.ns2.tbl"),
+      ShowTables(UnresolvedNamespace(Seq(catalog, "ns1", "ns2", "tbl")), None))
+    comparePlans(
+      parsePlan("SHOW TABLES IN ns1 '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq("ns1")), Some("*test*")))
+    comparePlans(
+      parsePlan("SHOW TABLES IN ns1 LIKE '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq("ns1")), Some("*test*")))
+  }
+
+  test("show table extended") {
+    comparePlans(
+      parsePlan("SHOW TABLE EXTENDED LIKE '*test*'"),
+      ShowTableStatement(None, "*test*", None))

Review comment:
       @imback82 do you know why we still have `ShowTableStatement` here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724303537






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724303516


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35415/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723486616






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519616744



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuite.scala
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{QueryTest, Row}
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
+import org.apache.spark.sql.catalyst.plans.logical.{ShowTables, ShowTableStatement}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+trait ShowTablesSuite extends QueryTest with SharedSparkSession with AnalysisTest {
+  protected def catalog: String

Review comment:
       OK makes sense.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r520052856



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command
+
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
+import org.apache.spark.sql.catalyst.plans.logical.{ShowTables, ShowTableStatement}
+import org.apache.spark.sql.test.SharedSparkSession
+
+class ShowTablesParserSuite extends AnalysisTest with SharedSparkSession {
+  private val catalog = "test_catalog"
+
+  test("show tables") {
+    comparePlans(
+      parsePlan("SHOW TABLES"),
+      ShowTables(UnresolvedNamespace(Seq.empty[String]), None))
+    comparePlans(
+      parsePlan("SHOW TABLES '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq.empty[String]), Some("*test*")))
+    comparePlans(
+      parsePlan("SHOW TABLES LIKE '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq.empty[String]), Some("*test*")))
+    comparePlans(
+      parsePlan(s"SHOW TABLES FROM $catalog.ns1.ns2.tbl"),
+      ShowTables(UnresolvedNamespace(Seq(catalog, "ns1", "ns2", "tbl")), None))
+    comparePlans(
+      parsePlan(s"SHOW TABLES IN $catalog.ns1.ns2.tbl"),
+      ShowTables(UnresolvedNamespace(Seq(catalog, "ns1", "ns2", "tbl")), None))
+    comparePlans(
+      parsePlan("SHOW TABLES IN ns1 '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq("ns1")), Some("*test*")))
+    comparePlans(
+      parsePlan("SHOW TABLES IN ns1 LIKE '*test*'"),
+      ShowTables(UnresolvedNamespace(Seq("ns1")), Some("*test*")))
+  }
+
+  test("show table extended") {
+    comparePlans(
+      parsePlan("SHOW TABLE EXTENDED LIKE '*test*'"),
+      ShowTableStatement(None, "*test*", None))

Review comment:
       @cloud-fan `ShowTable` is not migrated yet; only `ShowTables` with "s" is migrated. I will work on this migration soon.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-725207545


   thanks, merging to master!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724384980


   **[Test build #130805 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130805/testReport)** for PR 30287 at commit [`286e11a`](https://github.com/apache/spark/commit/286e11ad657a74b1b69f7caac6a1bfcdeb3a65fd).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723484571


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35357/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724727280


   **[Test build #130855 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130855/testReport)** for PR 30287 at commit [`6286152`](https://github.com/apache/spark/commit/6286152eb9aa4a253864592b815e94e3c88970fb).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519899707



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.v2
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
+import org.apache.spark.sql.connector.InMemoryTableCatalog
+import org.apache.spark.sql.execution.command.{ShowTablesSuite => CommonShowTablesSuite}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{StringType, StructType}
+
+class ShowTablesSuite extends QueryTest with SharedSparkSession with CommonShowTablesSuite {
+  override def catalog: String = "test_catalog_v2"
+  override protected def defaultUsing: String = "USING _"
+  override protected def showSchema: StructType = {
+    new StructType()
+      .add("namespace", StringType, nullable = false)
+      .add("tableName", StringType, nullable = false)
+  }
+  override protected def getRows(showRows: Seq[ShowRow]): Seq[Row] = {
+    showRows.map {
+      case ShowRow(namespace, table, _) => Row(namespace, table)
+    }
+  }
+
+  override def sparkConf: SparkConf = super.sparkConf
+    .set(s"spark.sql.catalog.$catalog", classOf[InMemoryTableCatalog].getName)
+
+  // The test fails with the exception `NoSuchDatabaseException` in V1 catalog.
+  // TODO(SPARK-33394): Throw `NoSuchDatabaseException` for not existing namespace
+  test("show table in a not existing namespace") {
+    runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.n1.n2.db
+  test("show tables in nested namespaces") {
+    withTable(s"$catalog.n1.n2.db") {
+      spark.sql(s"CREATE TABLE $catalog.n1.n2.db.table_name (id bigint, data string) $defaultUsing")
+      runShowTablesSql(
+        s"SHOW TABLES FROM $catalog.n1.n2.db",
+        Seq(ShowRow("n1.n2.db", "table_name", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table
+  test("using v2 catalog with empty namespace") {
+    withTable(s"$catalog.table") {
+      spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
+      runShowTablesSql(s"SHOW TABLES FROM $catalog", Seq(ShowRow("", "table", false)))
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.table
+  test("namespace is not specified and default v2 catalog is set") {
+    withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) {
+      withTable(s"$catalog.table") {
+        spark.sql(s"CREATE TABLE $catalog.table (id bigint, data string) $defaultUsing")
+        // v2 catalog is used where default namespace is empty for TestInMemoryTableCatalog.
+        runShowTablesSql("SHOW TABLES", Seq(ShowRow("", "table", false)))
+      }
+    }
+  }
+
+  // The test fails for V1 catalog with the error:
+  // org.apache.spark.sql.AnalysisException:
+  //   The namespace in session catalog must have exactly one name part: spark_catalog.ns1.ns2.tbl
+  test("SHOW TABLE EXTENDED not valid v1 database") {
+    def testV1CommandNamespace(sqlCommand: String, namespace: String): Unit = {
+      val e = intercept[AnalysisException] {
+        sql(sqlCommand)
+      }
+      assert(e.message.contains(s"The database name is not valid: ${namespace}"))
+    }
+
+    val namespace = s"$catalog.ns1.ns2"
+    val table = "tbl"
+    withTable(s"$namespace.$table") {
+      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
+        s"$defaultUsing PARTITIONED BY (id)")
+
+      testV1CommandNamespace(s"SHOW TABLE EXTENDED FROM $namespace LIKE 'tb*'",
+        namespace)
+      testV1CommandNamespace(s"SHOW TABLE EXTENDED IN $namespace LIKE 'tb*'",
+        namespace)
+      testV1CommandNamespace("SHOW TABLE EXTENDED " +
+        s"FROM $namespace LIKE 'tb*' PARTITION(id=1)",
+        namespace)
+      testV1CommandNamespace("SHOW TABLE EXTENDED " +
+        s"IN $namespace LIKE 'tb*' PARTITION(id=1)",
+        namespace)
+    }
+  }
+
+  // The test fails with the error in V1 session catalog:

Review comment:
       Again, to have a fail comparison, let's use one leve namespace here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724268112


   **[Test build #130805 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130805/testReport)** for PR 30287 at commit [`286e11a`](https://github.com/apache/spark/commit/286e11ad657a74b1b69f7caac6a1bfcdeb3a65fd).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r519612119



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuite.scala
##########
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{QueryTest, Row}
+import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan
+import org.apache.spark.sql.catalyst.plans.logical.{ShowTables, ShowTableStatement}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+trait ShowTablesSuite extends QueryTest with SharedSparkSession with AnalysisTest {
+  protected def catalog: String

Review comment:
       yeh, I thought about this but wanted to avoid dependency from a global variable. Also it will make harder to write tests that:
   - check `USE ...`
   - check both dsv1 and dsv2 in the same test
   
   And with explicit catalog, it should be easier to mix the common trait to other test suite like JDBC or Hive catalogs.
    




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a change in pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #30287:
URL: https://github.com/apache/spark/pull/30287#discussion_r520084037



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuite.scala
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.{QueryTest, Row}
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{BooleanType, StringType, StructType}
+
+trait ShowTablesSuite extends QueryTest with SharedSparkSession {
+  protected def catalog: String
+  protected def defaultUsing: String
+  case class ShowRow(namespace: String, table: String, isTemporary: Boolean)
+  protected def getRows(showRows: Seq[ShowRow]): Seq[Row]
+  // Gets the schema of `SHOW TABLES`
+  protected def showSchema: StructType
+
+  protected def runShowTablesSql(sqlText: String, expected: Seq[ShowRow]): Unit = {
+    val df = spark.sql(sqlText)
+    assert(df.schema === showSchema)
+    assert(df.collect() === getRows(expected))
+  }
+
+  protected def withSourceViews(f: => Unit): Unit = {
+    withTable("source", "source2") {
+      val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data")
+      df.createOrReplaceTempView("source")
+      val df2 = spark.createDataFrame(Seq((4L, "d"), (5L, "e"), (6L, "f"))).toDF("id", "data")
+      df2.createOrReplaceTempView("source2")
+      f
+    }
+  }
+
+  test("show an existing table") {
+    val namespace = "test"
+    val table = "people"
+    withDatabase(s"$catalog.$namespace") {
+      sql(s"CREATE DATABASE $catalog.$namespace")
+      withTable(s"$catalog.$namespace.$table") {
+        sql(s"CREATE TABLE $catalog.$namespace.$table (name STRING, id INT) $defaultUsing")
+        runShowTablesSql(s"SHOW TABLES IN $catalog.test", Seq(ShowRow(namespace, table, false)))
+      }
+    }
+  }
+
+  test("show tables with a pattern") {
+    withDatabase(s"$catalog.db", s"$catalog.db2") {
+      sql(s"CREATE DATABASE $catalog.db")
+      sql(s"CREATE DATABASE $catalog.db2")
+      withTable(
+        s"$catalog.db.table",
+        s"$catalog.db.table_name_1",
+        s"$catalog.db.table_name_2",
+        s"$catalog.db2.table_name_2") {
+        sql(s"CREATE TABLE $catalog.db.table (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db.table_name_1 (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db.table_name_2 (id bigint, data string) $defaultUsing")
+        sql(s"CREATE TABLE $catalog.db2.table_name_2 (id bigint, data string) $defaultUsing")
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db",
+          Seq(
+            ShowRow("db", "table", false),
+            ShowRow("db", "table_name_1", false),
+            ShowRow("db", "table_name_2", false)))
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db LIKE '*name*'",
+          Seq(
+            ShowRow("db", "table_name_1", false),
+            ShowRow("db", "table_name_2", false)))
+
+        runShowTablesSql(
+          s"SHOW TABLES FROM $catalog.db LIKE '*2'",
+          Seq(ShowRow("db", "table_name_2", false)))
+      }
+    }
+  }
+
+  // TODO(SPARK-33393): Support SHOW TABLE EXTENDED in DSv2
+  test("SHOW TABLE EXTENDED for default") {

Review comment:
       Create 2 separate tests for dsv1 and dsv2




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724391004






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-724295377






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30287: [SPARK-33382][SQL][TESTS] Unify datasource v1 and v2 SHOW TABLES tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30287:
URL: https://github.com/apache/spark/pull/30287#issuecomment-723480390


   **[Test build #130748 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130748/testReport)** for PR 30287 at commit [`947c8bc`](https://github.com/apache/spark/commit/947c8bc847b2eed260199f153fd8cd829d4a8fc3).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org