Skip to content
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

[SPARK-50600][CONNECT][SQL] Set analyzed on analysis failure #49383

Open
wants to merge 2 commits into
base: master
Choose a base branch
from

Conversation

jdesjean
Copy link
Contributor

@jdesjean jdesjean commented Jan 6, 2025

What changes were proposed in this pull request?

As part of SPARK-44145, a callback was added to track completion of analysis and optimization phase of a query. While the analyzed plan is sent when analysis completes successfully it does not when it fail. In that case, we should fallback to the ParsedPlan.

Why are the changes needed?

The purpose of the analyze event is to track when analysis completes, as such it should also be sent on both success & failure.

Does this PR introduce any user-facing change?

No

How was this patch tested?

Unit

Was this patch authored or co-authored using generative AI tooling?

No

@github-actions github-actions bot added the SQL label Jan 6, 2025
@jdesjean jdesjean changed the title [SPARK-50600] [SPARK-50600][SQL] Set analyzed on analysis failure Jan 6, 2025
@jdesjean jdesjean marked this pull request as ready for review January 6, 2025 20:55
@jdesjean jdesjean force-pushed the jdesjean/SPARK-50600 branch 2 times, most recently from 815a10c to d69efe0 Compare January 7, 2025 17:40
@jdesjean jdesjean force-pushed the jdesjean/SPARK-50600 branch from 5ef6c5c to 5863371 Compare January 8, 2025 23:33
@jdesjean jdesjean changed the title [SPARK-50600][SQL] Set analyzed on analysis failure [SPARK-50600][CONNECT][SQL] Set analyzed on analysis failure Jan 8, 2025
@jdesjean jdesjean force-pushed the jdesjean/SPARK-50600 branch 4 times, most recently from 29333e8 to 771da36 Compare January 8, 2025 23:43
@jdesjean jdesjean force-pushed the jdesjean/SPARK-50600 branch from 771da36 to 11d42f2 Compare January 8, 2025 23:44
@@ -147,6 +157,17 @@ class QueryPlanningTracker(
ret
}

/**
* Set when the query has been parsed.
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
* Set when the query has been parsed.
* Set when the query has been parsed but failed to be analyzed.

@@ -145,13 +145,19 @@ case class ExecuteEventsManager(executeHolder: ExecuteHolder, clock: Clock) {
*
* @param analyzedPlan
* The analyzed plan generated by the Connect request plan. None when the request does not
* generate a Spark plan or analysis fails.
* @param parsedPlan
* The analyzed plan generated by the Connect request plan. None when the request does not
Copy link
Contributor

Choose a reason for hiding this comment

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

does not fail analysis?

@@ -251,6 +257,10 @@ case class ExecuteEventsManager(executeHolder: ExecuteHolder, clock: Clock) {
postAnalyzed(Some(analyzedPlan))
}

override def analysisFailed(tracker: QueryPlanningTracker, parsedPlan: LogicalPlan): Unit = {
postAnalyzed(None, Some(parsedPlan))
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
postAnalyzed(None, Some(parsedPlan))
postAnalyzed(parsedPlan = Some(parsedPlan))

@@ -342,9 +352,15 @@ case class SparkListenerConnectOperationAnalyzed(
extends SparkListenerEvent {

/**
* Analyzed Spark plan generated by the Connect request. None when the Connect request does not
* Parsed Spark plan generated by the Connect request. None when the Connect request does not
* generate a Spark plan.
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto, it's None when request is analyzed successfully?

val plan = executePhase(QueryPlanningTracker.ANALYSIS) {
// We can't clone `logical` here, which will reset the `_analyzed` flag.
sparkSession.sessionState.analyzer.executeAndCheck(logical, tracker)
val plan = Try {
Copy link
Contributor

Choose a reason for hiding this comment

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

can we use the primitive try?

try {
  val plan = ...
  ...
} catch {
  case NonFatal(_) => tracker. setAnalysisFailed...
}

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants