juliuszsompolski commented on code in PR #41443:
URL: https://github.com/apache/spark/pull/41443#discussion_r1230125406


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -63,6 +63,9 @@ class SparkConnectStreamHandler(responseObserver: 
StreamObserver[ExecutePlanResp
         }
 
       val executeHolder = sessionHolder.createExecutePlanHolder(v)
+
+      sessionHolder.events.postStarted(executeHolder)

Review Comment:
   Note that in https://github.com/apache/spark/pull/41315 (WIP), I am moving 
the actual execution to a different thread, kind of like Thriftserver starts 
the actual execution in SparkExecuteOperation.runInternal -> execute.
   Do we want to consider having separate STARTED and EXECUTING/RUNNING, the 
way thriftserver has?



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/Events.scala:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connect.service
+
+import com.fasterxml.jackson.annotation.JsonIgnore
+import com.google.protobuf.Message
+
+import org.apache.spark.SparkContext
+import org.apache.spark.connect.proto
+import org.apache.spark.scheduler.SparkListenerEvent
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan}
+import org.apache.spark.sql.connect.common.ProtoUtils
+import org.apache.spark.util.{Clock, Utils}
+
+object Events {
+  // TODO: Make this configurable
+  val MAX_STATEMENT_TEXT_SIZE = 65535
+}
+
+/**
+ * Post Connect events to @link org.apache.spark.scheduler.LiveListenerBus.
+ *
+ * @param sessionHolder:
+ *   Session for which the events are generated.
+ * @param clock:
+ *   Source of time for unit tests.
+ */
+case class Events(sessionHolder: SessionHolder, clock: Clock) {
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationStarted to 
@link
+   * org.apache.spark.scheduler.LiveListenerBus.
+   *
+   * @param planHolder:
+   *   The Connect request plan to execute.
+   */
+  def postStarted(planHolder: ExecutePlanHolder): Unit = {
+    val sc = sessionHolder.session.sparkContext
+    val request = planHolder.request
+    val plan: Message =
+      request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.COMMAND => request.getPlan.getCommand
+        case proto.Plan.OpTypeCase.ROOT => request.getPlan.getRoot
+        case _ =>
+          throw new UnsupportedOperationException(
+            s"${request.getPlan.getOpTypeCase} not supported.")
+      }
+
+    sc.listenerBus.post(
+      SparkListenerConnectOperationStarted(
+        planHolder.jobGroupId,
+        planHolder.operationId,
+        clock.getTimeMillis(),
+        request.getSessionId,
+        request.getUserContext.getUserId,
+        request.getUserContext.getUserName,
+        Utils.redact(
+          sessionHolder.session.sessionState.conf.stringRedactionPattern,
+          ProtoUtils.abbreviate(plan, 
Events.MAX_STATEMENT_TEXT_SIZE).toString),
+        request.getClientType))
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationParsed to 
@link
+   * org.apache.spark.scheduler.LiveListenerBus.
+   *
+   * @param dataFrameOpt:
+   *   The Dataframe generated by the Connect request plan. None when the 
request does not
+   *   generate a plan.
+   */
+  def postParsed(dataFrameOpt: Option[DataFrame] = None): Unit = {
+    assertExecutedPlanPrepared(dataFrameOpt)
+    val jobGroupId = assertJobGroupId()
+    val operationId = assertOperationId(jobGroupId)
+    val event =
+      SparkListenerConnectOperationParsed(jobGroupId, operationId, 
clock.getTimeMillis())
+    event.analyzedPlan = dataFrameOpt.map(_.queryExecution.analyzed)
+    sessionHolder.session.sparkContext.listenerBus.post(event)
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationCanceled to
+   * \@link org.apache.spark.scheduler.LiveListenerBus.
+   *
+   * @param jobGroupIdOpt:
+   *   jobGroupId (@link org.apache.spark.SparkContext.setJobGroup) assigned 
by Connect for
+   *   request that was canceled. None when the jobGroupId is not set as a 
locale
+   */
+  def postCanceled(jobGroupIdOpt: Option[String] = None): Unit = {
+    val jobGroupId = jobGroupIdOpt.getOrElse(assertJobGroupId())
+    val operationId = assertOperationId(jobGroupId)
+    sessionHolder.session.sparkContext.listenerBus
+      .post(SparkListenerConnectOperationCanceled(jobGroupId, operationId, 
clock.getTimeMillis()))
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationFailed to 
@link
+   * org.apache.spark.scheduler.LiveListenerBus.
+   *
+   * @param errorMessage:
+   *   The message of the error thrown during the request.
+   */
+  def postFailed(errorMessage: String): Unit = {
+    val jobGroupId = assertJobGroupId()
+    val operationId = assertOperationId(jobGroupId)
+    sessionHolder.session.sparkContext.listenerBus.post(
+      SparkListenerConnectOperationFailed(
+        jobGroupId,
+        operationId,
+        clock.getTimeMillis(),
+        errorMessage))
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationParsed &
+   * @link
+   *   
org.apache.spark.sql.connect.service.SparkListenerConnectOperationFinished to 
@link
+   *   org.apache.spark.scheduler.LiveListenerBus.
+   *
+   * @param dataFrameOpt:
+   *   The Dataframe generated by the Connect request plan. None when the 
request does not
+   *   generate a plan.
+   */
+  def postParsedAndFinished(dataFrameOpt: Option[DataFrame] = None): Unit = {
+    postParsed(dataFrameOpt)
+    postFinished()
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationFinished to
+   * \@link org.apache.spark.scheduler.LiveListenerBus.
+   */
+  def postFinished(): Unit = {
+    val jobGroupId = assertJobGroupId()
+    val operationId = assertOperationId(jobGroupId)
+    sessionHolder.session.sparkContext.listenerBus
+      .post(SparkListenerConnectOperationFinished(jobGroupId, operationId, 
clock.getTimeMillis()))
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationClosed to 
@link
+   * org.apache.spark.scheduler.LiveListenerBus.
+   */
+  def postClosed(): Unit = {
+    val jobGroupId = assertJobGroupId()
+    val operationId = assertOperationId(jobGroupId)
+    sessionHolder.session.sparkContext.listenerBus
+      .post(SparkListenerConnectOperationClosed(jobGroupId, operationId, 
clock.getTimeMillis()))
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectSessionClosed to @link
+   * org.apache.spark.scheduler.LiveListenerBus.
+   */
+  def postSessionClosed(): Unit = {
+    sessionHolder.session.sparkContext.listenerBus
+      .post(SparkListenerConnectSessionClosed(sessionHolder.sessionId, 
clock.getTimeMillis()))
+  }
+
+  private def assertExecutedPlanPrepared(dataFrameOpt: Option[DataFrame]): 
Unit = {
+    dataFrameOpt.foreach { dataFrame =>
+      val isEagerlyExecuted = dataFrame.queryExecution.analyzed.find {
+        case _: Command => true
+        case _ => false
+      }.isDefined
+      val isStreaming = dataFrame.queryExecution.analyzed.isStreaming
+
+      if (!isEagerlyExecuted && !isStreaming) {
+        dataFrame.queryExecution.executedPlan
+      }
+    }
+  }
+
+  private def assertOperationId(jobGroupId: String): String = {
+    ExecutePlanHolder.getQueryOperationId(jobGroupId) match {
+      case Some(operationId) =>
+        operationId
+      case None =>
+        throw new RuntimeException(
+          s"Connect operationId cannot be resolved during sessionId: 
${sessionHolder.sessionId}")
+    }
+  }
+
+  private def assertJobGroupId(): String = {
+    getJobGroupId() match {
+      case Some(jobGroupId) =>
+        jobGroupId
+      case None =>
+        throw new RuntimeException(
+          s"Connect jobGroupId is not set during sessionId: 
${sessionHolder.sessionId}")
+    }
+  }
+
+  private def getJobGroupId(): Option[String] = {
+    Option(
+      sessionHolder.session.sparkContext
+        .getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID))
+  }
+}
+
+/**
+ * Event sent after reception of a Connect request that is ready for execution 
(i.e. not queued),
+ * but prior any analysis or execution.
+ *
+ * @param jobGroupId:
+ *   Opaque Spark jobGroupId (@link org.apache.spark.SparkContext.setJobGroup) 
assigned by Connect
+ *   during a request. Designed to be unique across sessions and requests.
+ * @param operationId:
+ *   32 characters UUID assigned by Connect during a request.
+ * @param eventTime:
+ *   The time in ms when the event was generated.
+ * @param sessionId:
+ *   32 characters UUID assigned by Connect the operation was executed on.

Review Comment:
   Currently sessionId is generated by the client. The Spark Client does 
generate this as an UUID, but this is not enforced at the protocol level, so a 
different 3rd party client implementing the protocol could in theory set it 
differently (or make it user-settable).
   Having this always as an UUID is a good idea. We should enforce this 
(document in protocol proto; enforce when requests are received)



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecutePlanHolder.scala:
##########
@@ -19,6 +19,18 @@ package org.apache.spark.sql.connect.service
 
 import org.apache.spark.connect.proto
 
+object ExecutePlanHolder {
+  val JOB_GROUP_ID_PATTERN = 
"User_(\\d*)_Session_([a-z0-9\\-]*)_Request_([a-z0-9\\-]*)".r
+  def getQueryOperationId(jobGroupId: String): Option[String] = {

Review Comment:
   I would be in favor of threading through ExecutePlanHolder and SessionHolder 
to various places as needed. I planned to do it in a refactoring PR.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/Events.scala:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connect.service
+
+import com.fasterxml.jackson.annotation.JsonIgnore
+import com.google.protobuf.Message
+
+import org.apache.spark.SparkContext
+import org.apache.spark.connect.proto
+import org.apache.spark.scheduler.SparkListenerEvent
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan}
+import org.apache.spark.sql.connect.common.ProtoUtils
+import org.apache.spark.util.{Clock, Utils}
+
+object Events {
+  // TODO: Make this configurable
+  val MAX_STATEMENT_TEXT_SIZE = 65535
+}
+
+/**
+ * Post Connect events to @link org.apache.spark.scheduler.LiveListenerBus.
+ *
+ * @param sessionHolder:
+ *   Session for which the events are generated.
+ * @param clock:
+ *   Source of time for unit tests.
+ */
+case class Events(sessionHolder: SessionHolder, clock: Clock) {
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationStarted to 
@link
+   * org.apache.spark.scheduler.LiveListenerBus.
+   *
+   * @param planHolder:
+   *   The Connect request plan to execute.
+   */
+  def postStarted(planHolder: ExecutePlanHolder): Unit = {
+    val sc = sessionHolder.session.sparkContext
+    val request = planHolder.request
+    val plan: Message =
+      request.getPlan.getOpTypeCase match {
+        case proto.Plan.OpTypeCase.COMMAND => request.getPlan.getCommand
+        case proto.Plan.OpTypeCase.ROOT => request.getPlan.getRoot
+        case _ =>
+          throw new UnsupportedOperationException(
+            s"${request.getPlan.getOpTypeCase} not supported.")
+      }
+
+    sc.listenerBus.post(
+      SparkListenerConnectOperationStarted(
+        planHolder.jobGroupId,
+        planHolder.operationId,
+        clock.getTimeMillis(),
+        request.getSessionId,
+        request.getUserContext.getUserId,
+        request.getUserContext.getUserName,
+        Utils.redact(
+          sessionHolder.session.sessionState.conf.stringRedactionPattern,
+          ProtoUtils.abbreviate(plan, 
Events.MAX_STATEMENT_TEXT_SIZE).toString),
+        request.getClientType))
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationParsed to 
@link
+   * org.apache.spark.scheduler.LiveListenerBus.
+   *
+   * @param dataFrameOpt:
+   *   The Dataframe generated by the Connect request plan. None when the 
request does not
+   *   generate a plan.
+   */
+  def postParsed(dataFrameOpt: Option[DataFrame] = None): Unit = {
+    assertExecutedPlanPrepared(dataFrameOpt)
+    val jobGroupId = assertJobGroupId()
+    val operationId = assertOperationId(jobGroupId)
+    val event =
+      SparkListenerConnectOperationParsed(jobGroupId, operationId, 
clock.getTimeMillis())
+    event.analyzedPlan = dataFrameOpt.map(_.queryExecution.analyzed)
+    sessionHolder.session.sparkContext.listenerBus.post(event)
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationCanceled to
+   * \@link org.apache.spark.scheduler.LiveListenerBus.
+   *
+   * @param jobGroupIdOpt:
+   *   jobGroupId (@link org.apache.spark.SparkContext.setJobGroup) assigned 
by Connect for
+   *   request that was canceled. None when the jobGroupId is not set as a 
locale
+   */
+  def postCanceled(jobGroupIdOpt: Option[String] = None): Unit = {
+    val jobGroupId = jobGroupIdOpt.getOrElse(assertJobGroupId())
+    val operationId = assertOperationId(jobGroupId)
+    sessionHolder.session.sparkContext.listenerBus
+      .post(SparkListenerConnectOperationCanceled(jobGroupId, operationId, 
clock.getTimeMillis()))
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationFailed to 
@link
+   * org.apache.spark.scheduler.LiveListenerBus.
+   *
+   * @param errorMessage:
+   *   The message of the error thrown during the request.
+   */
+  def postFailed(errorMessage: String): Unit = {
+    val jobGroupId = assertJobGroupId()
+    val operationId = assertOperationId(jobGroupId)
+    sessionHolder.session.sparkContext.listenerBus.post(
+      SparkListenerConnectOperationFailed(
+        jobGroupId,
+        operationId,
+        clock.getTimeMillis(),
+        errorMessage))
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationParsed &
+   * @link
+   *   
org.apache.spark.sql.connect.service.SparkListenerConnectOperationFinished to 
@link
+   *   org.apache.spark.scheduler.LiveListenerBus.
+   *
+   * @param dataFrameOpt:
+   *   The Dataframe generated by the Connect request plan. None when the 
request does not
+   *   generate a plan.
+   */
+  def postParsedAndFinished(dataFrameOpt: Option[DataFrame] = None): Unit = {
+    postParsed(dataFrameOpt)
+    postFinished()
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationFinished to
+   * \@link org.apache.spark.scheduler.LiveListenerBus.
+   */
+  def postFinished(): Unit = {
+    val jobGroupId = assertJobGroupId()
+    val operationId = assertOperationId(jobGroupId)
+    sessionHolder.session.sparkContext.listenerBus
+      .post(SparkListenerConnectOperationFinished(jobGroupId, operationId, 
clock.getTimeMillis()))
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectOperationClosed to 
@link
+   * org.apache.spark.scheduler.LiveListenerBus.
+   */
+  def postClosed(): Unit = {
+    val jobGroupId = assertJobGroupId()
+    val operationId = assertOperationId(jobGroupId)
+    sessionHolder.session.sparkContext.listenerBus
+      .post(SparkListenerConnectOperationClosed(jobGroupId, operationId, 
clock.getTimeMillis()))
+  }
+
+  /**
+   * Post @link 
org.apache.spark.sql.connect.service.SparkListenerConnectSessionClosed to @link
+   * org.apache.spark.scheduler.LiveListenerBus.
+   */
+  def postSessionClosed(): Unit = {
+    sessionHolder.session.sparkContext.listenerBus
+      .post(SparkListenerConnectSessionClosed(sessionHolder.sessionId, 
clock.getTimeMillis()))
+  }
+
+  private def assertExecutedPlanPrepared(dataFrameOpt: Option[DataFrame]): 
Unit = {
+    dataFrameOpt.foreach { dataFrame =>
+      val isEagerlyExecuted = dataFrame.queryExecution.analyzed.find {
+        case _: Command => true
+        case _ => false
+      }.isDefined
+      val isStreaming = dataFrame.queryExecution.analyzed.isStreaming
+
+      if (!isEagerlyExecuted && !isStreaming) {
+        dataFrame.queryExecution.executedPlan
+      }
+    }
+  }
+
+  private def assertOperationId(jobGroupId: String): String = {
+    ExecutePlanHolder.getQueryOperationId(jobGroupId) match {
+      case Some(operationId) =>
+        operationId
+      case None =>
+        throw new RuntimeException(
+          s"Connect operationId cannot be resolved during sessionId: 
${sessionHolder.sessionId}")
+    }
+  }
+
+  private def assertJobGroupId(): String = {
+    getJobGroupId() match {
+      case Some(jobGroupId) =>
+        jobGroupId
+      case None =>
+        throw new RuntimeException(
+          s"Connect jobGroupId is not set during sessionId: 
${sessionHolder.sessionId}")
+    }
+  }
+
+  private def getJobGroupId(): Option[String] = {
+    Option(
+      sessionHolder.session.sparkContext
+        .getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID))
+  }
+}
+
+/**
+ * Event sent after reception of a Connect request that is ready for execution 
(i.e. not queued),
+ * but prior any analysis or execution.
+ *
+ * @param jobGroupId:
+ *   Opaque Spark jobGroupId (@link org.apache.spark.SparkContext.setJobGroup) 
assigned by Connect
+ *   during a request. Designed to be unique across sessions and requests.

Review Comment:
   note that in https://github.com/apache/spark/pull/41440 I intend to change 
it to a newly introduced "jobTag", as there can be only one jobGroupId, which 
conflicts with other internal uses...
   I also intend to add the ability to add more user-defined tags to an 
execution, so that users can cancel queries at per-tag scope. The user defined 
tags would be of the same form (the server will also enforce adding session id 
and user id to them), but then instead of generated UUID would have the 
user-provided part.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -83,8 +86,8 @@ class SparkConnectStreamHandler(responseObserver: 
StreamObserver[ExecutePlanResp
 
       try {
         v.getPlan.getOpTypeCase match {
-          case proto.Plan.OpTypeCase.COMMAND => handleCommand(session, v)
-          case proto.Plan.OpTypeCase.ROOT => handlePlan(session, v)
+          case proto.Plan.OpTypeCase.COMMAND => handleCommand(session, v, 
sessionHolder.events)
+          case proto.Plan.OpTypeCase.ROOT => handlePlan(session, v, 
sessionHolder.events)

Review Comment:
   I would be in favor of just passing SessionHolder / ExecutePlanHolder 
everywhere, instead of having to modify these parameters all the time.



-- 
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]

Reply via email to