Skip to content
Closed
Changes from 1 commit
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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, ResolveDefaultColumns => DefaultCols}
import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._
import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, LookupCatalog, SupportsNamespaces, V1Table}
import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, DelegatingCatalogExtension, LookupCatalog, SupportsNamespaces, V1Table}
import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.command._
Expand Down Expand Up @@ -284,7 +284,8 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager)
case AnalyzeColumn(ResolvedV1TableOrViewIdentifier(ident), columnNames, allColumns) =>
AnalyzeColumnCommand(ident, columnNames, allColumns)

case RepairTable(ResolvedV1TableIdentifier(ident), addPartitions, dropPartitions) =>
case RepairTable(
ResolvedTableIdentifierInSessionCatalog(ident), addPartitions, dropPartitions) =>
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
ResolvedTableIdentifierInSessionCatalog(ident), addPartitions, dropPartitions) =>
ResolvedTableIdentifierInSessionCatalog(ident), addPartitions, dropPartitions) =>

Copy link
Contributor

Choose a reason for hiding this comment

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

nit: fix indentation

RepairTableCommand(ident, addPartitions, dropPartitions)

case LoadData(ResolvedV1TableIdentifier(ident), path, isLocal, isOverwrite, partition) =>
Expand Down Expand Up @@ -600,6 +601,14 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager)
}
}

object ResolvedTableIdentifierInSessionCatalog {
def unapply(resolved: LogicalPlan): Option[TableIdentifier] = resolved match {
case ResolvedTable(catalog, _, t: V1Table, _) if isSessionCatalog(catalog) =>
Some(t.catalogTable.identifier)
case _ => None
}
}

object ResolvedV1TableOrViewIdentifier {
def unapply(resolved: LogicalPlan): Option[TableIdentifier] = resolved match {
case ResolvedV1TableIdentifier(ident) => Some(ident)
Expand Down Expand Up @@ -684,7 +693,8 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager)
}

private def supportsV1Command(catalog: CatalogPlugin): Boolean = {
isSessionCatalog(catalog) &&
SQLConf.get.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).isEmpty
(isSessionCatalog(catalog) &&
Copy link
Contributor

Choose a reason for hiding this comment

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

I think isSessionCatalog(catalog) should always be checked.

SQLConf.get.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).isEmpty) ||
catalog.isInstanceOf[DelegatingCatalogExtension]
}
}