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
27 changes: 17 additions & 10 deletions udf/worker/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ WorkerDispatcher -- manages workers, creates sessions
|
v
WorkerSession -- one UDF execution
| 1. session.init(InitMessage(payload, inputSchema, outputSchema))
| 1. session.init(Init proto: udf payload + data format + schemas)
| 2. val results = session.process(inputBatches)
| 3. session.close()
```
Expand All @@ -34,12 +34,13 @@ provisioning service or daemon).
```
udf/worker/
├── proto/
│ worker_spec.proto -- UDFWorkerSpecification protobuf (+ generated Java classes)
│ worker_spec.proto -- UDFWorkerSpecification protobuf
│ udf_protocol.proto -- UDF execution protocol (Init, UdfPayload, ...)
│ common.proto -- shared enums (UDFWorkerDataFormat, etc.)
└── core/ -- abstract interfaces
WorkerDispatcher.scala -- creates sessions, manages worker lifecycle
WorkerSession.scala -- per-UDF init/process/cancel/close + InitMessage
WorkerSession.scala -- per-UDF init/process/cancel/close
WorkerConnection.scala -- transport channel abstraction
WorkerSecurityScope.scala -- security boundary for worker pooling
Expand Down Expand Up @@ -76,10 +77,12 @@ Workers are terminated via SIGTERM/SIGKILL when the dispatcher is closed.

