LogicalPlan — Logical Query Plan / Logical Operator

LogicalPlan is a logical QueryPlan (that can be a logical operator or a tree of logical operators and hence the entire structured query).

Note
LogicalPlan uses Catalyst Framework to represent itself as a tree of children LogicalPlans which are logical operators.

You can access the LogicalPlan of a Dataset through its QueryExecution.

val plan = dataset.queryExecution.logical

LogicalPlan can be analyzed which is to say that the plan (including children) has gone through analysis and verification.

scala> plan.analyzed
res1: Boolean = true

A logical plan can also be resolved to a specific schema.

scala> plan.resolved
res2: Boolean = true

A logical plan knows the size of objects that are results of query operators, like join, through Statistics object.

scala> val stats = plan.statistics
stats: org.apache.spark.sql.catalyst.plans.logical.Statistics = Statistics(8,false)

A logical plan knows the maximum number of records it can compute.

scala> val maxRows = plan.maxRows
maxRows: Option[Long] = None

LogicalPlan can be streaming if it contains one or more structured streaming sources.

Table 1. Logical Query Operators / Specialized Logical Plans
LogicalPlan Description

LeafNode

Logical operator with no child operators

UnaryNode

Logical operator with a single child operator

BinaryNode

Logical operator with two child operators

Command

RunnableCommand

Table 2. LogicalPlan’s Internal Registries and Counters (in alphabetical order)
Name Description

statsCache

Cached plan statistics (as Statistics) of the LogicalPlan

Computed and cached in stats.

Used in stats and verboseStringWithSuffix.

Reset in invalidateStatsCache

Getting Cached or Calculating Statistics — stats Method

stats(conf: CatalystConf): Statistics

stats returns the cached plan statistics or computes a new one (and caches it as statsCache).

Note

stats is used when:

invalidateStatsCache method

Caution
FIXME

verboseStringWithSuffix method

Caution
FIXME

resolveQuoted method

Caution
FIXME

setAnalyzed method

Caution
FIXME

Command — Logical Commands

Command is the base for leaf logical plans that represent non-query commands to be executed by the system. It defines output to return an empty collection of Attributes.

Known commands are:

  1. CreateTable

  2. Any RunnableCommand

Is Logical Plan Structured Streaming — isStreaming method

isStreaming: Boolean

isStreaming is a part of the public API of LogicalPlan and is enabled (i.e. true) when a logical plan is a streaming source.

By default, it walks over subtrees and calls itself, i.e. isStreaming, on every child node to find a streaming source.

val spark: SparkSession = ...

// Regular dataset
scala> val ints = spark.createDataset(0 to 9)
ints: org.apache.spark.sql.Dataset[Int] = [value: int]

scala> ints.queryExecution.logical.isStreaming
res1: Boolean = false

// Streaming dataset
scala> val logs = spark.readStream.format("text").load("logs/*.out")
logs: org.apache.spark.sql.DataFrame = [value: string]

scala> logs.queryExecution.logical.isStreaming
res2: Boolean = true

Computing Statistics (of All Child Logical Plans) — computeStats method

computeStats(conf: CatalystConf): Statistics

computeStats creates a Statistics with sizeInBytes being a product of statistics of all child logical plans.

For a no-children logical plan, computeStats reports a UnsupportedOperationException:

LeafNode [nodeName] must implement statistics.
Note
computeStats is a protected method that logical operators are expected to override to provide their own custom plan statistics calculation.
Note
computeStats is used exclusively when LogicalPlan is requested for plan statistics.

results matching ""

    No results matching ""