WweiL commented on code in PR #40937:
URL: https://github.com/apache/spark/pull/40937#discussion_r1177137568


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCache.scala:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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 java.util.concurrent.Executors
+import java.util.concurrent.ScheduledExecutorService
+import java.util.concurrent.TimeUnit
+import javax.annotation.concurrent.GuardedBy
+
+import scala.collection.mutable
+import scala.concurrent.duration.Duration
+import scala.concurrent.duration.DurationInt
+import scala.util.control.NonFatal
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.streaming.StreamingQuery
+import org.apache.spark.util.Clock
+import org.apache.spark.util.SystemClock
+
+/**
+ * Caches Spark-Connect streaming query references and the sessions. When a 
query is stopped (i.e.
+ * no longer active), it is cached for 1 hour so that it is accessible from 
the client side. It
+ * runs a background thread to run a periodic task that does the following:
+ *   - Check the status of the queries, and drops those that expired (1 hour 
after being stopped).
+ *   - Keep the associated session active by invoking supplied function 
`sessionKeepAliveFn`.
+ *
+ * This class helps with supporting following semantics for streaming query 
sessions:
+ *   - Keep the session and session mapping at connect server alive as long as 
a streaming query
+ *     is active. Even if the client side has disconnected.
+ *     - This matches how streaming queries behave in Spark. The queries 
continue to run if
+ *       notebook or job session is lost.
+ *   - Once a query is stopped, the reference and mappings are maintained for 
1 hour and will be
+ *     accessible from the client. This allows time for client to fetch status.
+ *     - During this time if the query is restarted (i.e. has a new run id), 
the reference to
+ *       previous run is dropped. As a result logical query has only the most 
recent query
+ *       reference cached. This policy can be revisited to cache multiple runs 
for a query.
+ *   - Note that these semantics are evolving and might change before being 
finalized in Connect.
+ *   - Future improvements:
+ *     - Provide an API for a users to access session even after they lose 
session id.
+ *       - Once a user is properly authenticated, the API could return list of 
sessions that are
+ *         still alive in the connect server for that user.
+ */
+private[connect] class SparkConnectStreamingQueryCache(
+    val sessionKeepAliveFn: (String, String) => Unit, // (userId, sessionId) 
=> Unit.
+    val clock: Clock = new SystemClock(),
+    private val stoppedQueryCachePeriod: Duration = 1.hour, // Configurable 
for testing.
+    private val sessionPollingPeriod: Duration = 1.minute // Configurable for 
testing.
+) extends Logging {
+
+  import SparkConnectStreamingQueryCache._
+
+  def registerNewStreamingQuery(sessionHolder: SessionHolder, query: 
StreamingQuery): Unit = {
+    queryCacheLock.synchronized {
+      val value = QueryCacheValue(
+        userId = sessionHolder.userId,
+        sessionId = sessionHolder.sessionId,
+        session = sessionHolder.session,
+        query = query,
+        expiresAtMs = None)
+
+      queryCache.put(QueryCacheKey(query.id.toString), value) match {
+        case Some(existing) => // Query is being replaced. Can happen when a 
query is restarted.
+          log.info(
+            s"Replacing existing query in the cache. Query Id: ${query.id}." +
+              s"Existing value $existing, new value $value.")
+        case None =>
+          log.info(s"Adding new query to the cache. Query Id ${query.id}, 
value $value.")
+      }
+
+      schedulePeriodicChecks() // Starts the scheduler thread if it hasn't 
started.

Review Comment:
   This method also seems to call `queryCacheLock.synchronized {`? Maybe we 
could remove that line in `schedulePeriodicChecks`?



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