cloud-fan commented on code in PR #53625:
URL: https://github.com/apache/spark/pull/53625#discussion_r2654689904
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -92,8 +93,17 @@ object SQLExecution extends Logging {
val sparkSession = queryExecution.sparkSession
val sc = sparkSession.sparkContext
val oldExecutionId = sc.getLocalProperty(EXECUTION_ID_KEY)
+ val oldQueryId = sc.getLocalProperty(QUERY_ID_KEY)
val executionId = SQLExecution.nextExecutionId
+ // Use the original queryId for the first execution, generate new ones for
+ // subsequent executions
+ val queryId = if (queryExecution.firstExecution.compareAndSet(true,
false)) {
+ queryExecution.queryId
+ } else {
+ UUIDv7Generator.generate()
+ }
Review Comment:
I'd like to keep things simple for the common cases. A query id should cover
the entire life cycle of the query, including analysis, optimization, planning,
execution, etc. The goal is we can use query id to filter out all the logs for
the entire life cycle of the query.
Again, repeated df execution is a corner case and we shouldn't complicate
our framework for it. It should not be counted as part of the original query's
life cycle. Thus it needs a new query id. IMO it's a new query, but it shares
the compiled physical plan with the original query, so its life cycle only has
execution. We can record this dependency though, by recording a
`dependent_query_id`. This extra id is only needed for the repeated df
execution and common cases can remain simple. We can add `dependent_query_id`
later though.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]