Skip to content

Commit c779c4b

Browse files
committed
Herman's comment
1 parent 8ab51ea commit c779c4b

File tree

4 files changed

+37
-34
lines changed

4 files changed

+37
-34
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -163,6 +163,7 @@ class SessionCatalog(
163163
/**
164164
* Retrieve the metadata of an existing metastore table.
165165
* If no database is specified, assume the table is in the current database.
166+
* If the specified table is not found in the database then an [[AnalysisException]] is thrown.
166167
*/
167168
def getTable(name: TableIdentifier): CatalogTable = {
168169
val db = name.database.getOrElse(currentDb)

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala

Lines changed: 8 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -93,10 +93,9 @@ class SparkSqlAstBuilder extends AstBuilder {
9393
}
9494

9595
/**
96-
* A command for users to list the properties for a table.
97-
* If propertyKey is specified, the value for the propertyKey
98-
* is returned. If propertyKey is not specified, all the keys
99-
* and their corresponding values are returned.
96+
* A command for users to list the properties for a table. If propertyKey is specified, the value
97+
* for the propertyKey is returned. If propertyKey is not specified, all the keys and their
98+
* corresponding values are returned.
10099
* The syntax of using this command in SQL is:
101100
* {{{
102101
* SHOW TBLPROPERTIES table_name[('propertyKey')];
@@ -236,14 +235,17 @@ class SparkSqlAstBuilder extends AstBuilder {
236235
override def visitTablePropertyList(
237236
ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) {
238237
ctx.tableProperty.asScala.map { property =>
239-
// A key can either be a String or a collection of dot separated elements. We need to treat
240-
// these differently.
241238
val key = visitTablePropertyKey(property.key)
242239
val value = Option(property.value).map(string).orNull
243240
key -> value
244241
}.toMap
245242
}
246243

244+
/**
245+
* A table property key can either be String or a collection of dot separated elements. This
246+
* function extracts the property key based on whether its a string literal or a table property
247+
* identifier.
248+
*/
247249
override def visitTablePropertyKey(key: TablePropertyKeyContext): String = {
248250
if (key.STRING != null) {
249251
string(key.STRING)

sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala

Lines changed: 20 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -374,10 +374,9 @@ case class ShowDatabasesCommand(databasePattern: Option[String]) extends Runnabl
374374
}
375375

376376
/**
377-
* A command for users to list the properties for a table.
378-
* If propertyKey is specified, the value for the propertyKey
379-
* is returned. If propertyKey is not specified, all the keys
380-
* and their corresponding values are returned.
377+
* A command for users to list the properties for a table If propertyKey is specified, the value
378+
* for the propertyKey is returned. If propertyKey is not specified, all the keys and their
379+
* corresponding values are returned.
381380
* The syntax of using this command in SQL is:
382381
* {{{
383382
* SHOW TBLPROPERTIES table_name[('propertyKey')];
@@ -395,26 +394,29 @@ case class ShowTablePropertiesCommand(
395394
AttributeReference("key", StringType, nullable = false)() ::
396395
AttributeReference("value", StringType, nullable = false)() :: Nil
397396
}
398-
propertyKey.map(p => withKeySchema).getOrElse(noKeySchema)
397+
propertyKey match {
398+
case None => noKeySchema
399+
case _ => withKeySchema
400+
}
399401
}
400402

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

404406
if (catalog.isTemporaryTable(table)) {
405-
throw new AnalysisException("This operation is unsupported for temporary tables")
406-
}
407-
val catalogTable = sqlContext.sessionState.catalog.getTable(table)
408-
409-
propertyKey match {
410-
case Some(p) =>
411-
val errorStr = s"Table ${catalogTable.qualifiedName} does not have property: $p"
412-
val propValue = catalogTable
413-
.properties
414-
.getOrElse(p, throw new AnalysisException(errorStr))
415-
Seq(Row(propValue))
416-
case None =>
417-
catalogTable.properties.map(p => Row(p._1, p._2)).toSeq
407+
Seq.empty[Row]
408+
} else {
409+
val catalogTable = sqlContext.sessionState.catalog.getTable(table)
410+
411+
propertyKey match {
412+
case Some(p) =>
413+
val propValue = catalogTable
414+
.properties
415+
.getOrElse(p, s"Table ${catalogTable.qualifiedName} does not have property: $p")
416+
Seq(Row(propValue))
417+
case None =>
418+
catalogTable.properties.map(p => Row(p._1, p._2)).toSeq
419+
}
418420
}
419421
}
420422
}

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala

Lines changed: 8 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -98,30 +98,28 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
9898
}.getMessage
9999
assert(message1.contains("Table badtable not found in database default"))
100100

101-
val message2 = intercept[AnalysisException] {
102-
sql("SHOW TBLPROPERTIES parquet_tab1('invalid.prop.key')")
103-
}.getMessage
104-
assert(
105-
message2.contains(s"Table default.parquet_tab1 does not have property: invalid.prop.key"))
101+
// When key is not found, a row containing the error is returned.
102+
checkAnswer(
103+
sql("SHOW TBLPROPERTIES parquet_tab1('invalid.prop.key')"),
104+
Row("Table default.parquet_tab1 does not have property: invalid.prop.key") :: Nil
105+
)
106106
}
107107

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

113-
test("show tblproperties for spark temporary table - error") {
113+
test("show tblproperties for spark temporary table - empty row") {
114114
withTempTable("parquet_temp") {
115115
sql(
116116
"""
117117
|CREATE TEMPORARY TABLE parquet_temp (c1 INT, c2 STRING)
118118
|USING org.apache.spark.sql.parquet.DefaultSource
119119
""".stripMargin)
120120

121-
val message1 = intercept[AnalysisException] {
122-
sql("SHOW TBLPROPERTIES parquet_temp")
123-
}.getMessage
124-
assert(message1.contains("This operation is unsupported for temporary tables"))
121+
// An empty sequence of row is returned for session temporary table.
122+
checkAnswer(sql("SHOW TBLPROPERTIES parquet_temp"), Nil)
125123
}
126124
}
127125
}

0 commit comments

Comments
 (0)