diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 20c1756ef4efa..d2a119556f7fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -238,6 +238,13 @@ class Analyzer(override val catalogManager: CatalogManager) errorOnExceed = true, maxIterationsSetting = SQLConf.ANALYZER_MAX_ITERATIONS.key) + /** + * Override to provide rules to do pre-resolution. Note that these rules will be executed + * in an individual batch. This batch is to run right before the normal resolution batch and + * execute its rules in one pass. + */ + val preResolutionRules: Seq[Rule[LogicalPlan]] = Nil + /** * Override to provide additional rules for the "Resolution" batch. */ @@ -276,6 +283,8 @@ class Analyzer(override val catalogManager: CatalogManager) LookupFunctions), Batch("Keep Legacy Outputs", Once, KeepLegacyOutputs), + Batch("PreResolution", Once, + preResolutionRules: _*), Batch("Resolution", fixedPoint, ResolveTableValuedFunctions(v1SessionCatalog) :: ResolveNamespace(catalogManager) :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala index a8ccc39ac478f..bc68d0e1800ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala @@ -38,7 +38,9 @@ import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} * This current provides the following extension points: * *