[SPARK-32555][SQL] Add unique ID on query execution

### What changes were proposed in this pull request?

This PR adds unique ID on QueryExecution, so that listeners can leverage the ID to deduplicate redundant calls.

### Why are the changes needed?

I've observed that Spark calls QueryExecutionListener multiple times on same QueryExecution instance (even same funcName for onSuccess). There's no unique ID on QueryExecution, hence it's a bit tricky if the listener would like to deal with same query execution only once.

Note that streaming query has both query ID and run ID which can be leveraged as unique ID.

### Does this PR introduce _any_ user-facing change?

Yes for who uses query execution listener - they'll see `id` field in QueryExecution and leverage it.

### How was this patch tested?

Manually tested. I think the change is obvious hence don't think it warrants a new UT. StreamingQueryListener has been using UUID as `queryId` and `runId` so it should work for the same.

Closes #29372 from HeartSaVioR/SPARK-32555.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
index e4b9322..eecf16d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
@@ -19,6 +19,7 @@
 
 import java.io.{BufferedWriter, OutputStreamWriter}
 import java.util.UUID
+import java.util.concurrent.atomic.AtomicLong
 
 import org.apache.hadoop.fs.Path
 
@@ -54,6 +55,8 @@
     val logical: LogicalPlan,
     val tracker: QueryPlanningTracker = new QueryPlanningTracker) extends Logging {
 
+  val id: Long = QueryExecution.nextExecutionId
+
   // TODO: Move the planner an optimizer into here from SessionState.
   protected def planner = sparkSession.sessionState.planner
 
@@ -319,6 +322,10 @@
 }
 
 object QueryExecution {
+  private val _nextExecutionId = new AtomicLong(0)
+
+  private def nextExecutionId: Long = _nextExecutionId.getAndIncrement
+
   /**
    * Construct a sequence of rules that are used to prepare a planned [[SparkPlan]] for execution.
    * These rules will make sure subqueries are planned, make use the data partitioning and ordering