-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-28856][SQL] Implement SHOW DATABASES for Data Source V2 Tables #25601
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
Changes from 3 commits
1f3b8b9
bab1a8f
87fe6ec
da65365
9974a58
ba1e7f4
9f738cf
7d606e1
672f526
6256aeb
2295f91
9a55a03
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,24 @@ | ||
| /* | ||
| * 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.catalyst.plans.logical.sql | ||
|
|
||
| /** | ||
| * A SHOW DATABASES statement, as parsed from SQL. | ||
| */ | ||
| case class ShowDatabasesStatement(pattern: Option[String]) | ||
| extends ParsedStatement |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,15 +20,15 @@ package org.apache.spark.sql.execution.datasources | |
| import scala.collection.mutable | ||
|
|
||
| import org.apache.spark.sql.{AnalysisException, SaveMode} | ||
| import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, TableCatalog} | ||
| import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog} | ||
| import org.apache.spark.sql.catalog.v2.expressions.Transform | ||
| import org.apache.spark.sql.catalyst.TableIdentifier | ||
| import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedRelation} | ||
| import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} | ||
| import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, Filter, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, ShowTables, SubqueryAlias} | ||
| import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowTablesStatement} | ||
| import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, Filter, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, ShowDatabases, ShowTables, SubqueryAlias} | ||
| import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowDatabasesStatement, ShowTablesStatement} | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} | ||
| import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowDatabasesCommand, ShowTablesCommand} | ||
| import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} | ||
|
|
@@ -178,6 +178,17 @@ case class DataSourceResolution( | |
| val aliased = delete.tableAlias.map(SubqueryAlias(_, relation)).getOrElse(relation) | ||
| DeleteFromTable(aliased, delete.condition) | ||
|
|
||
| case ShowDatabasesStatement(pattern) => | ||
| defaultCatalog match { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think this should be
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yea
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's implement switching the current catalog first, otherwise we are not able to test it.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @imback82 are you working on it?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, I am working on
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I should be able to send out the PR sometime tomorrow. |
||
| case Some(catalog: SupportsNamespaces) => | ||
| ShowDatabases(catalog, pattern) | ||
| case Some(_) => | ||
| throw new AnalysisException( | ||
| "The default v2 catalog doesn't support showing namespaces.") | ||
| case None => | ||
| ShowDatabasesCommand(pattern) | ||
| } | ||
|
|
||
| case ShowTablesStatement(None, pattern) => | ||
| defaultCatalog match { | ||
| case Some(catalog) => | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,62 @@ | ||
| /* | ||
| * 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.datasources.v2 | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.NamespaceHelper | ||
| import org.apache.spark.sql.catalog.v2.SupportsNamespaces | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.encoders.RowEncoder | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} | ||
| import org.apache.spark.sql.catalyst.util.StringUtils | ||
| import org.apache.spark.sql.execution.LeafExecNode | ||
|
|
||
| /** | ||
| * Physical plan node for showing databases. | ||
| */ | ||
| case class ShowDatabasesExec( | ||
| output: Seq[Attribute], | ||
| catalog: SupportsNamespaces, | ||
| pattern: Option[String]) | ||
| extends LeafExecNode { | ||
| override protected def doExecute(): RDD[InternalRow] = { | ||
| val namespaces = catalog.listNamespaces().flatMap(getNamespaces(catalog, _)) | ||
|
||
|
|
||
| val rows = new ArrayBuffer[InternalRow]() | ||
| val encoder = RowEncoder(schema).resolveAndBind() | ||
|
|
||
| namespaces.map(_.quoted).map { namespace => | ||
| if (pattern.map(StringUtils.filterPattern(Seq(namespace), _).nonEmpty).getOrElse(true)) { | ||
| rows += encoder | ||
| .toRow(new GenericRowWithSchema(Array(namespace), schema)) | ||
| .copy() | ||
| } | ||
| } | ||
|
|
||
| sparkContext.parallelize(rows, 1) | ||
| } | ||
|
|
||
| private def getNamespaces( | ||
| catalog: SupportsNamespaces, | ||
| parentNamespace: Array[String]): Array[Array[String]] = { | ||
| val namespaces = catalog.listNamespaces(parentNamespace) | ||
| Array(parentNamespace) ++ namespaces.flatMap(getNamespaces(catalog, _)) | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.