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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala:
##########
@@ -92,12 +92,15 @@ private[connect] class ExecuteThreadRunner(executeHolder: 
ExecuteHolder) extends
         "execute",
         executeHolder.responseObserver,
         executeHolder.sessionHolder.userId,
-        executeHolder.sessionHolder.sessionId)
+        executeHolder.sessionHolder.sessionId,
+        Some(executeHolder.events))
     }
   }
 
   // Inner executeInternal is wrapped by execute() for error handling.
   private def executeInternal() = {
+    // start event must be received before a cancel event from interrupt
+    executeHolder.events.postStarted()

Review Comment:
   if it fails to start a thread you may end up getting an error event thrown 
before.
   Comparing to Thriftserver, the started event there is outside the thread in 
SparkExecuteStatementOperation.runInternal



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala:
##########
@@ -44,6 +45,8 @@ case class SessionHolder(userId: String, sessionId: String, 
session: SparkSessio
   val executions: ConcurrentMap[String, ExecuteHolder] =
     new ConcurrentHashMap[String, ExecuteHolder]()
 
+  val events: SessionEvents = SessionEvents(this, new SystemClock())

Review Comment:
   naming nit: maybe eventManager: SessionEventsManager?
   "events" suggests that it buffers or stores the events, but it only manages 
getting them to the listener bus.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala:
##########
@@ -40,6 +40,8 @@ class SparkConnectExecutePlanHandler(responseObserver: 
StreamObserver[proto.Exec
         // Detached before execution finished.
         // TODO this doesn't happen yet without reattachable execution.
         responseObserver.onCompleted()
+      } else {
+        executeHolder.events.postClosed()
       }
     } finally {
       // TODO this will change with detachable execution.

Review Comment:
   for now just put it in the finally, since the detached code path can't 
happen yet (and e.g. `executeHolder.join()` should also not be happening for 
detached).
   this will change soon.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala:
##########
@@ -157,6 +161,7 @@ private[execution] class 
SparkConnectPlanExecution(executeHolder: ExecuteHolder)
 
             // Collect errors and propagate them to the main thread.
             future.onComplete { result =>
+              executePlan.events.postFinished()

Review Comment:
   in case of error, do you want to have error after finished (like you have 
here), or would you rather just post error and don't post finished? (as the 
query might have not finished before error).
   Thriftserver would post error without finished i believe.



##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/RequestEvents.scala:
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.connect.proto
+import org.apache.spark.scheduler.SparkListenerEvent
+import org.apache.spark.sql.catalyst.{QueryPlanningTracker, 
QueryPlanningTrackerCallback}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.connect.common.ProtoUtils
+import org.apache.spark.util.{Clock, Utils}
+
+object RequestEvents {
+  // TODO: Make this configurable
+  val MAX_STATEMENT_TEXT_SIZE = 65535
+}
+
+/**
+ * Post request Connect events to @link 
org.apache.spark.scheduler.LiveListenerBus.
+ *
+ * @param executeHolder:
+ *   Request for which the events are generated.
+ * @param clock:
+ *   Source of time for unit tests.
+ */
+case class RequestEvents(executeHolder: ExecuteHolder, clock: Clock) {

Review Comment:
   ditto maybe RequestEventsManager?
   or ExecuteEventsManager, given that we use "ExecuteHolder" for the request, 
and soon with detachable queries an execute will not be a single request.



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