LogicalPlan — Logical Relational Operators of Structured Query¶
LogicalPlan is an extension of the QueryPlan abstraction for logical operators to build a logical query plan (as a tree of logical operators).
LogicalPlan is eventually resolved (transformed) to a physical operator.
Implementations¶
BinaryNode¶
Logical operators with two child logical operators
Command¶
LeafNode¶
LeafNode is a logical operator with no child operators
UnaryNode¶
Logical operators with a single child logical operator
Other Logical Operators¶
- CreateTable
- IgnoreCachedData
- NamedRelation
- ParsedStatement
- SupportsSubquery
- V2CreateTablePlan
- View
- others
Statistics Cache¶
Cached plan statistics (as Statistics) of the LogicalPlan
Computed and cached in stats
Used in stats and verboseStringWithSuffix
Reset in invalidateStatsCache
Estimated Statistics¶
stats(
conf: CatalystConf): Statistics
stats returns the <
stats is used when:
- A
LogicalPlan<Statistics>> QueryExecutionis requested to build a complete text representationJoinSelectionchecks whether a plan can be broadcast et al- CostBasedJoinReorder.md[CostBasedJoinReorder] attempts to reorder inner joins
LimitPushDownis executed (for FullOuter join)AggregateEstimationestimatesStatisticsFilterEstimationestimates childStatisticsInnerOuterEstimationestimatesStatisticsof the left and right sides of a joinLeftSemiAntiEstimationestimatesStatisticsProjectEstimationestimatesStatistics
Refreshing Child Logical Operators¶
refresh(): Unit
refresh calls itself recursively for every child logical operator.
Note
refresh is overriden by LogicalRelation only (that refreshes the location of HadoopFsRelation relations only).
refresh is used when:
-
SessionCatalogis requested to refresh a table -
CatalogImplis requested to refresh a table
Resolving Column Attributes to References in Query Plan¶
resolve(
nameParts: Seq[String],
resolver: Resolver): Option[NamedExpression]
resolve(
schema: StructType,
resolver: Resolver): Seq[Attribute]
resolve requests the outputAttributes to resolve and then the outputMetadataAttributes if the first resolve did not give a NamedExpression.
Accessing Logical Query Plan of Structured Query¶
In order to get the logical plan of a structured query you should use the <
scala> :type q
org.apache.spark.sql.Dataset[Long]
val plan = q.queryExecution.logical
scala> :type plan
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
LogicalPlan goes through execution stages (as a QueryExecution). In order to convert a LogicalPlan to a QueryExecution you should use SessionState and request it to "execute" the plan.
scala> :type spark
org.apache.spark.sql.SparkSession
// You could use Catalyst DSL to create a logical query plan
scala> :type plan
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
val qe = spark.sessionState.executePlan(plan)
scala> :type qe
org.apache.spark.sql.execution.QueryExecution
Maximum Number of Records¶
maxRows: Option[Long]
maxRows is undefined (None).
Note
maxRows is supposed to be overriden by the implementations for logical optimizations to eliminate in particular (e.g., EliminateLimits and EliminateOffsets logical optimizations) or optimize in general (e.g., OptimizeOneRowPlan) logical query plans.
See:
maxRows is used when:
EliminateLimitslogical optimization is executedEliminateOffsetslogical optimization is executedInferWindowGroupLimitlogical optimization is executed- LimitPushDown logical optimization is executed (for
LeftSemiOrAntiwith no join condition) LimitPushDownThroughWindowlogical optimization is executedLogicalPlanis requested for maxRowsPerPartitionOptimizeOneRowPlanlogical optimization is executedRewriteCorrelatedScalarSubquerylogical optimization is executed
Maximum Number of Records per Partition¶
maxRowsPerPartition: Option[Long]
maxRowsPerPartition is exactly the maximum number of records by default.
maxRowsPerPartition is used when LimitPushDown logical optimization is executed.
Executing Logical Plan¶
A common idiom in Spark SQL to make sure that a logical plan can be analyzed is to request a SparkSession for the SessionState that is in turn requested to "execute" the logical plan (which simply creates a QueryExecution).
scala> :type plan
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
val qe = sparkSession.sessionState.executePlan(plan)
qe.assertAnalyzed()
// the following gives the analyzed logical plan
// no exceptions are expected since analysis went fine
val analyzedPlan = qe.analyzed
Converting Logical Plan to Dataset¶
Another common idiom in Spark SQL to convert a LogicalPlan into a Dataset is to use Dataset.ofRows internal method that "executes" the logical plan followed by creating a Dataset with the QueryExecution and RowEncoder.
childrenResolved¶
childrenResolved: Boolean
A logical operator is considered partially resolved when its child operators are resolved (aka children resolved).
resolved¶
resolved: Boolean
resolved is true for all expressions and the children resolved.
Lazy Value
resolved is a Scala lazy value to guarantee that the code to initialize it is executed once only (when accessed for the first time) and the computed value never changes afterwards.
Metadata Output Attributes¶
metadataOutput: Seq[Attribute]
metadataOutput requests the children for the metadata output attributes (recursively).
Note
metadataOutput should be overridden by operators that do not propagate its children's output.
See:
metadataOutput is used when:
- AddMetadataColumns logical resolution rule is executed
UnresolvedStarexpression is requested to expandLogicalPlanis requested for the child metadata output attributes and outputMetadataAttributes
childMetadataAttributes¶
childMetadataAttributes: AttributeSeq
childMetadataAttributes is an AttributeSeq of the (non-empty) metadataOutputs of the children of this operator.
Lazy Value
childMetadataAttributes is a Scala lazy value to guarantee that the code to initialize it is executed once only (when accessed for the first time) and the computed value never changes afterwards.
Learn more in the Scala Language Specification.
childMetadataAttributes is used when:
LogicalPlanis requested to resolveChildren
outputMetadataAttributes¶
outputMetadataAttributes: AttributeSeq
outputMetadataAttributes is an AttributeSeq of the metadataOutput of this operator.
Lazy Value
outputMetadataAttributes is a Scala lazy value to guarantee that the code to initialize it is executed once only (when accessed for the first time) and the computed value never changes afterwards.
Learn more in the Scala Language Specification.
outputMetadataAttributes is used when:
LogicalPlanis requested to resolve