aboutsummaryrefslogtreecommitdiff
path: root/sql/catalyst
diff options
context:
space:
mode:
authorDilip Biswal <dbiswal@us.ibm.com>2016-04-27 09:28:24 +0800
committerCheng Lian <lian@databricks.com>2016-04-27 09:28:24 +0800
commitd93976d8660b68eeef646d1fe687cfce01f50f9d (patch)
treee7b6b96ef5abd5972c161033d10112e58483b464 /sql/catalyst
parentbd2c9a6d48ef6d489c747d9db2642bdef6b1f728 (diff)
downloadspark-d93976d8660b68eeef646d1fe687cfce01f50f9d.tar.gz
spark-d93976d8660b68eeef646d1fe687cfce01f50f9d.tar.bz2
spark-d93976d8660b68eeef646d1fe687cfce01f50f9d.zip
[SPARK-14445][SQL] Support native execution of SHOW COLUMNS and SHOW PARTITIONS
## What changes were proposed in this pull request? This PR adds Native execution of SHOW COLUMNS and SHOW PARTITION commands. Command Syntax: ``` SQL SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database] ``` ``` SQL SHOW PARTITIONS [db_name.]table_name [PARTITION(partition_spec)] ``` ## How was this patch tested? Added test cases in HiveCommandSuite to verify execution and DDLCommandSuite to verify plans. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #12222 from dilipbiswal/dkb_show_columns.
Diffstat (limited to 'sql/catalyst')
-rw-r--r--sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g44
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala7
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala13
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala16
4 files changed, 32 insertions, 8 deletions
diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
index 64f68c9e9e..9de313ae87 100644
--- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
+++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -106,6 +106,9 @@ statement
| SHOW DATABASES (LIKE pattern=STRING)? #showDatabases
| SHOW TBLPROPERTIES table=tableIdentifier
('(' key=tablePropertyKey ')')? #showTblProperties
+ | SHOW COLUMNS (FROM | IN) tableIdentifier
+ ((FROM | IN) db=identifier)? #showColumns
+ | SHOW PARTITIONS tableIdentifier partitionSpec? #showPartitions
| SHOW FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))? #showFunctions
| (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction
| (DESC | DESCRIBE) option=(EXTENDED | FORMATTED)?
@@ -128,7 +131,6 @@ hiveNativeCommands
: DELETE FROM tableIdentifier (WHERE booleanExpression)?
| TRUNCATE TABLE tableIdentifier partitionSpec?
(COLUMNS identifierList)?
- | SHOW COLUMNS (FROM | IN) tableIdentifier ((FROM|IN) identifier)?
| START TRANSACTION (transactionMode (',' transactionMode)*)?
| COMMIT WORK?
| ROLLBACK WORK?
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
index b8f0e458fa..28a67067d0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala
@@ -300,8 +300,13 @@ class InMemoryCatalog extends ExternalCatalog {
override def listPartitions(
db: String,
- table: String): Seq[CatalogTablePartition] = synchronized {
+ table: String,
+ partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = synchronized {
requireTableExists(db, table)
+ if (partialSpec.nonEmpty) {
+ throw new AnalysisException("listPartition does not support partition spec in " +
+ "InMemoryCatalog.")
+ }
catalog(db).tables(table).partitions.values.toSeq
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index b36a76a888..402aacfc1f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -471,13 +471,18 @@ class SessionCatalog(
}
/**
- * List all partitions in a table, assuming it exists.
- * If no database is specified, assume the table is in the current database.
+ * List the metadata of all partitions that belong to the specified table, assuming it exists.
+ *
+ * A partial partition spec may optionally be provided to filter the partitions returned.
+ * For instance, if there exist partitions (a='1', b='2'), (a='1', b='3') and (a='2', b='4'),
+ * then a partial spec of (a='1') will return the first two only.
*/
- def listPartitions(tableName: TableIdentifier): Seq[CatalogTablePartition] = {
+ def listPartitions(
+ tableName: TableIdentifier,
+ partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = {
val db = tableName.database.getOrElse(currentDb)
val table = formatTableName(tableName.table)
- externalCatalog.listPartitions(db, table)
+ externalCatalog.listPartitions(db, table, partialSpec)
}
// ----------------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
index fd5bcad0f8..9e90987731 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
@@ -157,8 +157,20 @@ abstract class ExternalCatalog {
def getPartition(db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition
- // TODO: support listing by pattern
- def listPartitions(db: String, table: String): Seq[CatalogTablePartition]
+ /**
+ * List the metadata of all partitions that belong to the specified table, assuming it exists.
+ *
+ * A partial partition spec may optionally be provided to filter the partitions returned.
+ * For instance, if there exist partitions (a='1', b='2'), (a='1', b='3') and (a='2', b='4'),
+ * then a partial spec of (a='1') will return the first two only.
+ * @param db database name
+ * @param table table name
+ * @param partialSpec partition spec
+ */
+ def listPartitions(
+ db: String,
+ table: String,
+ partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition]
// --------------------------------------------------------------------------
// Functions