Skip to content

[SPARK-14348][SQL] Support native execution of SHOW TBLPROPERTIES command #12133

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 5 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,8 @@ statement
| SHOW TABLES ((FROM | IN) db=identifier)?
(LIKE? pattern=STRING)? #showTables
| SHOW DATABASES (LIKE pattern=STRING)? #showDatabases
| SHOW TBLPROPERTIES table=tableIdentifier
('(' key=tablePropertyKey ')')? #showTblProperties
| SHOW FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))? #showFunctions
| (DESC | DESCRIBE) FUNCTION EXTENDED? qualifiedName #describeFunction
| (DESC | DESCRIBE) option=(EXTENDED | FORMATTED)?
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,7 @@ class SessionCatalog(
/**
* Retrieve the metadata of an existing metastore table.
* If no database is specified, assume the table is in the current database.
* If the specified table is not found in the database then an [[AnalysisException]] is thrown.
*/
def getTable(name: TableIdentifier): CatalogTable = {
val db = name.database.getOrElse(currentDb)
Expand Down Expand Up @@ -271,6 +272,16 @@ class SessionCatalog(
}
}

/**
* Return whether a table with the specified name is a temporary table.
*
* Note: The temporary table cache is checked only when database is not
* explicitly specified.
*/
def isTemporaryTable(name: TableIdentifier): Boolean = {
!name.database.isDefined && tempTables.contains(formatTableName(name.table))
}

/**
* List all tables in the specified database, including temporary tables.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,22 @@ class SparkSqlAstBuilder extends AstBuilder {
ShowDatabasesCommand(Option(ctx.pattern).map(string))
}

/**
* A command for users to list the properties for a table. If propertyKey is specified, the value
* for the propertyKey is returned. If propertyKey is not specified, all the keys and their
* corresponding values are returned.
* The syntax of using this command in SQL is:
* {{{
* SHOW TBLPROPERTIES table_name[('propertyKey')];
* }}}
*/
override def visitShowTblProperties(
ctx: ShowTblPropertiesContext): LogicalPlan = withOrigin(ctx) {
ShowTablePropertiesCommand(
visitTableIdentifier(ctx.tableIdentifier),
Option(ctx.key).map(visitTablePropertyKey))
}

/**
* Create a [[RefreshTable]] logical plan.
*/
Expand Down Expand Up @@ -219,18 +235,25 @@ class SparkSqlAstBuilder extends AstBuilder {
override def visitTablePropertyList(
ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) {
ctx.tableProperty.asScala.map { property =>
// A key can either be a String or a collection of dot separated elements. We need to treat
// these differently.
val key = if (property.key.STRING != null) {
string(property.key.STRING)
} else {
property.key.getText
}
val key = visitTablePropertyKey(property.key)
val value = Option(property.value).map(string).orNull
key -> value
}.toMap
}

/**
* A table property key can either be String or a collection of dot separated elements. This
* function extracts the property key based on whether its a string literal or a table property
* identifier.
*/
override def visitTablePropertyKey(key: TablePropertyKeyContext): String = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you add a little bit of documentation for this method. The comment on line 239 is a good starting point.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure. Will do,

if (key.STRING != null) {
string(key.STRING)
} else {
key.getText
}
}

/**
* Create a [[CreateDatabase]] command.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.util.NoSuchElementException

import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Dataset, Row, SQLContext}
import org.apache.spark.sql.{AnalysisException, Dataset, Row, SQLContext}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
Expand Down Expand Up @@ -373,6 +373,48 @@ case class ShowDatabasesCommand(databasePattern: Option[String]) extends Runnabl
}
}

/**
* A command for users to list the properties for a table If propertyKey is specified, the value
* for the propertyKey is returned. If propertyKey is not specified, all the keys and their
* corresponding values are returned.
* The syntax of using this command in SQL is:
* {{{
* SHOW TBLPROPERTIES table_name[('propertyKey')];
* }}}
*/
case class ShowTablePropertiesCommand(
table: TableIdentifier,
propertyKey: Option[String]) extends RunnableCommand {

override val output: Seq[Attribute] = {
val schema = AttributeReference("value", StringType, nullable = false)() :: Nil
propertyKey match {
case None => AttributeReference("key", StringType, nullable = false)() :: schema
case _ => schema
}
}

override def run(sqlContext: SQLContext): Seq[Row] = {
val catalog = sqlContext.sessionState.catalog

if (catalog.isTemporaryTable(table)) {
Seq.empty[Row]
} else {
val catalogTable = sqlContext.sessionState.catalog.getTable(table)

propertyKey match {
case Some(p) =>
val propValue = catalogTable
.properties
.getOrElse(p, s"Table ${catalogTable.qualifiedName} does not have property: $p")
Seq(Row(propValue))
case None =>
catalogTable.properties.map(p => Row(p._1, p._2)).toSeq
}
}
}
}

/**
* A command for users to list all of the registered functions.
* The syntax of using this command in SQL is:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -773,4 +773,12 @@ class DDLCommandSuite extends PlanTest {
comparePlans(parsed2, expected2)
}

test("show tblproperties") {
val parsed1 = parser.parsePlan("SHOW TBLPROPERTIES tab1")
val expected1 = ShowTablePropertiesCommand(TableIdentifier("tab1", None), None)
val parsed2 = parser.parsePlan("SHOW TBLPROPERTIES tab1('propKey1')")
val expected2 = ShowTablePropertiesCommand(TableIdentifier("tab1", None), Some("propKey1"))
comparePlans(parsed1, expected1)
comparePlans(parsed2, expected2)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,125 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.hive.execution

import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.test.SQLTestUtils

class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't we test most of this stuff in sql/core or is this hard to do because of the current lack of CREATE TABLE support?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah... I actually started writing the tests in sql/core. But ran into errors which is why i moved it back to hive.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just curious, what kind of errors?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

DROP table did not work as an example :-).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah you are totally right about that.

protected override def beforeAll(): Unit = {
super.beforeAll()
sql(
"""
|CREATE EXTERNAL TABLE parquet_tab1 (c1 INT, c2 STRING)
|USING org.apache.spark.sql.parquet.DefaultSource
""".stripMargin)

sql(
"""
|CREATE EXTERNAL TABLE parquet_tab2 (c1 INT, c2 STRING)
|STORED AS PARQUET
|TBLPROPERTIES('prop1Key'="prop1Val", '`prop2Key`'="prop2Val")
""".stripMargin)
}

override protected def afterAll(): Unit = {
try {
sql("DROP TABLE IF EXISTS parquet_tab1")
sql("DROP TABLE IF EXISTS parquet_tab2")
} finally {
super.afterAll()
}
}

test("show tables") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this test here? You are testing SHOW TABLES commands, instead of SHOW TBLPROPERTIES commands.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nevermind - I got it.

withTable("show1a", "show2b") {
sql("CREATE TABLE show1a(c1 int)")
sql("CREATE TABLE show2b(c2 int)")
checkAnswer(
sql("SHOW TABLES IN default 'show1*'"),
Row("show1a", false) :: Nil)
checkAnswer(
sql("SHOW TABLES IN default 'show1*|show2*'"),
Row("show1a", false) ::
Row("show2b", false) :: Nil)
checkAnswer(
sql("SHOW TABLES 'show1*|show2*'"),
Row("show1a", false) ::
Row("show2b", false) :: Nil)
assert(
sql("SHOW TABLES").count() >= 2)
assert(
sql("SHOW TABLES IN default").count() >= 2)
}
}

test("show tblproperties of data source tables - basic") {
checkAnswer(
sql("SHOW TBLPROPERTIES parquet_tab1")
.filter(s"key = 'spark.sql.sources.provider'"),
Row("spark.sql.sources.provider", "org.apache.spark.sql.parquet.DefaultSource") :: Nil
)

checkAnswer(
sql("SHOW TBLPROPERTIES parquet_tab1(spark.sql.sources.provider)"),
Row("org.apache.spark.sql.parquet.DefaultSource") :: Nil
)

checkAnswer(
sql("SHOW TBLPROPERTIES parquet_tab1")
.filter(s"key = 'spark.sql.sources.schema.numParts'"),
Row("spark.sql.sources.schema.numParts", "1") :: Nil
)

checkAnswer(
sql("SHOW TBLPROPERTIES parquet_tab1('spark.sql.sources.schema.numParts')"),
Row("1"))
}

test("show tblproperties for datasource table - errors") {
val message1 = intercept[AnalysisException] {
sql("SHOW TBLPROPERTIES badtable")
}.getMessage
assert(message1.contains("Table badtable not found in database default"))

// When key is not found, a row containing the error is returned.
checkAnswer(
sql("SHOW TBLPROPERTIES parquet_tab1('invalid.prop.key')"),
Row("Table default.parquet_tab1 does not have property: invalid.prop.key") :: Nil
)
}

test("show tblproperties for hive table") {
checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('prop1Key')"), Row("prop1Val"))
checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('`prop2Key`')"), Row("prop2Val"))
}

test("show tblproperties for spark temporary table - empty row") {
withTempTable("parquet_temp") {
sql(
"""
|CREATE TEMPORARY TABLE parquet_temp (c1 INT, c2 STRING)
|USING org.apache.spark.sql.parquet.DefaultSource
""".stripMargin)

// An empty sequence of row is returned for session temporary table.
checkAnswer(sql("SHOW TBLPROPERTIES parquet_temp"), Nil)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1811,26 +1811,4 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
}
}
}

test("show tables") {
withTable("show1a", "show2b") {
sql("CREATE TABLE show1a(c1 int)")
sql("CREATE TABLE show2b(c2 int)")
checkAnswer(
sql("SHOW TABLES IN default 'show1*'"),
Row("show1a", false) :: Nil)
checkAnswer(
sql("SHOW TABLES IN default 'show1*|show2*'"),
Row("show1a", false) ::
Row("show2b", false) :: Nil)
checkAnswer(
sql("SHOW TABLES 'show1*|show2*'"),
Row("show1a", false) ::
Row("show2b", false) :: Nil)
assert(
sql("SHOW TABLES").count() >= 2)
assert(
sql("SHOW TABLES IN default").count() >= 2)
}
}
}