Skip to content

Conversation

@asl3
Copy link
Contributor

@asl3 asl3 commented Dec 27, 2025

What changes were proposed in this pull request?

Add a new UUIDv7 queryId object to SparkListenerSQLExecutionStart and propagate it through the SQL execution lifecycle via SparkContext local properties.

Currently, Spark uses executionId to connect jobs, stages, and tasks with SQL executions. However, this field is not globally unique, as multiple Spark applications can include the same executionIds. UUIDv7 allows for a time-ordered, globally unique identifier for improved telemetry across systems.

In a separate PR, plan to add queryId as a new field to SparkUI.

Why are the changes needed?

Add a globally unique, time-ordered identifier for Spark SQL query execution events.

Does this PR introduce any user-facing change?

No, this PR simply adds the internal queryId which is not yet surfaced.

How was this patch tested?

Added tests for UUIDv7 generator and SQLExecution queryId propagation.

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

No.

@asl3 asl3 changed the title [SPARK-54854][SQL] Add queryId (UUIDv7) to SQL Execution Events [SPARK-54854][SQL] Add a UUIDv7 queryId to SQL Execution Events Dec 27, 2025
@asl3 asl3 changed the title [SPARK-54854][SQL] Add a UUIDv7 queryId to SQL Execution Events [SPARK-54854][SQL] Add a UUIDv7 queryId to SQLExecution Events Dec 27, 2025
val queryId: UUID = UUIDv7Generator.generate()

// Tracks how many times this QueryExecution has been executed.
// Used by SQLExecution to determine whether to use qplQueryId or generate a new one.
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
// Used by SQLExecution to determine whether to use qplQueryId or generate a new one.
// Used by SQLExecution to determine whether to use the existing queryId or generate a new one.

val outputMode: OutputMode,
val checkpointLocation: String,
val queryId: UUID,
override val queryId: UUID,
Copy link
Contributor

Choose a reason for hiding this comment

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

does this use UUIDv7?

rootExecutionId: Option[Long],
// A unique identifier for the query execution. For the first execution it equals
// QueryExecution.queryId, for subsequent executions a new UUIDv7 is generated.
queryId: Option[UUID] = None,
Copy link
Contributor

Choose a reason for hiding this comment

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

for better compatibility, can we add it at the end?

Copy link
Contributor

Choose a reason for hiding this comment

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

and we should put it in SparkListenerSQLExecutionEnd as well

Copy link
Contributor

Choose a reason for hiding this comment

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

let's also check JsonProtocol. We should set this field to None when reading legacy event logs.

Copy link

Copilot AI left a comment

Choose a reason for hiding this comment

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

Pull request overview

This PR adds a UUIDv7-based queryId to SQL execution events to provide a globally unique, time-ordered identifier for tracking SQL queries across systems. The key changes include:

  • Introduction of a UUIDv7 generator for creating time-ordered unique identifiers
  • Addition of queryId field to QueryExecution and propagation through the SQL execution lifecycle
  • Modification of SparkListenerSQLExecutionStart to include the queryId

Reviewed changes

Copilot reviewed 15 out of 15 changed files in this pull request and generated 3 comments.

Show a summary per file
File Description
sql/core/src/main/scala/org/apache/spark/sql/util/UUIDv7Generator.scala New UUIDv7 generator implementation following RFC draft specification
sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala Adds queryId field and executionCount tracking to QueryExecution
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala Implements queryId propagation via SparkContext local properties and generation logic
sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala Adds queryId parameter to SparkListenerSQLExecutionStart event
sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala Updates event handler to extract queryId from events
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/runtime/IncrementalExecution.scala Declares queryId as override since it's now inherited from QueryExecution
sql/core/src/test/scala/org/apache/spark/sql/util/UUIDv7GeneratorSuite.scala Comprehensive test suite for UUIDv7 generator covering format, uniqueness, monotonicity, and timestamp accuracy
sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala Tests for queryId propagation in concurrent and sequential execution scenarios
sql/core/src/test/scala/org/apache/spark/sql/execution/ui/*.scala Updates test event constructors to include None for queryId parameter
sql/core/src/test/scala/org/apache/spark/sql/execution/history/*.scala Updates test event constructors to include None for queryId parameter
sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListenerSuite.scala Updates test event constructors to include None for queryId parameter

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

* https://datatracker.ietf.org/doc/html/draft-peabody-dispatch-new-uuid-format#section-5.2
*/

private val random = new Random()
Copy link

Copilot AI Dec 28, 2025

Choose a reason for hiding this comment

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

The shared Random instance in the object is not thread-safe. Since generateFrom can be called from multiple threads concurrently (via SQLExecution.withNewExecutionId0), the shared Random instance may produce non-unique UUIDs due to race conditions in Random.nextLong().

Consider using ThreadLocalRandom.current() instead of a shared Random instance to ensure thread-safety. ThreadLocalRandom is the standard approach for concurrent random number generation in Java/Scala and is used elsewhere in the Spark codebase.

Copilot uses AI. Check for mistakes.
val timestampMs = epochMilli & 0xFFFFFFFFFFFFL

// 12 bits, avoid LSB as most HW clocks have resolution in range of 10-40 ns
val randA = (nano>> 4) & 0xFFF
Copy link

Copilot AI Dec 28, 2025

Choose a reason for hiding this comment

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

Missing space after the right shift operator. The code has (nano>> 4) but should be (nano >> 4) for consistency with Scala conventions and the rest of the codebase. You can see this convention in other files like HashedRelation.scala which uses (address >>> SIZE_BITS) with spaces.

Suggested change
val randA = (nano>> 4) & 0xFFF
val randA = (nano >> 4) & 0xFFF

Copilot uses AI. Check for mistakes.
val queryId: UUID = UUIDv7Generator.generate()

// Tracks how many times this QueryExecution has been executed.
// Used by SQLExecution to determine whether to use qplQueryId or generate a new one.
Copy link

Copilot AI Dec 28, 2025

Choose a reason for hiding this comment

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

The comment refers to "qplQueryId" which appears to be a typo or outdated term. Based on the code and PR description, this should likely be "queryId" for clarity and consistency with the actual field name used throughout the codebase.

Suggested change
// Used by SQLExecution to determine whether to use qplQueryId or generate a new one.
// Used by SQLExecution to determine whether to use queryId or generate a new one.

Copilot uses AI. Check for mistakes.
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