```scala
import org.apache.spark.udf.worker.{
DirectWorker, ProcessCallable, UDFProtoCommunicationPattern,
UDFWorkerDataFormat, UDFWorkerProperties, UDFWorkerSpecification,
UnixDomainSocket, WorkerCapabilities, WorkerConnectionSpec, WorkerEnvironment}
DirectWorker, Init, ProcessCallable, UdfPayload,
UDFProtoCommunicationPattern, UDFWorkerDataFormat, UDFWorkerProperties,
UDFWorkerSpecification, UnixDomainSocket, WorkerCapabilities,
WorkerConnectionSpec, WorkerEnvironment}
import org.apache.spark.udf.worker.core._
import com.google.protobuf.ByteString

// 1. Define a worker spec (direct creation mode).
val spec = UDFWorkerSpecification.newBuilder()
Expand Down Expand Up @@ -112,10 +115,14 @@ val dispatcher: WorkerDispatcher = ...
val session = dispatcher.createSession(securityScope = None)
try {
// 4. Initialize with the serialized function and schemas.
session.init(InitMessage(
functionPayload = serializedFunction,
inputSchema = arrowInputSchema,
outputSchema = arrowOutputSchema))
session.init(Init.newBuilder()
.setUdf(UdfPayload.newBuilder()
.setPayload(ByteString.copyFrom(serializedFunction))
.setFormat(payloadFormat)) // worker-recognised tag
.setDataFormat(UDFWorkerDataFormat.ARROW)
.setInputSchema(ByteString.copyFrom(arrowInputSchema))
.setOutputSchema(ByteString.copyFrom(arrowOutputSchema))
.build())

// 5. Process data -- Iterator in, Iterator out.
val results: Iterator[Array[Byte]] =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,31 +19,7 @@ package org.apache.spark.udf.worker.core
import java.util.concurrent.atomic.AtomicBoolean

import org.apache.spark.annotation.Experimental

/**
* :: Experimental ::
* Carries all information needed to initialize a UDF execution on a worker.
*
* This message is passed to [[WorkerSession#init]] and contains the function
* definition, schemas, and any additional configuration.
*
* Placeholder: will be replaced by a generated proto message once the
* UDF wire protocol lands. Do not rely on case-class equality --
* `Array[Byte]` fields compare by reference.
*
* @param functionPayload serialized function (e.g., pickled Python, JVM bytes)
* @param inputSchema serialized input schema (e.g., Arrow schema bytes)
* @param outputSchema serialized output schema (e.g., Arrow schema bytes)
* @param properties additional key-value configuration. Can carry
* protocol-specific or engine-specific metadata that
* does not yet have a dedicated field.
*/
@Experimental
case class InitMessage(
functionPayload: Array[Byte],
inputSchema: Array[Byte],
outputSchema: Array[Byte],
properties: Map[String, String] = Map.empty)
import org.apache.spark.udf.worker.Init

/**
* :: Experimental ::
Expand All @@ -62,7 +38,10 @@ case class InitMessage(
* {{{
* val session = dispatcher.createSession(securityScope = None)
* try {
* session.init(InitMessage(functionPayload, inputSchema, outputSchema))
* session.init(Init.newBuilder()
* .setUdf(UdfPayload.newBuilder().setPayload(callable).setFormat(fmt))
* .setDataFormat(UDFWorkerDataFormat.ARROW)
* .build())
* val results = session.process(inputBatches)
* results.foreach(handleBatch)
* } finally {
Expand All @@ -74,7 +53,18 @@ case class InitMessage(
* - [[init]] must be called exactly once before [[process]].
* - [[process]] must be called at most once per session.
* - [[close]] must always be called (use try-finally).
* - [[cancel]] may be called at any time to abort execution.
* - [[cancel]] may be called at any time, including before [[init]]
* or after [[process]]/[[close]] has returned. Implementations
* treat such calls as a no-op so that callers driven by a task
* interruption listener (which has no view into the session state)
* do not need to coordinate with the thread driving [[process]].
*
* Cancel-vs-finish race: when the session driver has finished
* sending input (and therefore queued an implicit finish on the
* underlying transport) and a [[cancel]] arrives concurrently, both
* are valid stream-terminating actions; the response side carries
* either a `FinishResponse` or a `CancelResponse` depending on which
* the worker observes first, and either is acceptable to the caller.
*
* The lifecycle is enforced here: [[init]] and [[process]] are `final`
* and delegate to [[doInit]] / [[doProcess]] after AtomicBoolean guards.
Expand All @@ -93,10 +83,12 @@ abstract class WorkerSession extends AutoCloseable {
*
* Throws `IllegalStateException` if called more than once.
*
* @param message the initialization parameters including the serialized
* function, input/output schemas, and configuration.
* @param message the [[Init]] proto carrying the UDF body, the wire
* data format, optional input/output schemas, and any
* engine-side session context the worker needs to start
* processing.
*/
final def init(message: InitMessage): Unit = {
final def init(message: Init): Unit = {
if (!initialized.compareAndSet(false, true)) {
throw new IllegalStateException("init has already been called on this session")
}
Expand Down Expand Up @@ -128,7 +120,7 @@ abstract class WorkerSession extends AutoCloseable {
}

/** Subclass hook for [[init]]. Called once, after the guard. */
protected def doInit(message: InitMessage): Unit
protected def doInit(message: Init): Unit

/** Subclass hook for [[process]]. Called at most once, after the guard. */
protected def doProcess(input: Iterator[Array[Byte]]): Iterator[Array[Byte]]
Expand All @@ -138,11 +130,29 @@ abstract class WorkerSession extends AutoCloseable {
*
* '''Thread-safety:''' implementations must allow [[cancel]] to be called
* from a thread different from the one driving [[process]] (typically a
* task interruption thread). It may be invoked at any point after
* [[init]] and should be a no-op if execution has already finished.
* task interruption thread).
*
* '''Lifecycle:''' [[cancel]] is idempotent and safe at any point in
* the session's life:
* - before [[init]] -- nothing has been sent on the transport yet,
* so [[cancel]] is a no-op (the session may still be closed
* normally via [[close]]).
* - between [[init]] and [[process]] -- transitions the session
* into a cancelled state; subsequent [[process]] calls observe
* the cancellation.
* - during [[process]] -- aborts the active stream.
* - after [[process]] / [[close]] has returned -- a no-op.
*
* Implementations are responsible for the no-op behavior described
* above so that callers (e.g. task interruption listeners) do not
* need to coordinate with the thread driving [[process]].
*/
def cancel(): Unit

/** Closes this session and releases resources. */
/**
* Closes this session and releases resources. Idempotent; safe to
* call from a `finally` block regardless of whether [[init]],
* [[process]], or [[cancel]] have been invoked.
*/
override def close(): Unit
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import org.scalatest.BeforeAndAfterEach
import org.scalatest.funsuite.AnyFunSuite

import org.apache.spark.udf.worker.{
DirectWorker, LocalTcpConnection, ProcessCallable, UDFWorkerProperties,
DirectWorker, Init, LocalTcpConnection, ProcessCallable, UDFWorkerProperties,
UDFWorkerSpecification, UnixDomainSocket, WorkerConnectionSpec,
WorkerEnvironment}
import org.apache.spark.udf.worker.core.direct.{DirectUnixSocketWorkerDispatcher,
Expand Down Expand Up @@ -58,7 +58,7 @@ class SocketFileConnection(socketPath: String)
class StubWorkerSession(
workerProcess: DirectWorkerProcess) extends DirectWorkerSession(workerProcess) {

override protected def doInit(message: InitMessage): Unit = {}
override protected def doInit(message: Init): Unit = {}

override protected def doProcess(
input: Iterator[Array[Byte]]): Iterator[Array[Byte]] =
Expand Down
4 changes: 2 additions & 2 deletions udf/worker/proto/src/main/protobuf/common.proto
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ option java_multiple_files = true;
// The UDF in & output data format.
enum UDFWorkerDataFormat {
UDF_WORKER_DATA_FORMAT_UNSPECIFIED = 0;

// The worker accepts and produces Apache arrow batches.
ARROW = 1;
}
Expand All @@ -42,7 +42,7 @@ enum UDFWorkerDataFormat {
enum UDFProtoCommunicationPattern {
UDF_PROTO_COMMUNICATION_PATTERN_UNSPECIFIED = 0;

// Data exachanged as a bidrectional
// Data exchanged as a bidirectional
// stream of bytes.
BIDIRECTIONAL_STREAMING = 1;
}
Loading