Skip to content

[SPARK-6586][SQL] Add the capability of retrieving original logical plan of DataFrame #5241

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

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
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 @@ -64,9 +64,17 @@ class Analyzer(catalog: Catalog,
UnresolvedHavingClauseAttributes ::
TrimGroupingAliases ::
typeCoercionRules ++
extendedResolutionRules : _*)
extendedResolutionRules : _*),
Batch("SetAnalyzed", Once, SetLogicalPlanAnalyzed)
)

object SetLogicalPlanAnalyzed extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = {
plan.analyzed = true
plan
}
}

/**
* Removes no-op Alias expressions from the plan.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,15 @@ import org.apache.spark.sql.catalyst.trees
abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging {
self: Product =>

protected[sql] var analyzed: Boolean = false

def originalPlan: LogicalPlan = this transform {
case p if p.analyzed =>
p._origPlan.getOrElse(p)
}

protected[sql] var _origPlan: Option[LogicalPlan] = None

/**
* Computes [[Statistics]] for this plan. The default implementation assumes the output
* cardinality is the product of of all child plan's cardinality, i.e. applies in the case
Expand Down
2 changes: 2 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,8 @@ class DataFrame private[sql](
queryExecution.analyzed
}

@transient val originalPlan: LogicalPlan = logicalPlan.originalPlan

/**
* An implicit conversion function internal to this class for us to avoid doing
* "new DataFrame(...)" everywhere.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1107,7 +1107,14 @@ class SQLContext(@transient val sparkContext: SparkContext)
protected[sql] class QueryExecution(val logical: LogicalPlan) {
def assertAnalyzed(): Unit = checkAnalysis(analyzed)

lazy val analyzed: LogicalPlan = analyzer(logical)
lazy val analyzed: LogicalPlan = if (!logical.analyzed) {
val plan = analyzer(logical)
plan._origPlan = Some(logical)
plan
} else {
logical
}

lazy val withCachedData: LogicalPlan = {
assertAnalyzed()
cacheManager.useCachedData(analyzed)
Expand Down