-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 #37588
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 6 commits
39f3c42
cbe7433
023414b
0697773
676455e
9876120
7d71eb0
5c40c7c
3c3651a
b1cc56c
ab50fb0
9c4a9ac
56ceeac
be4199c
3a296c0
f54a832
892de53
f0b2db4
d898932
ffd3148
54531d8
59369a5
482dea1
3b12d76
7684a94
474b963
ac7586f
bcb5db7
74e52ba
723e664
0f5c3c7
e06f867
4af1c5e
ee254f4
02065eb
9c3940c
b47e619
be5b337
73ff15b
e6978e3
1bab140
26ae679
ef4213a
d318f6b
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 |
|---|---|---|
|
|
@@ -2016,12 +2016,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { | |
| "config" -> LEGACY_CTE_PRECEDENCE_POLICY.key)) | ||
| } | ||
|
|
||
| def commandUnsupportedInV2TableError(name: String): Throwable = { | ||
| new AnalysisException( | ||
| errorClass = "_LEGACY_ERROR_TEMP_1200", | ||
| messageParameters = Map("name" -> name)) | ||
| } | ||
|
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. We can remove the error class too.
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. To reduce the complexity of this PR, it will be placed in a separate PR to complete the renaming or removing of the error class-related operations.
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. In fact, just remove the error class, it will not increase the complexity.
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 misunderstood your meaning. That's right, we can remove it from |
||
|
|
||
| def cannotResolveColumnNameAmongAttributesError( | ||
| colName: String, fieldNames: String): Throwable = { | ||
| new AnalysisException( | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -32,9 +32,9 @@ import org.codehaus.commons.compiler.{CompileException, InternalCompilerExceptio | |
| import org.apache.spark._ | ||
| import org.apache.spark.launcher.SparkLauncher | ||
| import org.apache.spark.memory.SparkOutOfMemoryError | ||
| import org.apache.spark.sql.catalyst.{TableIdentifier, WalkedTypePath} | ||
| import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier, WalkedTypePath} | ||
| import org.apache.spark.sql.catalyst.ScalaReflection.Schema | ||
| import org.apache.spark.sql.catalyst.analysis.UnresolvedGenerator | ||
| import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, UnresolvedGenerator} | ||
| import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable} | ||
| import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} | ||
| import org.apache.spark.sql.catalyst.parser.ParseException | ||
|
|
@@ -2790,4 +2790,17 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { | |
| "location" -> toSQLValue(location.toString, StringType), | ||
| "identifier" -> toSQLId(tableId.nameParts))) | ||
| } | ||
|
|
||
| def showTableExtendedMultiPartitionUnsupportedError(tableName: String): Throwable = { | ||
| new UnsupportedOperationException( | ||
| s"The table $tableName does not support show table extended of multiple partition.") | ||
| } | ||
|
|
||
| def notExistPartitionError( | ||
|
||
| tableName: String, | ||
| partitionIdent: InternalRow, | ||
| partitionSchema: StructType): Throwable = { | ||
| new NoSuchPartitionException(tableName, partitionIdent, partitionSchema) | ||
| } | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -65,6 +65,11 @@ object DataSourceV2Implicits { | |
| } | ||
| } | ||
|
|
||
| def isPartitionable: Boolean = table match { | ||
|
||
| case _: SupportsPartitionManagement => true | ||
| case _ => false | ||
| } | ||
|
|
||
| def asPartitionable: SupportsPartitionManagement = { | ||
| table match { | ||
| case support: SupportsPartitionManagement => | ||
|
|
||
| Original file line number | Diff line number | Diff line change | ||||
|---|---|---|---|---|---|---|
|
|
@@ -17,14 +17,25 @@ | |||||
|
|
||||||
| package org.apache.spark.sql.execution.datasources.v2 | ||||||
|
|
||||||
| import scala.collection.JavaConverters._ | ||||||
| import scala.collection.convert.ImplicitConversions.`map AsScala` | ||||||
| import scala.collection.mutable | ||||||
| import scala.collection.mutable.ArrayBuffer | ||||||
|
|
||||||
| import org.apache.spark.sql.catalyst.InternalRow | ||||||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||||||
| import org.apache.spark.sql.catalyst.util.StringUtils | ||||||
| import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} | ||||||
| import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper | ||||||
| import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec.convertToPartIdent | ||||||
| import org.apache.spark.sql.catalyst.catalog.CatalogTableType | ||||||
| import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec | ||||||
| import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName | ||||||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} | ||||||
| import org.apache.spark.sql.catalyst.util.{quoteIdentifier, StringUtils} | ||||||
| import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, SupportsPartitionManagement, Table, TableCatalog} | ||||||
| import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ | ||||||
| import org.apache.spark.sql.errors.QueryExecutionErrors | ||||||
| import org.apache.spark.sql.execution.LeafExecNode | ||||||
| import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits.TableHelper | ||||||
| import org.apache.spark.sql.types.StringType | ||||||
| import org.apache.spark.sql.util.PartitioningUtils.{normalizePartitionSpec, requireExactMatchedPartitionSpec} | ||||||
|
|
||||||
| /** | ||||||
| * Physical plan node for showing tables. | ||||||
|
|
@@ -33,14 +44,37 @@ case class ShowTablesExec( | |||||
| output: Seq[Attribute], | ||||||
| catalog: TableCatalog, | ||||||
| namespace: Seq[String], | ||||||
| pattern: Option[String]) extends V2CommandExec with LeafExecNode { | ||||||
| pattern: Option[String], | ||||||
| isExtended: Boolean = false, | ||||||
| partitionSpec: Option[TablePartitionSpec] = None) extends V2CommandExec with LeafExecNode { | ||||||
|
||||||
| partitionSpec: Option[ResolvedPartitionSpec]) extends V2CommandExec with LeafExecNode { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
is it to follow the v1 behavior?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This one is not needed. The job should be done by ResolvePartitionSpec.resolvePartitionSpec, or there is some reason to bypass it?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, the latest version has eliminated the above logic.
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Where is it used?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The functions extendedPartition() is invoked only for non-empty partition spec as I can see, or not? Is there any test for this case?
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit:
| case len if (len > 1) => | |
| case len if len > 1 => |
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This loop:
var i = 0
while (i < len) {
i += 1
}can be simplified by:
for (i <- 0 until len) {
}
Uh oh!
There was an error while loading. Please reload this page.