Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions udf/worker/core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,11 @@
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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.udf.worker.core

import org.apache.spark.annotation.Experimental
import org.apache.spark.udf.worker.UDFWorkerSpecification

/**
* :: Experimental ::
* Creates [[WorkerDispatcher]] instances and controls their
* lifecycle after all sessions have closed.
*
* Implementations are passed to [[UDFDispatcherManager]] which
* handles caching, session tracking, and shutdown.
*/
@Experimental
trait UDFDispatcherFactory {

/**
* Creates a new [[WorkerDispatcher]] for the given specification.
* It is expected that creating the dispatcher
* itself is not slow while creating a session might be.
*/
def createDispatcher(
workerSpec: UDFWorkerSpecification,
logger: WorkerLogger): WorkerDispatcher

/**
* Called when the last active session for a dispatcher is closed.
* Implementations must decide what to do with the now-idle
* dispatcher: close it immediately, schedule idle-timeout
* eviction, etc.
* Not called during [[UDFDispatcherManager#stop]] -- the manager
* cleans up dispatchers it holds directly in that case.
*/
def onAllDispatcherSessionsClosed(
dispatcher: WorkerDispatcher): Unit

/**
* Called when the executor/driver stops. Implementations should
* clean up any dispatchers/resources they hold beyond what the
* [[UDFDispatcherManager]] manages.
*/
def onStop(): Unit
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,212 @@
/*
* 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.udf.worker.core

import java.util.{ArrayList, HashMap}

import org.apache.spark.annotation.Experimental
import org.apache.spark.udf.worker.UDFWorkerSpecification

/**
* :: Experimental :: Creates [[WorkerSession]] instances for a given
* [[UDFWorkerSpecification]], managing [[WorkerDispatcher]] instances and
* their lifecycle internally.
*
* Dispatchers are cached by spec (protobuf value equality) and reused across
* sessions. The manager tracks the number of active sessions per dispatcher
* via [[WorkerSession#addSessionCompletionListener]]. When the last session
* for a dispatcher is closed, the entry is removed and
* [[UDFDispatcherFactory#onAllDispatcherSessionsClosed]] is called.
*
* You might be wondering why the Dispatcher does not track the number of
* active sessions itself. The reason is that this would create a
* unavoidable race condition: Clients can provide different worker
* specs. Therefore, different dispatchers may be required, which cannot all
* exist for the whole Spark lifetime -> Dispatchers need to be removed/terminated
* at some point. If Dispatchers were to track their active sessions themselves
* and we would use this to decide on the dispatcher lifetime, it can always
* happen that there are concurrent [[createSession]] requests while
* the Dispatcher is being disposed off - which would create session
* initialization errors and may cause Spark task/query failures.
* Instead, we track the active sessions per Dispatcher globally
* in this manager.
*
* Thread safety: a single lock guards all state -- dispatchers, active
* sessions, and the stopping flag.
*/
@Experimental
class UDFDispatcherManager(
private val dispatcherFactory: UDFDispatcherFactory,
workerLogger: WorkerLogger = WorkerLogger.NoOp
) {

private val logger: WorkerLogger =
workerLogger.forClass(getClass)

/*
* Why do we need an [[activeSessionCount]] and an [[activeSessions]]
* list? [[activeSessionCount]] is per dispatcher. [[activeSessions]]
* is globally and allows us to perform session cleanup on [[stop]].
* Moreover, this distinction allows us to create sessions without
* requiring a lock on [[lock]].
*/
private class DispatcherEntry(val dispatcher: WorkerDispatcher) {
var activeSessionCount: Int = 0
}

// All fields below are guarded by `lock`.
private val lock = new Object
private val dispatchers =
new HashMap[UDFWorkerSpecification, DispatcherEntry]()
private val activeSessions = new ArrayList[WorkerSession]()
private var stopped = false

/**
* Creates a [[WorkerSession]] for the given worker specification and
* optional security scope.
*
* If a dispatcher for this spec already exists it is reused; otherwise
* [[UDFDispatcherFactory#createDispatcher]] is called to create one.
* A completion listener is registered on the session to track when
* it closes.
*/
final def createSession(
workerSpec: UDFWorkerSpecification,
securityScope: Option[WorkerSecurityScope] = None
): WorkerSession = {
// Get the dispatcher
val entry = lock.synchronized {
if (stopped) {
throwStopped()
}
getOrCreateDispatcherEntry(workerSpec)
}

// Create a new session (potentially slow -> outside the lock).
// Note: This might fail if Spark is concurrently being stopped
// and the dispatcher is cleaned up. As Spark is stopping,
// this failure is acceptable. On the happy path, no sessions
// should try to be created while Spark is shutting down.
val session = entry.dispatcher.createSession(securityScope)
lock.synchronized {
if (stopped) {
session.close()
throwStopped()
}
activeSessions.add(session)
}

logger.info(s"Created session ${session.sessionId}" +
s" on dispatcher ${entry.dispatcher.dispatcherId}" +
s" (active: ${entry.activeSessionCount})")

// Register a completion listener that updates the
// state when the session is canceled or closed
session.addSessionCompletionListener { session =>
logger.info(s"Session ${session.sessionId} terminated")
lock.synchronized {
if (!stopped) {
activeSessions.remove(session)
handleSessionTermination(workerSpec)
}
}
}

session
}

/**
* Called on driver/executor shutdown. Cancels any active sessions,
* closes all cached dispatchers, and resets internal state.
*
* Safety net -- in normal operation, sessions are closed
* by the physical Spark operators and dispatchers are cleaned up via
* [[UDFDispatcherFactory#onAllDispatcherSessionsClosed]] when their
* last session closes.
*/
final def stop(): Unit = {
logger.info("UDFDispatcherManager stopping" +
s" (${activeSessions.size()} active sessions," +
s" ${dispatchers.size()} dispatchers)")

lock.synchronized {
stopped = true

// Cancel any sessions that are still active. Cancel is a
// no-op if the session was already closed/cancelled.
activeSessions.forEach { session =>
logger.debug(s"Cancelling session ${session.sessionId}" +
" during stop")
session.cancel()
}
activeSessions.clear()

// Close all dispatchers we control.
// When spark is stopped in a clean state
// (only finished tasks), it is expected
// that all dispatchers have been terminated
// already. This is a safety-net.
dispatchers.forEach { (_, entry) =>
logger.debug(s"Closing dispatcher" +
s" ${entry.dispatcher.dispatcherId} during stop")
entry.dispatcher.close()
}
dispatchers.clear()
}

// Perform cleanup in the factory
dispatcherFactory.onStop()
logger.info("UDFDispatcherManager stopped")
}

private def throwStopped(): Nothing =
throw new IllegalStateException(
"UDFDispatcherManager is stopped")

// Must be called while holding `lock`.
private def handleSessionTermination(
workerSpec: UDFWorkerSpecification
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

maybe we shall also pass the session object here?

Copy link
Copy Markdown
Contributor Author

@sven-weber-db sven-weber-db May 7, 2026

Choose a reason for hiding this comment

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

Do you mean move the activeSessions.remove(session) call into this function like so?

private def handleSessionTermination(
     session: WorkerSession,
     workerSpec: UDFWorkerSpecification
  ): Unit = {
    activeSessions.remove(session)
    
    val entry = dispatchers.get(workerSpec)
    // Note: entry == null is unexpected and should
    // throw here.
    entry.activeSessionCount -= 1
    if (entry.activeSessionCount == 0) {
      logger.info("All sessions closed for dispatcher " +
        s"${entry.dispatcher.dispatcherId}, removing from cache")
      dispatchers.remove(workerSpec)
      onAllDispatcherSessionsClosed(entry.dispatcher)
    }
  }

): Unit = {
val entry = dispatchers.get(workerSpec)
// Note: entry == null is unexpected and should
// throw here.
entry.activeSessionCount -= 1
if (entry.activeSessionCount == 0) {
logger.info("All sessions closed for dispatcher " +
s"${entry.dispatcher.dispatcherId}, removing from cache")
dispatchers.remove(workerSpec)
dispatcherFactory.onAllDispatcherSessionsClosed(
entry.dispatcher)
}
}

// Must be called while holding `lock`.
private def getOrCreateDispatcherEntry(
workerSpec: UDFWorkerSpecification
): DispatcherEntry = {
var entry = dispatchers.get(workerSpec)
if (entry == null) {
val dispatcher = dispatcherFactory.createDispatcher(
workerSpec, logger)
logger.info(s"Created dispatcher ${dispatcher.dispatcherId}")
entry = new DispatcherEntry(dispatcher)
dispatchers.put(workerSpec, entry)
}
entry.activeSessionCount += 1
entry
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,10 @@ import org.apache.spark.udf.worker.UDFWorkerSpecification
@Experimental
trait WorkerDispatcher extends AutoCloseable {

/** Unique identifier for this dispatcher. */
val dispatcherId: String =
java.util.UUID.randomUUID().toString

def workerSpec: UDFWorkerSpecification

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,15 +36,43 @@ import org.apache.spark.annotation.Experimental
trait WorkerLogger {
def warn(msg: => String): Unit
def warn(msg: => String, t: Throwable): Unit
def info(msg: => String): Unit
def info(msg: => String, t: Throwable): Unit
def debug(msg: => String): Unit
def debug(msg: => String, t: Throwable): Unit

/**
* Returns a new [[WorkerLogger]] that prefixes every message with
* `[className]`. Useful for identifying which class produced a
* log line.
*/
def forClass(clazz: Class[_]): WorkerLogger = {
val prefix = s"[${clazz.getSimpleName}] "
val parent = this
new WorkerLogger {
override def warn(msg: => String): Unit =
parent.warn(prefix + msg)
override def warn(msg: => String, t: Throwable): Unit =
parent.warn(prefix + msg, t)
override def info(msg: => String): Unit =
parent.info(prefix + msg)
override def info(msg: => String, t: Throwable): Unit =
parent.info(prefix + msg, t)
override def debug(msg: => String): Unit =
parent.debug(prefix + msg)
override def debug(msg: => String, t: Throwable): Unit =
parent.debug(prefix + msg, t)
}
}
}

object WorkerLogger {
/** Discards all messages. Default for callers that don't wire up logging. */
val NoOp: WorkerLogger = new WorkerLogger {
override def warn(msg: => String): Unit = ()
override def warn(msg: => String, t: Throwable): Unit = ()
override def info(msg: => String): Unit = ()
override def info(msg: => String, t: Throwable): Unit = ()
override def debug(msg: => String): Unit = ()
override def debug(msg: => String, t: Throwable): Unit = ()
}
Expand Down
Loading