-
Notifications
You must be signed in to change notification settings - Fork 29.2k
[SPARK-56661] Implementing UDFDispatcherManager for new UDF worker sessions #55712
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
sven-weber-db
wants to merge
1
commit into
apache:master
Choose a base branch
from
sven-weber-db:sven-weber_data/SPARK-56661-udf-changes
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
59 changes: 59 additions & 0 deletions
59
udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/UDFDispatcherFactory.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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 | ||
| } |
212 changes: 212 additions & 0 deletions
212
udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/UDFDispatcherManager.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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 | ||
| ): 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 | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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?
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
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?