From 2477fa3fc253fe6faa362c47317460a7a6950b47 Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Sun, 3 May 2026 16:12:48 +0000 Subject: [PATCH 01/13] grpc udf protocol --- .../proto/src/main/protobuf/common.proto | 4 +- .../src/main/protobuf/udf_protocol.proto | 459 ++++++++++++++++++ 2 files changed, 461 insertions(+), 2 deletions(-) create mode 100644 udf/worker/proto/src/main/protobuf/udf_protocol.proto diff --git a/udf/worker/proto/src/main/protobuf/common.proto b/udf/worker/proto/src/main/protobuf/common.proto index ee032def73efe..7028b13571874 100644 --- a/udf/worker/proto/src/main/protobuf/common.proto +++ b/udf/worker/proto/src/main/protobuf/common.proto @@ -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; } @@ -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; } diff --git a/udf/worker/proto/src/main/protobuf/udf_protocol.proto b/udf/worker/proto/src/main/protobuf/udf_protocol.proto new file mode 100644 index 0000000000000..a4d997c966752 --- /dev/null +++ b/udf/worker/proto/src/main/protobuf/udf_protocol.proto @@ -0,0 +1,459 @@ +/* + * 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. + */ + +syntax = "proto3"; + +import "common.proto"; + +package org.apache.spark.udf.worker; + +option java_package = "org.apache.spark.udf.worker"; +option java_multiple_files = true; + +// ===================================================================== +// Language-agnostic UDF execution protocol. +// +// The Spark engine acts as the gRPC client; a UDF worker (in any +// language) acts as the gRPC server. +// ===================================================================== + +// The default UDF gRPC service. A worker that exposes this service +// MUST do so over the default connection of the worker specification. +// +// In future, additional connections (e.g. a separate channel) may be +// reserved by the worker spec for other purposes. +service Worker { + // Per-execution stream. Exactly one [[Init]] is sent first, followed + // by 0..N data batches in either direction, terminated by exactly + // one [[Finish]] or [[Cancel]] from the engine. The worker MUST + // respond with the matching Init / Finish / Cancel responses on the + // response stream. + // + // For stateful execution, the state is maintained per bi-directional + // stream, mapping to a `WorkerSession` on the engine side + // (`org.apache.spark.udf.worker.core.WorkerSession`). + rpc Execute(stream UdfRequest) returns (stream UdfResponse); + + // Worker-scoped management RPC, independent of any per-execution + // stream. Used for heartbeat, capability query, and graceful + // shutdown. Kept unary so it does not depend on the lifecycle of an + // active Execute stream. + rpc Manage(WorkerRequest) returns (WorkerResponse); +} + +// ===================================================================== +// Execute stream -- envelope +// ===================================================================== + +// Engine -> Worker. Either a control message ([[Init]] / [[PayloadChunk]] +// / [[Finish]] / [[Cancel]]) or a data message. +message UdfRequest { + // Exactly one branch MUST be set; receivers MUST reject messages + // with no branch set. + oneof request { + UdfControlRequest control = 1; + DataRequest data = 2; + } +} + +// Worker -> Engine. Either a control response ([[InitResponse]] / +// [[FinishResponse]] / [[CancelResponse]]) or a data response message. +message UdfResponse { + // Exactly one branch MUST be set; receivers MUST reject messages + // with no branch set. + oneof response { + UdfControlResponse control = 1; + DataResponse data = 2; + } +} + +// Engine -> Worker control messages. +// +// Wire order on an Execute stream is exactly: +// Init { ... } +// PayloadChunk { ... }* // optional; 0..N chunks, only used when +// // the single UDF payload on Init is too +// // large to fit inline. +// ( DataRequest | )* +// Finish { ... } OR Cancel { ... } // exactly one terminator +// +// The worker MUST emit [[InitResponse]] before sending any +// [[DataResponse]], and MUST emit exactly one [[FinishResponse]] or +// [[CancelResponse]] before closing the response stream. +// +// A worker that receives messages out of this order (e.g. a second Init, +// a PayloadChunk after the first DataRequest, a DataRequest before Init, +// or a Cancel before Init) MUST close the stream with an error. +message UdfControlRequest { + // Exactly one branch MUST be set; receivers MUST reject messages + // with no branch set. + oneof control { + Init init = 1; + PayloadChunk payload = 2; + Finish finish = 3; + Cancel cancel = 4; + } +} + +// Worker -> Engine control messages. +message UdfControlResponse { + // Exactly one branch MUST be set; receivers MUST reject messages + // with no branch set. + oneof control { + InitResponse init = 1; + FinishResponse finish = 2; + CancelResponse cancel = 3; + } +} + +// ===================================================================== +// Init phase +// ===================================================================== + +// Sent once, as the first message on an Execute stream. Describes +// the UDF body to run plus the minimum metadata the worker needs to +// start processing it. +// +// Today the protocol mandates exactly one Init per UDF execution +// (one Init -> data -> Finish). This is the simplest contract and +// covers all currently supported UDF kinds. In the future we may +// evolve to support multiple init phases on the same stream -- e.g. +// when worker setup requires an interactive handshake (negotiate a +// schema, exchange capabilities, fetch driver-side metadata, ...) +// before the data plane opens. Such an extension would be additive +// and would not change the single-Init semantics already in use. +// +// Engine vs. client split: +// * Most fields on Init are engine-side. They describe what +// flows on the wire for this session ([[data_format]] / +// [[input_schema]] / [[output_schema]] -- matching the worker +// spec, not the function's view) and what per-session +// context the worker needs ([[timezone]], [[session_conf]], +// [[task_context]], [[parameters]]). +// * [[UdfPayload]] carries everything the client side of Spark +// (where the UDF is defined and serialized) packs -- the +// callable bytes themselves, plus optional custom encoders +// that override the worker's built-in decoders only when the +// UDF deals in types the worker doesn't already know how to +// convert (e.g. recovering Arrow batches into client-provided +// Scala case classes or other user-defined types). +message Init { + // (Required) Wire format used for [[DataRequest.data]] and + // [[DataResponse.data]] for the life of this session. Must be + // one of the formats the worker declared in + // [[WorkerCapabilities.supported_data_formats]]; the client side + // of the protocol picks one at planning time and sticks with it. + // Receivers MUST reject an Init whose [[data_format]] is + // `UDF_WORKER_DATA_FORMAT_UNSPECIFIED`. + UDFWorkerDataFormat data_format = 1; + + // (Required) The UDF body to execute on the worker for this + // session. Exactly one payload per Execute stream. + UdfPayload udf = 2; + + // (Optional) Schema of the input data plane in the wire format + // declared by [[data_format]] -- e.g. an Arrow IPC schema when + // data_format = ARROW. This is an engine-side requirement: it + // describes the bytes the engine will actually put on + // [[DataRequest.data]] for this session, matching what the + // worker advertised in its spec. It is NOT necessarily the + // schema the function definer expressed; the UDF's own type + // information lives inside [[UdfPayload]] (embedded in the + // payload, or as a structured [[UdfPayload.input_encoder]] + // that converts wire bytes into language-native values). + // + // Left unset when the worker can derive the schema from the + // payload alone. + optional bytes input_schema = 3; + + // (Optional) Schema of the output data plane in the wire format + // declared by [[data_format]]. Same semantics as + // [[input_schema]] -- engine-side requirement describing the + // bytes the engine expects on [[DataResponse.data]]. + optional bytes output_schema = 4; + + // (Optional; defaults to an empty map.) Per-task context + // provided by the engine. Common keys identify the task instance + // for diagnostics, logging, and stateful workers -- e.g. + // partition id, task attempt id, stage id, micro-batch id. + // Engine and worker agree on the keys they share; the protocol + // does not enumerate them. + map task_context = 5; + + // (Optional; defaults to an empty map.) Worker-private knobs not + // already captured by typed fields above. Free-form; both sides + // agree on the keys they need. + // + // Any key that two languages converge on is a candidate for + // promotion to a structured proto field -- once promoted, it gets + // a typed field number from the reserved range right after this + // block and is removed from [[session_conf]]. [[timezone]] below + // is an example of a key that has already been promoted. + map session_conf = 6; + + // (Optional) Session timezone, promoted out of [[session_conf]] + // because every eval needs it for timestamp encoding/decoding. + optional string timezone = 7; + + // Reserved for future typed Init fields, in particular keys + // graduated from [[session_conf]] (see the [[timezone]] precedent + // above). Numbers >= 100 are intentionally NOT reserved here; if + // a future revision needs an opaque escape-hatch field, give it a + // number >= 100 alongside [[parameters]] and add a field-level + // comment so the convention stays visible. + reserved 8 to 99; + + // (Optional) Engine-packed opaque parameters specific to a + // particular kind of UDF execution. The escape hatch for + // anything the engine needs the worker to see at init time + // that is not already captured by the typed fields above and + // does not fit naturally into [[task_context]]. The encoding + // is agreed between the engine and the worker; the protocol + // does not interpret it. The matching response, also opaque + // bytes, is returned via [[InitResponse.data]]. + // + // Numbers >= 100 are reserved by convention for opaque + // escape-hatch fields like this one; new typed fields use the + // reserved 8..99 range. + // + // Client-side init data (anything packed by the layer that + // defines and serializes the UDF) does NOT belong here -- it + // travels inside [[UdfPayload.payload]] instead. + optional bytes parameters = 100; +} + +// Acknowledgment for [[Init]]. The worker MUST send exactly one +// [[InitResponse]] before any [[DataResponse]]. +// +// The init phase allows the engine to interact with the UDF before +// data starts flowing -- the worker can return inline bytes here for +// the engine (or higher-level code on the engine side) to consume +// during setup. The semantics of those bytes are agreed between the +// client side of the protocol and the worker; this message itself is +// otherwise opaque. +message InitResponse { + // (Optional) Inline init result returned by the worker. Opaque + // to the protocol; the client side of the protocol and the + // worker agree on what (if anything) it carries. + optional bytes data = 1; +} + +// Optional. Used to stream the single UDF payload when it does not +// fit in a single gRPC message. The default is to send the payload +// inline on [[UdfPayload.payload]]; chunking is only needed when a +// payload exceeds the gRPC message size limit. +// +// When used, chunks are sent zero or more times after [[Init]] and +// before the first [[DataRequest]]. The worker concatenates the +// inline [[UdfPayload.payload]] (if any) followed by all chunks in +// arrival order to form the final payload. +message PayloadChunk { + // (Required, non-empty.) Bytes appended to the [[Init.udf]] + // payload. + bytes data = 1; + + // (Optional) Set to true on the final chunk. Receivers MAY use + // this as an early signal that the payload is complete and + // decoding can begin; receivers that prefer to wait for the + // first [[DataRequest]] (which marks the end of the chunking + // phase) MAY ignore this. When unset, the receiver determines + // completeness by the arrival of the first [[DataRequest]]. + optional bool last = 2; +} + +// ===================================================================== +// Data phase +// +// `data` is intentionally a top-level `bytes` field on both request +// and response messages -- not nested inside a wrapper -- so that +// implementations can avoid an extra copy when reading or writing +// the payload. The wire format (Arrow IPC etc.) is declared once per +// session via [[Init.data_format]] and stays the same for the life +// of the stream. +// ===================================================================== + +// Engine -> Worker per-batch payload. +message DataRequest { + // (Required, non-empty.) Encoded data bytes for one batch in the + // session-declared format. + bytes data = 1; +} + +// Worker -> Engine per-batch payload. The worker emits zero or more +// [[DataResponse]]s between [[InitResponse]] and [[FinishResponse]] / +// [[CancelResponse]]. Sink-style UDFs (which consume input but +// produce no output rows on the data plane) emit exactly zero. +message DataResponse { + // (Required, non-empty.) Encoded data bytes for one batch in the + // session-declared format. + bytes data = 1; +} + +// ===================================================================== +// Finish / Cancel phase +// ===================================================================== + +// Sent by the engine when no more input batches will arrive. The +// worker MUST drain any remaining output, then emit +// [[FinishResponse]] and close the response stream. +// +// Exactly one of [[Finish]] or [[Cancel]] is sent per Execute stream; +// they are mutually exclusive. If the engine has already sent +// [[Finish]] it MUST NOT send [[Cancel]] afterwards (and vice versa). +message Finish {} + +// Worker -> Engine completion message. May carry summary metrics. +message FinishResponse { + // Final metrics aggregated over the whole session (e.g. rows + // in/out, time per phase). Free-form; names are worker-defined. + map metrics = 1; + + // (Optional) Inline finish result returned by the worker. + // Mirrors [[InitResponse.data]] -- the finish phase allows the + // engine to interact with the UDF after data has stopped + // flowing, with the worker returning opaque bytes the engine (or + // higher-level code) may consume during teardown. The semantics + // of those bytes are agreed between the client side of the + // protocol and the worker. + optional bytes data = 2; +} + +// Engine -> Worker explicit cancel. Distinct from a gRPC stream error +// so the worker can run cleanup deterministically (release file +// handles, drop temp state, etc.). After receiving [[Cancel]] the +// worker MUST stop emitting [[DataResponse]] messages, run cleanup, +// and emit [[CancelResponse]] before closing. +// +// Exactly one of [[Finish]] or [[Cancel]] is sent per Execute stream; +// see [[Finish]]. [[Cancel]] is the cooperative cancellation path; +// gRPC-level stream errors are the involuntary fallback. If the +// stream breaks before [[CancelResponse]] arrives, the engine +// considers the worker uncancellable for this session and relies on +// process-level cleanup. +message Cancel { + // (Optional) Free-form reason for diagnostics. + optional string reason = 1; +} + +// Worker -> Engine acknowledgment of [[Cancel]]. +message CancelResponse {} + +// The single UDF body delivered to the worker on [[Init]]. Opaque to +// the engine: the engine forwards [[payload]] and [[format]] +// unchanged, and the worker decodes them per the format the client +// and worker have agreed on. +message UdfPayload { + // (Required, may be empty when chunked.) Serialized callable + // bundle, opaque to the engine. The encoding is declared in + // [[format]]. + // + // For payloads too large to fit on a single gRPC message, this + // field MAY be left empty (zero-length bytes) and the bytes + // delivered via the [[PayloadChunk]] mechanism instead. See + // [[PayloadChunk]] for chunking semantics. + bytes payload = 1; + + // (Required, non-empty.) Format tag identifying the encoding of + // [[payload]] (e.g. "py-cloudpickle-v3", "wasm-v1"). Engine does + // not interpret this; the client side of the protocol and the + // worker agree on its meaning. + string format = 2; + + // (Optional) Total payload size in bytes. Useful when chunked + // streaming is used so the worker can pre-allocate buffers. + optional int64 payload_size = 3; + + // (Optional) Human-readable name for diagnostics and metrics. + optional string name = 4; + + // (Optional) Worker / language-specific dispatch hint. A + // free-form string the worker uses to pick the code path that + // handles this payload. The protocol does not enumerate eval + // types because they are language-specific; the client side of + // the protocol and the worker agree on the namespace and the + // values. + // + // When the worker can derive the eval type from the payload + // itself (embedded metadata, format tag, etc.), this field is + // left unset. Otherwise the client side of the protocol sets it + // explicitly. + optional string eval_type = 5; + + // (Optional) Custom input encoder bytes. The worker already + // ships with built-in decoders for its standard types (e.g. a + // Python worker turns Arrow batches into pandas / pyarrow + // values out of the box; a JVM worker has its own defaults). + // Set this field only when the UDF needs a conversion the + // worker doesn't know about -- for example, recovering Arrow + // batches into client-provided Scala case classes, or any + // other user-defined type the function definer requires. + // + // Packed by the client side of the protocol; opaque to the + // wire protocol. Left unset whenever the worker's built-in + // decoders are sufficient. + optional bytes input_encoder = 6; + + // (Optional) Custom output encoder bytes. Mirror of + // [[input_encoder]]: set only when the UDF produces values the + // worker cannot convert to [[DataResponse.data]] using its + // built-in encoders, and the client side of the protocol needs + // to ship the conversion alongside the UDF. + optional bytes output_encoder = 7; +} + +// ===================================================================== +// Manage RPC -- worker-scoped operations independent of Execute +// ===================================================================== + +// Engine -> Worker. Wraps the manage operations in a oneof so the RPC +// is a single typed call, leaving room for future operations +// (capability query, profiling, ...). +message WorkerRequest { + // Exactly one branch MUST be set; receivers MUST reject messages + // with no branch set. + oneof manage { + Heartbeat heartbeat = 1; + ShutdownRequest shutdown = 2; + } +} + +// Worker -> Engine. +message WorkerResponse { + // Exactly one branch MUST be set, mirroring the request oneof. + oneof manage { + HeartbeatAck heartbeat = 1; + ShutdownResponse shutdown = 2; + } +} + +// Liveness probe. The engine may send this periodically to detect a +// hung worker. The worker SHOULD reply within a small bounded time. +message Heartbeat {} + +// Acknowledgment for [[Heartbeat]]. +message HeartbeatAck {} + +// Engine-initiated graceful shutdown request. Independent of SIGTERM +// (which is the OS-level fallback) -- this lets the worker know the +// engine has finished with it and intends no further Execute streams. +message ShutdownRequest { + // (Optional) Free-form reason for diagnostics. + optional string reason = 1; +} + +// Worker -> Engine acknowledgment of [[ShutdownRequest]]. +message ShutdownResponse {} From 0736d42f5cbf22d032cfd489617022cdcc8ac450 Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Mon, 4 May 2026 07:53:04 +0000 Subject: [PATCH 02/13] update README, remove InitMessage place holder. --- udf/worker/README.md | 27 +++++++----- .../spark/udf/worker/core/WorkerSession.scala | 41 +++++-------------- .../core/DirectWorkerDispatcherSuite.scala | 4 +- 3 files changed, 30 insertions(+), 42 deletions(-) diff --git a/udf/worker/README.md b/udf/worker/README.md index b843c430d0e04..707005721c693 100644 --- a/udf/worker/README.md +++ b/udf/worker/README.md @@ -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() ``` @@ -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 │ @@ -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() @@ -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("py-cloudpickle-v3")) + .setDataFormat(UDFWorkerDataFormat.ARROW) + .setInputSchema(ByteString.copyFrom(arrowInputSchema)) + .setOutputSchema(ByteString.copyFrom(arrowOutputSchema)) + .build()) // 5. Process data -- Iterator in, Iterator out. val results: Iterator[Array[Byte]] = diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala index f4c4091688c94..f122dbd6c0d21 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala @@ -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 :: @@ -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 { @@ -93,10 +72,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") } @@ -128,7 +109,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]] diff --git a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/DirectWorkerDispatcherSuite.scala b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/DirectWorkerDispatcherSuite.scala index 60f5e2211b702..43444ed89b9c6 100644 --- a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/DirectWorkerDispatcherSuite.scala +++ b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/DirectWorkerDispatcherSuite.scala @@ -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, @@ -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]] = From 973b037792420c4be0b7fc3bdd77ab6b7924aa7b Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Tue, 5 May 2026 08:22:56 +0000 Subject: [PATCH 03/13] address comments --- .../spark/udf/worker/core/WorkerSession.scala | 37 +++++++++-- .../src/main/protobuf/udf_protocol.proto | 65 +++++++++---------- 2 files changed, 63 insertions(+), 39 deletions(-) diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala index f122dbd6c0d21..f34c15459eb34 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala @@ -53,7 +53,18 @@ import org.apache.spark.udf.worker.Init * - [[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. @@ -119,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 } diff --git a/udf/worker/proto/src/main/protobuf/udf_protocol.proto b/udf/worker/proto/src/main/protobuf/udf_protocol.proto index a4d997c966752..c5ed5987a7812 100644 --- a/udf/worker/proto/src/main/protobuf/udf_protocol.proto +++ b/udf/worker/proto/src/main/protobuf/udf_protocol.proto @@ -146,11 +146,10 @@ message UdfControlResponse { // [[task_context]], [[parameters]]). // * [[UdfPayload]] carries everything the client side of Spark // (where the UDF is defined and serialized) packs -- the -// callable bytes themselves, plus optional custom encoders -// that override the worker's built-in decoders only when the -// UDF deals in types the worker doesn't already know how to -// convert (e.g. recovering Arrow batches into client-provided -// Scala case classes or other user-defined types). +// serialized callable, an opaque format tag, and any encoder +// metadata bundled with the callable. The wire protocol does +// not enumerate encoder shapes; that is left to the client and +// worker to agree on per UDF type. message Init { // (Required) Wire format used for [[DataRequest.data]] and // [[DataResponse.data]] for the life of this session. Must be @@ -172,9 +171,9 @@ message Init { // [[DataRequest.data]] for this session, matching what the // worker advertised in its spec. It is NOT necessarily the // schema the function definer expressed; the UDF's own type - // information lives inside [[UdfPayload]] (embedded in the - // payload, or as a structured [[UdfPayload.input_encoder]] - // that converts wire bytes into language-native values). + // information lives inside [[UdfPayload]], typically embedded + // alongside the callable in [[UdfPayload.payload]] (e.g. as + // input/output encoders chosen per UDF type). // // Left unset when the worker can derive the schema from the // payload alone. @@ -207,6 +206,13 @@ message Init { // (Optional) Session timezone, promoted out of [[session_conf]] // because every eval needs it for timestamp encoding/decoding. + // + // Format follows Spark's `spark.sql.session.timeZone` config -- + // typically an IANA TZ id (e.g. "America/Los_Angeles") or a + // fixed offset (e.g. "+08:00"). The engine MUST pass the value + // it would resolve from the session conf without further + // transformation, so the worker can interpret it the same way + // Spark does. optional string timezone = 7; // Reserved for future typed Init fields, in particular keys @@ -261,6 +267,11 @@ message InitResponse { // before the first [[DataRequest]]. The worker concatenates the // inline [[UdfPayload.payload]] (if any) followed by all chunks in // arrival order to form the final payload. +// +// Chunks are part of the Init handshake, not standalone control +// messages: they extend [[Init.udf.payload]] and are not +// individually acknowledged. The single [[InitResponse]] covers +// Init plus all of its chunks together. message PayloadChunk { // (Required, non-empty.) Bytes appended to the [[Init.udf]] // payload. @@ -357,9 +368,14 @@ message CancelResponse {} // unchanged, and the worker decodes them per the format the client // and worker have agreed on. message UdfPayload { - // (Required, may be empty when chunked.) Serialized callable - // bundle, opaque to the engine. The encoding is declared in - // [[format]]. + // (Required, may be empty when chunked.) Serialized UDF bundle, + // opaque to the engine. The encoding is declared in [[format]]. + // + // The bundle is not necessarily just the serialized callable; + // it is up to the client side of the protocol and the worker to + // agree on what is packed inside it -- e.g. custom encoders for + // user-defined types, type hints, or any other metadata the + // worker needs to invoke the UDF. // // For payloads too large to fit on a single gRPC message, this // field MAY be left empty (zero-length bytes) and the bytes @@ -392,27 +408,6 @@ message UdfPayload { // left unset. Otherwise the client side of the protocol sets it // explicitly. optional string eval_type = 5; - - // (Optional) Custom input encoder bytes. The worker already - // ships with built-in decoders for its standard types (e.g. a - // Python worker turns Arrow batches into pandas / pyarrow - // values out of the box; a JVM worker has its own defaults). - // Set this field only when the UDF needs a conversion the - // worker doesn't know about -- for example, recovering Arrow - // batches into client-provided Scala case classes, or any - // other user-defined type the function definer requires. - // - // Packed by the client side of the protocol; opaque to the - // wire protocol. Left unset whenever the worker's built-in - // decoders are sufficient. - optional bytes input_encoder = 6; - - // (Optional) Custom output encoder bytes. Mirror of - // [[input_encoder]]: set only when the UDF produces values the - // worker cannot convert to [[DataResponse.data]] using its - // built-in encoders, and the client side of the protocol needs - // to ship the conversion alongside the UDF. - optional bytes output_encoder = 7; } // ===================================================================== @@ -435,8 +430,8 @@ message WorkerRequest { message WorkerResponse { // Exactly one branch MUST be set, mirroring the request oneof. oneof manage { - HeartbeatAck heartbeat = 1; - ShutdownResponse shutdown = 2; + HeartbeatResponse heartbeat = 1; + ShutdownResponse shutdown = 2; } } @@ -445,7 +440,7 @@ message WorkerResponse { message Heartbeat {} // Acknowledgment for [[Heartbeat]]. -message HeartbeatAck {} +message HeartbeatResponse {} // Engine-initiated graceful shutdown request. Independent of SIGTERM // (which is the OS-level fallback) -- this lets the worker know the From 5c30dda52e738ad660c5169fd2b52931f4729eaa Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Thu, 7 May 2026 04:31:57 +0000 Subject: [PATCH 04/13] address comments. --- udf/worker/proto/src/main/protobuf/udf_protocol.proto | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/udf/worker/proto/src/main/protobuf/udf_protocol.proto b/udf/worker/proto/src/main/protobuf/udf_protocol.proto index c5ed5987a7812..00980b14a077a 100644 --- a/udf/worker/proto/src/main/protobuf/udf_protocol.proto +++ b/udf/worker/proto/src/main/protobuf/udf_protocol.proto @@ -384,9 +384,10 @@ message UdfPayload { bytes payload = 1; // (Required, non-empty.) Format tag identifying the encoding of - // [[payload]] (e.g. "py-cloudpickle-v3", "wasm-v1"). Engine does - // not interpret this; the client side of the protocol and the - // worker agree on its meaning. + // [[payload]]. The protocol does not enumerate the values: the + // client side of the protocol and the worker agree on the + // namespace, and each worker recognises the tags it knows how + // to decode. The engine forwards this string unchanged. string format = 2; // (Optional) Total payload size in bytes. Useful when chunked From 7ca4b62fbc73c4c2b998c442c635d006e266e8cc Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Thu, 7 May 2026 06:36:03 +0000 Subject: [PATCH 05/13] fix README --- udf/worker/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/udf/worker/README.md b/udf/worker/README.md index 707005721c693..8a0c12683a48a 100644 --- a/udf/worker/README.md +++ b/udf/worker/README.md @@ -118,7 +118,7 @@ try { session.init(Init.newBuilder() .setUdf(UdfPayload.newBuilder() .setPayload(ByteString.copyFrom(serializedFunction)) - .setFormat("py-cloudpickle-v3")) + .setFormat(payloadFormat)) // worker-recognised tag .setDataFormat(UDFWorkerDataFormat.ARROW) .setInputSchema(ByteString.copyFrom(arrowInputSchema)) .setOutputSchema(ByteString.copyFrom(arrowOutputSchema)) From 747f334b42a2dee981554f62813aea8b6968d97d Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Wed, 13 May 2026 08:43:07 +0000 Subject: [PATCH 06/13] address comments --- udf/worker/README.md | 20 +- .../udf/worker/core/WorkerDispatcher.scala | 9 + .../spark/udf/worker/core/WorkerSession.scala | 73 +++-- .../core/direct/DirectWorkerSession.scala | 2 +- .../core/DirectWorkerDispatcherSuite.scala | 4 +- .../src/main/protobuf/udf_protocol.proto | 252 ++++++++++++++---- 6 files changed, 282 insertions(+), 78 deletions(-) diff --git a/udf/worker/README.md b/udf/worker/README.md index 8a0c12683a48a..4aa5388858c17 100644 --- a/udf/worker/README.md +++ b/udf/worker/README.md @@ -56,6 +56,23 @@ worker creation where Spark spawns local OS processes. Future packages (e.g., `core/indirect/`) can implement alternative creation modes such as obtaining workers from a provisioning service or daemon. +## Wire protocol + +Each UDF execution uses a single bidirectional `Execute` gRPC stream: + +``` +Engine -> Worker: Init -> PayloadChunk* -> (DataRequest)* -> (Finish | Cancel) +Worker -> Engine: InitResponse -> (DataResponse)* -> (CodeError)? -> (FinishResponse | CancelResponse) +``` + +`DataRequest` and `DataResponse` are independent streams: the worker may emit +`DataResponse` messages at any point after `InitResponse`, including before the +first `DataRequest` arrives. Generator-style UDFs may have zero `DataRequest` +messages, with the engine sending `Finish` directly after `Init`. +`PayloadChunk.last = true` is the canonical end-of-chunking signal. +See `udf/worker/proto/src/main/protobuf/udf_protocol.proto` for the complete +ordering invariants, gRPC error contract, and cancel-vs-finish race contract. + ### Direct worker creation `DirectWorkerDispatcher` spawns worker processes locally. On the first @@ -118,7 +135,8 @@ try { session.init(Init.newBuilder() .setUdf(UdfPayload.newBuilder() .setPayload(ByteString.copyFrom(serializedFunction)) - .setFormat(payloadFormat)) // worker-recognised tag + .setFormat(payloadFormat) // worker-recognised tag + .build()) .setDataFormat(UDFWorkerDataFormat.ARROW) .setInputSchema(ByteString.copyFrom(arrowInputSchema)) .setOutputSchema(ByteString.copyFrom(arrowOutputSchema)) diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerDispatcher.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerDispatcher.scala index 008cfc2993a09..8218a2f8b30eb 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerDispatcher.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerDispatcher.scala @@ -27,6 +27,15 @@ import org.apache.spark.udf.worker.UDFWorkerSpecification * as security scope). It owns the underlying worker processes and connections, * handling pooling, reuse, and lifecycle behind the scenes. Spark interacts with * workers exclusively through the [[WorkerSession]]s returned by [[createSession]]. + * + * '''Worker invalidation:''' if a session's Execute stream terminates with a gRPC + * transport error the worker that backed it MUST NOT be returned to any reuse pool. + * A transport error leaves the worker in an unknown state; only workers that + * complete sessions cleanly (via [[org.apache.spark.udf.worker.FinishResponse]] or + * [[org.apache.spark.udf.worker.CancelResponse]]) are eligible for reuse. + * Implementations are responsible for tracking this condition -- typically + * [[WorkerSession.doProcess]] flags the worker as invalid before [[WorkerSession.doClose]] + * releases it, so the dispatcher can distinguish a clean release from a failed one. */ @Experimental trait WorkerDispatcher extends AutoCloseable { diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala index f34c15459eb34..57544f3f561c5 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala @@ -39,7 +39,7 @@ import org.apache.spark.udf.worker.Init * val session = dispatcher.createSession(securityScope = None) * try { * session.init(Init.newBuilder() - * .setUdf(UdfPayload.newBuilder().setPayload(callable).setFormat(fmt)) + * .setUdf(UdfPayload.newBuilder().setPayload(callable).setFormat(fmt).build()) * .setDataFormat(UDFWorkerDataFormat.ARROW) * .build()) * val results = session.process(inputBatches) @@ -59,12 +59,9 @@ import org.apache.spark.udf.worker.Init * 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 cancel-vs-finish race contract is defined in the `Finish` + * message of `udf_protocol.proto`. Implementations MUST resolve + * that race so the transport never observes both on the same stream. * * The lifecycle is enforced here: [[init]] and [[process]] are `final` * and delegate to [[doInit]] / [[doProcess]] after AtomicBoolean guards. @@ -119,10 +116,29 @@ abstract class WorkerSession extends AutoCloseable { doProcess(input) } - /** Subclass hook for [[init]]. Called once, after the guard. */ + /** + * Subclass hook for [[init]]. Called once, after the guard. + * Implementations MUST NOT open the Execute gRPC stream before + * this call: [[cancel]] before [[init]] is contractually a no-op + * at the transport level, which only holds if no stream has been + * opened yet. + */ protected def doInit(message: Init): Unit - /** Subclass hook for [[process]]. Called at most once, after the guard. */ + /** + * Subclass hook for [[process]]. Called at most once, after the guard. + * + * If the Execute stream terminates with a gRPC transport error (i.e. + * the connection broke rather than the worker sending a protocol + * response), the implementation MUST: + * - throw an appropriate exception so the caller observes a failure + * rather than a silent empty result; and + * - ensure the underlying worker is not returned to any reuse pool, + * since a transport error leaves the worker in an unknown state. + * Implementations signal this to the [[WorkerDispatcher]] via + * whatever mechanism the dispatcher provides (e.g. flagging the + * worker as invalid before calling [[doClose]]). + */ protected def doProcess(input: Iterator[Array[Byte]]): Iterator[Array[Byte]] /** @@ -134,18 +150,19 @@ abstract class WorkerSession extends AutoCloseable { * * '''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. + * - before [[init]] -- a no-op; the session may still be closed + * normally via [[close]]. + * - between [[init]] and [[process]] -- signals that the session + * should be terminated; the caller should not invoke [[process]] + * and should call [[close]] to release resources. * - 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]]. + * Implementations are responsible for the lifecycle-aware behavior + * described above (no-op outside the active window; cancellation + * thrown from a subsequent [[process]] when applicable) so that + * callers (e.g. task interruption listeners) do not need to + * coordinate with the thread driving [[process]]. */ def cancel(): Unit @@ -153,6 +170,24 @@ abstract class WorkerSession extends AutoCloseable { * Closes this session and releases resources. Idempotent; safe to * call from a `finally` block regardless of whether [[init]], * [[process]], or [[cancel]] have been invoked. + * + * If [[init]] was called but [[process]] was not (e.g. an exception + * was thrown between the two), [[close]] sends `Cancel` on the + * Execute stream before releasing resources, so the worker can clean + * up deterministically rather than observing a gRPC transport error. + * Subclasses implement [[doClose]] for resource teardown; the base + * class handles the cancel-before-close guarantee automatically. + */ + final override def close(): Unit = { + if (initialized.get() && !processed.get()) { + cancel() + } + doClose() + } + + /** Subclass hook for [[close]]. The base class guarantees that + * [[cancel]] has already been called if [[init]] was invoked but + * [[process]] was not. */ - override def close(): Unit + protected def doClose(): Unit } diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerSession.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerSession.scala index 7cdc5329350e3..5de1f486e832e 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerSession.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerSession.scala @@ -48,7 +48,7 @@ abstract class DirectWorkerSession( /** The connection to the worker for this session. */ def connection: WorkerConnection = workerProcess.connection - override def close(): Unit = { + override protected def doClose(): Unit = { if (released.compareAndSet(false, true)) { workerProcess.releaseSession() } diff --git a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/DirectWorkerDispatcherSuite.scala b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/DirectWorkerDispatcherSuite.scala index 43444ed89b9c6..7302c697d93c0 100644 --- a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/DirectWorkerDispatcherSuite.scala +++ b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/DirectWorkerDispatcherSuite.scala @@ -51,8 +51,8 @@ class SocketFileConnection(socketPath: String) * TODO: [[cancel]] is a no-op here. Once a concrete [[DirectWorkerSession]] * with real data-plane wiring lands, add tests exercising cancel() in * particular: cancel from a different thread than process(), cancel - * after process() has returned, and cancel before init (should be a - * no-op). Tracking the thread-safety contract in the docstring on + * after process() has returned, and cancel before init (should be a no-op). + * See the thread-safety contract in the docstring on * [[org.apache.spark.udf.worker.core.WorkerSession.cancel]]. */ class StubWorkerSession( diff --git a/udf/worker/proto/src/main/protobuf/udf_protocol.proto b/udf/worker/proto/src/main/protobuf/udf_protocol.proto index 00980b14a077a..9e88eddb308e8 100644 --- a/udf/worker/proto/src/main/protobuf/udf_protocol.proto +++ b/udf/worker/proto/src/main/protobuf/udf_protocol.proto @@ -36,16 +36,19 @@ option java_multiple_files = true; // // In future, additional connections (e.g. a separate channel) may be // reserved by the worker spec for other purposes. -service Worker { - // Per-execution stream. Exactly one [[Init]] is sent first, followed - // by 0..N data batches in either direction, terminated by exactly - // one [[Finish]] or [[Cancel]] from the engine. The worker MUST - // respond with the matching Init / Finish / Cancel responses on the - // response stream. +service UdfWorker { + // Per-execution stream. See [[UdfControlRequest]] for the complete + // wire protocol and ordering invariants. + // + // Error contract: if the gRPC connection breaks at any point, gRPC + // surfaces an error on the stream. The engine therefore never needs + // to poll or time out waiting for a response -- the absence of a + // gRPC error guarantees that a proper protocol response will + // eventually arrive. This applies to every in-flight response, not + // only [[CancelResponse]]. // // For stateful execution, the state is maintained per bi-directional - // stream, mapping to a `WorkerSession` on the engine side - // (`org.apache.spark.udf.worker.core.WorkerSession`). + // stream, mapping to a `WorkerSession` on the engine side. rpc Execute(stream UdfRequest) returns (stream UdfResponse); // Worker-scoped management RPC, independent of any per-execution @@ -71,7 +74,8 @@ message UdfRequest { } // Worker -> Engine. Either a control response ([[InitResponse]] / -// [[FinishResponse]] / [[CancelResponse]]) or a data response message. +// [[FinishResponse]] / [[CancelResponse]] / [[CodeError]]) or a +// data response message. message UdfResponse { // Exactly one branch MUST be set; receivers MUST reject messages // with no branch set. @@ -83,19 +87,31 @@ message UdfResponse { // Engine -> Worker control messages. // -// Wire order on an Execute stream is exactly: -// Init { ... } -// PayloadChunk { ... }* // optional; 0..N chunks, only used when -// // the single UDF payload on Init is too -// // large to fit inline. -// ( DataRequest | )* -// Finish { ... } OR Cancel { ... } // exactly one terminator +// Wire protocol for one Execute stream (both directions): +// +// Engine -> Worker: Init -> PayloadChunk* -> (DataRequest)* -> (Finish | Cancel) +// Worker -> Engine: InitResponse -> (DataResponse)* -> (CodeError)? -> (FinishResponse | CancelResponse) +// +// DataRequest and DataResponse are independent streams: the worker +// may emit DataResponse messages at any point after InitResponse, +// including before the first DataRequest arrives. For generator-style +// UDFs that produce output without consuming input, there may be zero +// DataRequest messages -- the engine sends Finish directly after Init. +// The arrows above denote ordering constraints within each direction, +// not a request/response pairing. // -// The worker MUST emit [[InitResponse]] before sending any -// [[DataResponse]], and MUST emit exactly one [[FinishResponse]] or -// [[CancelResponse]] before closing the response stream. +// Ordering invariants: +// - PayloadChunk* only after Init and before the first DataRequest; +// [[PayloadChunk.last]] = true is the canonical end-of-chunking signal. +// - InitResponse MUST be emitted before any DataResponse. +// - CodeError (if any) MUST be emitted after all DataResponse +// messages and before FinishResponse. It is always followed by +// FinishResponse (not CancelResponse) so the stream terminates cleanly. +// - Exactly one terminator pair: Finish -> FinishResponse OR +// Cancel -> CancelResponse (never both on the same stream; see [[Finish]] +// for the cancel-vs-finish race contract on the engine side). // -// A worker that receives messages out of this order (e.g. a second Init, +// A worker that receives messages out of order (e.g. a second Init, // a PayloadChunk after the first DataRequest, a DataRequest before Init, // or a Cancel before Init) MUST close the stream with an error. message UdfControlRequest { @@ -117,6 +133,7 @@ message UdfControlResponse { InitResponse init = 1; FinishResponse finish = 2; CancelResponse cancel = 3; + CodeError error = 4; } } @@ -151,18 +168,30 @@ message UdfControlResponse { // not enumerate encoder shapes; that is left to the client and // worker to agree on per UDF type. message Init { + // (Optional) Protocol version declared by the engine for this stream. + // Allows the worker to detect version mismatches early and reject + // streams using a protocol revision it does not support. When not set, + // the worker SHOULD assume the initial protocol version. + optional uint32 protocol_version = 1; + // (Required) Wire format used for [[DataRequest.data]] and // [[DataResponse.data]] for the life of this session. Must be // one of the formats the worker declared in // [[WorkerCapabilities.supported_data_formats]]; the client side // of the protocol picks one at planning time and sticks with it. - // Receivers MUST reject an Init whose [[data_format]] is - // `UDF_WORKER_DATA_FORMAT_UNSPECIFIED`. - UDFWorkerDataFormat data_format = 1; + // + // Workers MUST reject an [[Init]] whose [[data_format]] is + // `UDF_WORKER_DATA_FORMAT_UNSPECIFIED`, or whose value is not + // present in their declared + // [[WorkerCapabilities.supported_data_formats]]. The latter covers + // unknown enum values that proto3 passes through as numeric + // constants -- e.g. a newer engine selecting a format the worker + // does not implement. + UDFWorkerDataFormat data_format = 2; // (Required) The UDF body to execute on the worker for this // session. Exactly one payload per Execute stream. - UdfPayload udf = 2; + UdfPayload udf = 3; // (Optional) Schema of the input data plane in the wire format // declared by [[data_format]] -- e.g. an Arrow IPC schema when @@ -177,13 +206,13 @@ message Init { // // Left unset when the worker can derive the schema from the // payload alone. - optional bytes input_schema = 3; + optional bytes input_schema = 4; // (Optional) Schema of the output data plane in the wire format // declared by [[data_format]]. Same semantics as // [[input_schema]] -- engine-side requirement describing the // bytes the engine expects on [[DataResponse.data]]. - optional bytes output_schema = 4; + optional bytes output_schema = 5; // (Optional; defaults to an empty map.) Per-task context // provided by the engine. Common keys identify the task instance @@ -191,7 +220,7 @@ message Init { // partition id, task attempt id, stage id, micro-batch id. // Engine and worker agree on the keys they share; the protocol // does not enumerate them. - map task_context = 5; + map task_context = 6; // (Optional; defaults to an empty map.) Worker-private knobs not // already captured by typed fields above. Free-form; both sides @@ -202,7 +231,7 @@ message Init { // a typed field number from the reserved range right after this // block and is removed from [[session_conf]]. [[timezone]] below // is an example of a key that has already been promoted. - map session_conf = 6; + map session_conf = 7; // (Optional) Session timezone, promoted out of [[session_conf]] // because every eval needs it for timestamp encoding/decoding. @@ -213,7 +242,7 @@ message Init { // it would resolve from the session conf without further // transformation, so the worker can interpret it the same way // Spark does. - optional string timezone = 7; + optional string timezone = 8; // Reserved for future typed Init fields, in particular keys // graduated from [[session_conf]] (see the [[timezone]] precedent @@ -221,7 +250,7 @@ message Init { // a future revision needs an opaque escape-hatch field, give it a // number >= 100 alongside [[parameters]] and add a field-level // comment so the convention stays visible. - reserved 8 to 99; + reserved 9 to 99; // (Optional) Engine-packed opaque parameters specific to a // particular kind of UDF execution. The escape hatch for @@ -234,7 +263,7 @@ message Init { // // Numbers >= 100 are reserved by convention for opaque // escape-hatch fields like this one; new typed fields use the - // reserved 8..99 range. + // reserved 9..99 range. // // Client-side init data (anything packed by the layer that // defines and serializes the UDF) does NOT belong here -- it @@ -243,7 +272,9 @@ message Init { } // Acknowledgment for [[Init]]. The worker MUST send exactly one -// [[InitResponse]] before any [[DataResponse]]. +// [[InitResponse]] before any [[DataResponse]]. When [[PayloadChunk]] +// is used to deliver the UDF payload, the worker MUST also wait until +// end-of-chunking to emit it (see [[PayloadChunk]]). // // The init phase allows the engine to interact with the UDF before // data starts flowing -- the worker can return inline bytes here for @@ -271,18 +302,28 @@ message InitResponse { // Chunks are part of the Init handshake, not standalone control // messages: they extend [[Init.udf.payload]] and are not // individually acknowledged. The single [[InitResponse]] covers -// Init plus all of its chunks together. +// Init plus all of its chunks together. [[PayloadChunk.last]] = true +// is the canonical end-of-chunking signal; the worker MUST NOT send +// [[InitResponse]] before receiving it. +// +// When [[UdfPayload.payload_size]] is set on [[Init.udf]], receivers +// MAY validate that the total assembled payload (inline +// [[UdfPayload.payload]] bytes plus all chunk bytes) matches it; a +// mismatch is a protocol error. message PayloadChunk { // (Required, non-empty.) Bytes appended to the [[Init.udf]] // payload. bytes data = 1; - // (Optional) Set to true on the final chunk. Receivers MAY use - // this as an early signal that the payload is complete and - // decoding can begin; receivers that prefer to wait for the - // first [[DataRequest]] (which marks the end of the chunking - // phase) MAY ignore this. When unset, the receiver determines - // completeness by the arrival of the first [[DataRequest]]. + // Marks the final chunk. When the engine uses [[PayloadChunk]] + // at all, it MUST set `last = true` on the last chunk. This is + // the canonical end-of-chunking signal: the worker MUST wait for + // it before emitting [[InitResponse]] and before treating any + // subsequent message as a [[DataRequest]]. Non-final chunks omit + // this field. + // + // Kept `optional` so future revisions can distinguish "engine did + // not set this field" from "engine set false" without renumbering. optional bool last = 2; } @@ -299,8 +340,12 @@ message PayloadChunk { // Engine -> Worker per-batch payload. message DataRequest { - // (Required, non-empty.) Encoded data bytes for one batch in the - // session-declared format. + // (Required) Encoded data bytes for one batch in the + // session-declared format. What "empty" means for a batch is + // defined by the session's [[Init.data_format]] -- for Arrow IPC + // even a zero-row batch carries a non-empty header, while future + // formats may permit truly zero-length payloads. Validation + // beyond "non-empty bytes" is delegated to the format decoder. bytes data = 1; } @@ -309,8 +354,9 @@ message DataRequest { // [[CancelResponse]]. Sink-style UDFs (which consume input but // produce no output rows on the data plane) emit exactly zero. message DataResponse { - // (Required, non-empty.) Encoded data bytes for one batch in the - // session-declared format. + // (Required) Encoded data bytes for one batch in the + // session-declared format. See [[DataRequest.data]] for the + // meaning of "empty". bytes data = 1; } @@ -325,6 +371,16 @@ message DataResponse { // Exactly one of [[Finish]] or [[Cancel]] is sent per Execute stream; // they are mutually exclusive. If the engine has already sent // [[Finish]] it MUST NOT send [[Cancel]] afterwards (and vice versa). +// +// The mutual-exclusion rule is a wire-level contract: workers never +// observe both messages on the same Execute stream. Engine-side +// implementations that drive [[Finish]] and [[Cancel]] from different +// threads (e.g. a session driver vs. a task-interruption listener) are +// responsible for ensuring that at most one of the two reaches the +// transport. A typical implementation tracks whether [[Finish]] has +// already been written and, if so, suppresses any subsequent [[Cancel]] +// (relying on [[FinishResponse]] to terminate the stream); equivalently, +// a [[Cancel]] that wins the race suppresses the implicit [[Finish]]. message Finish {} // Worker -> Engine completion message. May carry summary metrics. @@ -350,11 +406,17 @@ message FinishResponse { // and emit [[CancelResponse]] before closing. // // Exactly one of [[Finish]] or [[Cancel]] is sent per Execute stream; -// see [[Finish]]. [[Cancel]] is the cooperative cancellation path; -// gRPC-level stream errors are the involuntary fallback. If the -// stream breaks before [[CancelResponse]] arrives, the engine -// considers the worker uncancellable for this session and relies on -// process-level cleanup. +// see [[Finish]]. [[Cancel]] is the cooperative cancellation path; a +// broken gRPC connection is the involuntary fallback -- in that case +// gRPC surfaces an error on the stream (see the error contract on +// [[Execute]]). +// +// Worker behavior on involuntary stream error: when the worker observes +// a gRPC error on the Execute stream (i.e. the engine-side connection +// dropped), it MUST treat this as equivalent to [[Cancel]] for cleanup +// purposes -- stop producing output and release resources. The worker +// MUST NOT attempt to send [[CancelResponse]] or any other message, +// since the stream is already dead. message Cancel { // (Optional) Free-form reason for diagnostics. optional string reason = 1; @@ -363,6 +425,41 @@ message Cancel { // Worker -> Engine acknowledgment of [[Cancel]]. message CancelResponse {} +// Worker -> Engine. Signals a non-gRPC error during execution: +// either user code raised an exception, or the worker implementation +// itself detected an application-level error it wants to surface to +// the engine. Distinct from a gRPC stream error: gRPC errors indicate +// transport or protocol failures, whereas [[CodeError]] carries +// application-level errors that should be raised as user-facing +// exceptions rather than transport failures. +// +// Wire position: emitted at most once, after all [[DataResponse]] +// messages and before [[FinishResponse]]. [[FinishResponse]] is still +// sent after [[CodeError]] so the stream terminates cleanly. +// The engine MUST raise a user-facing exception (not a protocol error) +// upon receiving this message. +// +// Cancel interaction: if [[Cancel]] arrives while the worker is +// preparing to send [[CodeError]], the cancel-vs-finish race rules +// in [[Finish]] apply. The worker emits exactly one terminator: +// [[FinishResponse]] (if [[CodeError]] was already sent or the +// worker chooses to complete) or [[CancelResponse]] (if [[Cancel]] +// wins the race before [[CodeError]] is sent). +message CodeError { + // (Required) Human-readable error message from the UDF. + string message = 1; + + // (Optional) Full stack trace or traceback in the worker's + // language-specific format. Forwarded to the user as-is for + // diagnostics. + optional string traceback = 2; + + // (Optional) Language-specific error class name (e.g. "ValueError" + // in Python, "RuntimeException" in Java). Allows the engine to + // surface a more informative exception type to the user. + optional string error_class = 3; +} + // The single UDF body delivered to the worker on [[Init]]. Opaque to // the engine: the engine forwards [[payload]] and [[format]] // unchanged, and the worker decodes them per the format the client @@ -429,7 +526,10 @@ message WorkerRequest { // Worker -> Engine. message WorkerResponse { - // Exactly one branch MUST be set, mirroring the request oneof. + // Exactly one branch MUST be set, and it MUST be the branch + // matching the request's oneof (e.g. [[Heartbeat]] is answered + // with [[HeartbeatResponse]], not [[ShutdownResponse]]). A + // mismatched response is a protocol error. oneof manage { HeartbeatResponse heartbeat = 1; ShutdownResponse shutdown = 2; @@ -437,15 +537,57 @@ message WorkerResponse { } // Liveness probe. The engine may send this periodically to detect a -// hung worker. The worker SHOULD reply within a small bounded time. -message Heartbeat {} +// hung worker process. The worker SHOULD reply within a small bounded +// time. +// +// This is an application-level liveness check distinct from gRPC's +// transport-level keepalive: gRPC keepalive proves the TCP connection +// is alive, whereas [[Heartbeat]] proves the worker's request-handling +// thread is responsive. Deployments may use either or both; they do +// not replace each other. +// +// What the engine does in response to a missed heartbeat (e.g., +// tearing down the worker) is outside the scope of this protocol and +// depends on the worker management mode defined in the worker +// specification. +// +// Note: [[Heartbeat]] can only detect a hung worker process (one +// whose request-handling thread is unresponsive). It cannot detect +// user code that is executing but taking unexpectedly long -- during +// init, data processing, or finish -- because such code is +// indistinguishable from legitimately slow UDF execution. Handling +// hanging user code (e.g. via UDF-level timeouts) is the +// responsibility of the UDF author or the worker implementation, and +// is outside the scope of this protocol. +message Heartbeat { + // Reserved for future additive fields (e.g. an engine-side + // sequence number or a request-id tag for correlating heartbeats + // when sent over a long-lived connection). + reserved 1; +} // Acknowledgment for [[Heartbeat]]. -message HeartbeatResponse {} +message HeartbeatResponse { + // Reserved for future additive fields (e.g. a server-side load + // hint to inform scheduling decisions, or a matching sequence + // number echoed from [[Heartbeat]]). + reserved 1; +} -// Engine-initiated graceful shutdown request. Independent of SIGTERM -// (which is the OS-level fallback) -- this lets the worker know the -// engine has finished with it and intends no further Execute streams. +// Engine-initiated graceful shutdown request. This lets the worker +// know the engine has finished with it and intends no further Execute +// streams. OS-level process management is outside the scope of this +// protocol and is defined in the worker specification. +// +// Interaction with in-flight Execute streams: [[ShutdownRequest]] is a +// cooperative signal that no further Execute streams will be started, +// not an order to abort active ones. In-flight Execute streams MUST be +// allowed to complete normally (via [[Finish]] or [[Cancel]]). The +// engine SHOULD NOT send [[ShutdownRequest]] while it still intends to +// start new Execute streams; the worker is not required to enforce or +// detect this. Once [[ShutdownRequest]] has been received and all +// Execute streams have terminated (cleanly or via gRPC error), the +// worker SHOULD exit. message ShutdownRequest { // (Optional) Free-form reason for diagnostics. optional string reason = 1; From 1913d2fd2689a9136b0b3e58557d769df6bf5a1f Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Wed, 13 May 2026 09:29:19 +0000 Subject: [PATCH 07/13] unit tests generating a grpc server based on the proto. --- udf/worker/README.md | 2 +- .../udf/worker/core/EchoProtocolSuite.scala | 634 ++++++++++++++++++ .../src/main/protobuf/udf_protocol.proto | 155 +++-- 3 files changed, 749 insertions(+), 42 deletions(-) create mode 100644 udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala diff --git a/udf/worker/README.md b/udf/worker/README.md index 4aa5388858c17..4946425826328 100644 --- a/udf/worker/README.md +++ b/udf/worker/README.md @@ -62,7 +62,7 @@ Each UDF execution uses a single bidirectional `Execute` gRPC stream: ``` Engine -> Worker: Init -> PayloadChunk* -> (DataRequest)* -> (Finish | Cancel) -Worker -> Engine: InitResponse -> (DataResponse)* -> (CodeError)? -> (FinishResponse | CancelResponse) +Worker -> Engine: InitResponse -> (DataResponse)* -> (ExecutionError)? -> (FinishResponse | CancelResponse) ``` `DataRequest` and `DataResponse` are independent streams: the worker may emit diff --git a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala new file mode 100644 index 0000000000000..8ada918e21fb1 --- /dev/null +++ b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala @@ -0,0 +1,634 @@ +/* + * 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.concurrent.{CountDownLatch, LinkedBlockingQueue, TimeUnit} +import java.util.concurrent.atomic.AtomicBoolean + +import com.google.protobuf.ByteString + +// NOTE: These imports require adding grpc-stub + grpc-inprocess to the pom. +// The proto module currently only runs protoc (no grpc-java plugin), so +// UdfWorkerGrpc does not exist yet. Adding it requires: +// 1. grpc-java codegen plugin in udf/worker/proto/pom.xml +// 2. grpc-stub + grpc-netty (or grpc-inprocess) deps in udf/worker/core/pom.xml +import io.grpc.stub.StreamObserver +import io.grpc.{ManagedChannel, Server, Status} +import io.grpc.inprocess.{InProcessChannelBuilder, InProcessServerBuilder} +import org.apache.spark.udf.worker.UdfWorkerGrpc + +import org.apache.spark.udf.worker.{ + Cancel, CancelResponse, DataRequest, DataResponse, + ExecutionError, UserError, WorkerError, ProtocolError, + Finish, FinishResponse, Heartbeat, HeartbeatResponse, + Init, InitResponse, ShutdownRequest, ShutdownResponse, + UDFWorkerDataFormat, UdfControlRequest, UdfControlResponse, + UdfPayload, UdfRequest, UdfResponse, WorkerRequest, WorkerResponse +} + +// scalastyle:off funsuite +import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.BeforeAndAfterEach + +/** + * Validates the UDF gRPC protocol by implementing a dummy echo worker and + * an engine client. The worker echoes each DataRequest batch back as a + * DataResponse. Error paths (ExecutionError, Cancel) are exercised with fake + * triggers. + * + * FINDINGS -- things that were unclear or missing from the proto when + * writing this implementation: + * See the FINDING comments throughout the file. + */ +class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { +// scalastyle:on funsuite + + private val SUPPORTED_VERSION: Int = 1 + // Trigger word: a DataRequest whose payload equals this string causes the + // worker to emit ExecutionError + FinishResponse instead of echoing. + private val ERROR_TRIGGER: ByteString = ByteString.copyFromUtf8("ERROR") + + private var server: Server = _ + private var channel: ManagedChannel = _ + private var stub: UdfWorkerGrpc.UdfWorkerStub = _ + + override def beforeEach(): Unit = { + val serverName = InProcessServerBuilder.generateName() + server = InProcessServerBuilder.forName(serverName) + .directExecutor() + .addService(new EchoWorkerService) + .build() + .start() + channel = InProcessChannelBuilder.forName(serverName).directExecutor().build() + stub = UdfWorkerGrpc.newStub(channel) + } + + override def afterEach(): Unit = { + channel.shutdownNow() + server.shutdownNow() + } + + // =========================================================================== + // WORKER SIDE (gRPC server) + // =========================================================================== + + /** Worker state machine for one Execute stream. */ + private sealed trait WorkerState + private case object AwaitingInit extends WorkerState + // FINDING 1: The proto does not state explicitly what the worker does + // between Init and the first data message when no chunks are expected. + // The chunking section says "when the engine uses PayloadChunk at all, + // it MUST set last=true." So the worker enters Chunking only after it + // actually receives a PayloadChunk. Until then it stays in AwaitingData, + // ready to send InitResponse immediately when the first DataRequest or + // Finish arrives. The proto should state this explicitly: + // "If no PayloadChunk arrives, InitResponse MUST be sent before the + // first DataRequest or Finish is processed." + private case class AwaitingData(initPayload: ByteString) extends WorkerState + private case class Chunking(accumulated: ByteString) extends WorkerState + private case object Data extends WorkerState + private case object Done extends WorkerState + + private class EchoWorkerService extends UdfWorkerGrpc.UdfWorkerImplBase { + + override def execute( + responseObserver: StreamObserver[UdfResponse]): StreamObserver[UdfRequest] = + new ExecuteStreamHandler(responseObserver) + + override def manage( + request: WorkerRequest, + responseObserver: StreamObserver[WorkerResponse]): Unit = { + request.getManage match { + case WorkerRequest.Manage.Heartbeat(_) => + responseObserver.onNext(WorkerResponse.newBuilder() + .setHeartbeat(HeartbeatResponse.getDefaultInstance) + .build()) + responseObserver.onCompleted() + + case WorkerRequest.Manage.Shutdown(req) => + // FINDING 2: The proto says the worker SHOULD exit after all Execute + // streams terminate. But ShutdownResponse gives no way to indicate + // "acknowledged, draining" vs "acknowledged, already idle." A boolean + // or enum field on ShutdownResponse would make the worker's state + // visible to the engine without requiring a separate Heartbeat probe. + responseObserver.onNext(WorkerResponse.newBuilder() + .setShutdown(ShutdownResponse.getDefaultInstance) + .build()) + responseObserver.onCompleted() + + case _ => + responseObserver.onError( + Status.INVALID_ARGUMENT.withDescription("empty manage request") + .asRuntimeException()) + } + } + } + + private class ExecuteStreamHandler( + responseObserver: StreamObserver[UdfResponse]) extends StreamObserver[UdfRequest] { + + @volatile private var state: WorkerState = AwaitingInit + // Guards responseObserver: gRPC does not allow concurrent onNext calls. + // FINDING 3: The proto says DataRequest and DataResponse are "independent + // streams" and the worker may emit DataResponse at any time. In practice, + // if the worker dispatches processing to a thread pool, multiple threads + // could race to call responseObserver.onNext(). The proto does not mention + // this constraint. Worker implementations must serialize all writes to the + // response observer themselves. + private val responseLock = new Object + + override def onNext(request: UdfRequest): Unit = { + request.getRequest match { + case UdfRequest.Request.Control(ctrl) => handleControl(ctrl) + case UdfRequest.Request.Data(data) => handleDataRequest(data) + case _ => closeWithProtocolError("empty request oneof") + } + } + + private def handleControl(ctrl: UdfControlRequest): Unit = { + ctrl.getControl match { + case UdfControlRequest.Control.Init(init) => handleInit(init) + case UdfControlRequest.Control.Payload(chunk) => handleChunk(chunk) + case UdfControlRequest.Control.Finish(_) => handleFinish() + case UdfControlRequest.Control.Cancel(cancel) => handleCancel(cancel) + case _ => closeWithProtocolError("empty control oneof") + } + } + + private def handleInit(init: Init): Unit = state match { + case AwaitingInit => + // FINDING 4 (resolved): unsupported protocol_version is now surfaced + // via ExecutionError(ProtocolError). The worker sends ProtocolError + // + FinishResponse, keeping the stream lifecycle intact. + if (init.hasProtocolVersion && init.getProtocolVersion != SUPPORTED_VERSION) { + sendControl(UdfControlResponse.newBuilder() + .setError(ExecutionError.newBuilder() + .setProtocol(ProtocolError.newBuilder() + .setMessage(s"unsupported protocol version: ${init.getProtocolVersion}") + .build()) + .build()) + .build()) + sendControl(UdfControlResponse.newBuilder() + .setFinish(FinishResponse.getDefaultInstance) + .build()) + responseLock.synchronized { responseObserver.onCompleted() } + state = Done + return + } + val inlinePayload = init.getUdf.getPayload + state = AwaitingData(inlinePayload) + + case _ => closeWithProtocolError(s"Init received in state $state") + } + + private def handleChunk(chunk: org.apache.spark.udf.worker.PayloadChunk): Unit = + state match { + case AwaitingData(existing) => + val updated = existing.concat(chunk.getData) + if (chunk.hasLast && chunk.getLast) { + // Payload is complete. Send InitResponse and move to Data. + sendInitResponse() + state = Data + } else { + state = Chunking(updated) + } + + case Chunking(existing) => + val updated = existing.concat(chunk.getData) + if (chunk.hasLast && chunk.getLast) { + sendInitResponse() + state = Data + } else { + state = Chunking(updated) + } + + case _ => closeWithProtocolError(s"PayloadChunk received in state $state") + } + + private def handleDataRequest(data: DataRequest): Unit = state match { + case AwaitingData(_) => + // No chunks were sent. Send InitResponse before handling data, + // per the ordering invariant "InitResponse MUST be emitted before + // any DataResponse." + sendInitResponse() + state = Data + processEcho(data) + + case Data => processEcho(data) + + case _ => closeWithProtocolError(s"DataRequest received in state $state") + } + + private def processEcho(data: DataRequest): Unit = { + if (data.getData == ERROR_TRIGGER) { + // Simulate a user-code error: send ExecutionError(UserError), then + // wait for the engine's Finish or Cancel before sending the terminator. + sendControl(UdfControlResponse.newBuilder() + .setError(ExecutionError.newBuilder() + .setUser(UserError.newBuilder() + .setMessage("simulated user-code error") + .setErrorClass("SimulatedError") + .build()) + .build()) + .build()) + // Transition to a "post-error, awaiting terminator" state reusing Done + // to stop further DataRequest processing. The actual FinishResponse or + // CancelResponse is sent when Finish or Cancel arrives from the engine. + state = Data // stay in Data to receive Finish/Cancel normally + } else { + // Echo the raw bytes back. + responseLock.synchronized { + responseObserver.onNext(UdfResponse.newBuilder() + .setData(DataResponse.newBuilder().setData(data.getData).build()) + .build()) + } + } + } + + private def handleFinish(): Unit = state match { + case AwaitingData(_) => + // Generator-style UDF: engine sends Finish directly after Init. + sendInitResponse() + sendControl(UdfControlResponse.newBuilder() + .setFinish(FinishResponse.newBuilder().build()) + .build()) + responseLock.synchronized { responseObserver.onCompleted() } + state = Done + + case Data => + sendControl(UdfControlResponse.newBuilder() + .setFinish(FinishResponse.newBuilder() + .putMetrics("status", "ok") + .build()) + .build()) + responseLock.synchronized { responseObserver.onCompleted() } + state = Done + + case _ => closeWithProtocolError(s"Finish received in state $state") + } + + private def handleCancel(cancel: Cancel): Unit = state match { + case Data | AwaitingData(_) => + // Stop producing output and send CancelResponse. + sendControl(UdfControlResponse.newBuilder() + .setCancel(CancelResponse.getDefaultInstance) + .build()) + responseLock.synchronized { responseObserver.onCompleted() } + state = Done + + case _ => closeWithProtocolError(s"Cancel received in state $state") + } + + override def onError(t: Throwable): Unit = { + // gRPC transport error from the engine side (connection dropped). + // Per the protocol: treat as equivalent to Cancel for cleanup purposes; + // do NOT attempt to send CancelResponse (stream is dead). + state = Done + } + + // FINDING 5: The proto does not specify what the worker should do when + // the gRPC request stream is half-closed by the engine (onCompleted) if + // the worker is still in an active state. In the normal flow the engine + // half-closes after sending Finish, so onCompleted arrives after the + // worker has already sent FinishResponse and moved to Done. But if + // onCompleted arrives unexpectedly (e.g. engine bug), the worker has no + // way to distinguish "normal half-close after Finish" from + // "engine closed stream early." The proto should say: if onCompleted is + // received while the worker is still in an active state it SHOULD treat + // it as a protocol error and close with a gRPC error. + override def onCompleted(): Unit = state match { + case Done => // normal: Finish or Cancel was already processed + case _ => + closeWithProtocolError( + s"request stream closed by engine in unexpected state $state") + } + + private def sendInitResponse(): Unit = + sendControl(UdfControlResponse.newBuilder() + .setInit(InitResponse.getDefaultInstance) + .build()) + + private def sendControl(ctrl: UdfControlResponse): Unit = + responseLock.synchronized { + responseObserver.onNext( + UdfResponse.newBuilder().setControl(ctrl).build()) + } + + private def closeWithProtocolError(msg: String): Unit = { + // FINDING 5 (resolved): use ProtocolError instead of a gRPC error so + // the stream lifecycle (FinishResponse/CancelResponse) stays intact. + state = Done + sendControl(UdfControlResponse.newBuilder() + .setError(ExecutionError.newBuilder() + .setProtocol(ProtocolError.newBuilder().setMessage(msg).build()) + .build()) + .build()) + sendControl(UdfControlResponse.newBuilder() + .setFinish(FinishResponse.getDefaultInstance) + .build()) + responseLock.synchronized { responseObserver.onCompleted() } + } + } + + // =========================================================================== + // ENGINE SIDE (gRPC client) + // =========================================================================== + + /** + * Minimal engine client. Collects received DataResponse bytes and exposes + * latch-based synchronization so tests can wait for stream completion. + */ + private class EngineClient(stub: UdfWorkerGrpc.UdfWorkerStub) { + private val results = new LinkedBlockingQueue[Array[Byte]]() + private val done = new CountDownLatch(1) + @volatile var executionError: Option[ExecutionError] = None + @volatile var streamError: Option[Throwable] = None + // Tracks whether Finish has been sent on the request stream so that a + // concurrent Cancel can suppress it (cancel-vs-finish race). + private val finishSent = new AtomicBoolean(false) + + private val responseObserver = new StreamObserver[UdfResponse] { + override def onNext(response: UdfResponse): Unit = { + response.getResponse match { + case UdfResponse.Response.Data(data) => + results.add(data.getData.toByteArray) + + case UdfResponse.Response.Control(ctrl) => + ctrl.getControl match { + case UdfControlResponse.Control.Init(_) => + // InitResponse received: data phase can start. + // FINDING 6: The proto does not say whether the engine + // should start sending DataRequests before or after receiving + // InitResponse. In this implementation the engine sends all + // DataRequests eagerly after Init without waiting for + // InitResponse; this works because gRPC buffers them. But + // some implementations might want to wait for InitResponse + // before sending data (e.g., to use fields from InitResponse + // to configure the data pipeline). The proto should clarify + // whether the engine MAY pipeline DataRequests before + // receiving InitResponse. + + case UdfControlResponse.Control.Error(err) => + // FINDING 8: record the error; the engine still drives the + // terminator. If Finish was already sent the engine waits for + // FinishResponse. If not, the engine sends Cancel and waits for + // CancelResponse. The exception is raised after the terminator. + executionError = Some(err) + if (!finishSent.get()) { + sendCancel("aborting after ExecutionError") + } + // Fall through: FinishResponse or CancelResponse will arrive. + + case UdfControlResponse.Control.Finish(_) => + done.countDown() + + case UdfControlResponse.Control.Cancel(_) => + done.countDown() + + case _ => + } + + case _ => + } + } + + override def onError(t: Throwable): Unit = { + streamError = Some(t) + done.countDown() + } + + override def onCompleted(): Unit = { + // Server closed the response stream. If done hasn't been counted down + // yet (no FinishResponse / CancelResponse seen), treat as unexpected. + done.countDown() + } + } + + // FINDING 7: After sending Finish, the engine must also half-close the + // request stream by calling requestObserver.onCompleted(). The proto + // says "Finish is the last message from the engine" but does not mention + // the gRPC half-close. If the engine never calls onCompleted(), the server's + // StreamObserver.onCompleted() is never triggered, which could leave the + // server hanging in some implementations. The proto should say: + // "The engine MUST half-close the request stream (call onCompleted on + // the gRPC stream) immediately after sending Finish or Cancel." + private val requestObserver: StreamObserver[UdfRequest] = stub.execute(responseObserver) + + def sendInit(payloadBytes: Array[Byte], sendChunked: Boolean = false): Unit = { + if (sendChunked) { + // Send Init with empty inline payload, then PayloadChunk. + val initReq = UdfRequest.newBuilder() + .setControl(UdfControlRequest.newBuilder() + .setInit(Init.newBuilder() + .setProtocolVersion(SUPPORTED_VERSION) + .setDataFormat(UDFWorkerDataFormat.ARROW) + .setUdf(UdfPayload.newBuilder() + // Intentionally empty: payload arrives via chunks. + .setPayload(ByteString.EMPTY) + .setFormat("echo") + .build()) + .build()) + .build()) + .build() + requestObserver.onNext(initReq) + + // Send payload as a single chunk with last=true. + requestObserver.onNext(UdfRequest.newBuilder() + .setControl(UdfControlRequest.newBuilder() + .setPayload(org.apache.spark.udf.worker.PayloadChunk.newBuilder() + .setData(ByteString.copyFrom(payloadBytes)) + .setLast(true) + .build()) + .build()) + .build()) + } else { + requestObserver.onNext(UdfRequest.newBuilder() + .setControl(UdfControlRequest.newBuilder() + .setInit(Init.newBuilder() + .setProtocolVersion(SUPPORTED_VERSION) + .setDataFormat(UDFWorkerDataFormat.ARROW) + .setUdf(UdfPayload.newBuilder() + .setPayload(ByteString.copyFrom(payloadBytes)) + .setFormat("echo") + .build()) + .build()) + .build()) + .build()) + } + } + + def sendData(data: Array[Byte]): Unit = + requestObserver.onNext(UdfRequest.newBuilder() + .setData(DataRequest.newBuilder() + .setData(ByteString.copyFrom(data)) + .build()) + .build()) + + def sendFinish(): Unit = { + if (finishSent.compareAndSet(false, true)) { + requestObserver.onNext(UdfRequest.newBuilder() + .setControl(UdfControlRequest.newBuilder() + .setFinish(Finish.getDefaultInstance) + .build()) + .build()) + requestObserver.onCompleted() + } + } + + def sendCancel(reason: String = ""): Unit = { + // Only send Cancel if Finish hasn't already been sent (race guard). + if (finishSent.compareAndSet(false, true)) { + requestObserver.onNext(UdfRequest.newBuilder() + .setControl(UdfControlRequest.newBuilder() + .setCancel(Cancel.newBuilder().setReason(reason).build()) + .build()) + .build()) + requestObserver.onCompleted() + } + // If Finish was already sent, suppress Cancel: rely on FinishResponse + // to terminate the stream (per the cancel-vs-finish race contract). + } + + def awaitDone(timeoutMs: Long = 5000): Boolean = + done.await(timeoutMs, TimeUnit.MILLISECONDS) + + def drainResults(): Seq[Array[Byte]] = { + val buf = new java.util.ArrayList[Array[Byte]]() + results.drainTo(buf) + import scala.jdk.CollectionConverters._ + buf.asScala.toSeq + } + } + + // =========================================================================== + // TESTS + // =========================================================================== + + test("echo: single DataRequest round-trip") { + val client = new EngineClient(stub) + client.sendInit("dummy-payload".getBytes) + client.sendData("hello".getBytes) + client.sendFinish() + + assert(client.awaitDone(), "stream did not complete in time") + assert(client.streamError.isEmpty, s"unexpected stream error: ${client.streamError}") + assert(client.executionError.isEmpty, s"unexpected code error: ${client.executionError}") + val results = client.drainResults() + assert(results.length == 1) + assert(new String(results.head) == "hello") + } + + test("echo: multiple DataRequest batches are all echoed") { + val client = new EngineClient(stub) + client.sendInit("dummy-payload".getBytes) + Seq("batch1", "batch2", "batch3").foreach(b => client.sendData(b.getBytes)) + client.sendFinish() + + assert(client.awaitDone()) + assert(client.streamError.isEmpty) + val results = client.drainResults().map(new String(_)) + assert(results == Seq("batch1", "batch2", "batch3")) + } + + test("echo: chunked payload delivery") { + val client = new EngineClient(stub) + client.sendInit("chunked-payload".getBytes, sendChunked = true) + client.sendData("data".getBytes) + client.sendFinish() + + assert(client.awaitDone()) + assert(client.streamError.isEmpty) + assert(new String(client.drainResults().head) == "data") + } + + test("echo: generator-style UDF (zero DataRequests, engine sends Finish after Init)") { + val client = new EngineClient(stub) + client.sendInit("generator-payload".getBytes) + // No DataRequests at all -- engine sends Finish directly after Init. + client.sendFinish() + + assert(client.awaitDone()) + assert(client.streamError.isEmpty) + assert(client.drainResults().isEmpty) + } + + test("cancel: engine cancels mid-stream") { + val client = new EngineClient(stub) + client.sendInit("dummy-payload".getBytes) + client.sendData("batch1".getBytes) + client.sendCancel("task interrupted") + + assert(client.awaitDone()) + assert(client.streamError.isEmpty) + // Results before Cancel may or may not have arrived; no guarantee. + } + + test("cancel-vs-finish race: second call is suppressed") { + val client = new EngineClient(stub) + client.sendInit("dummy-payload".getBytes) + client.sendData("data".getBytes) + client.sendFinish() + client.sendCancel("late cancel -- should be suppressed") + + assert(client.awaitDone()) + // Only FinishResponse should arrive; the Cancel was suppressed by finishSent. + assert(client.streamError.isEmpty) + } + + test("ExecutionError: error trigger causes UserError, engine sends Cancel, gets CancelResponse") { + val client = new EngineClient(stub) + client.sendInit("dummy-payload".getBytes) + client.sendData(ERROR_TRIGGER.toByteArray) + // Engine has NOT sent Finish yet; on receiving ExecutionError it will send + // Cancel and wait for CancelResponse before raising the error. + + assert(client.awaitDone()) + assert(client.streamError.isEmpty, s"expected no gRPC error, got ${client.streamError}") + assert(client.executionError.isDefined, "expected an ExecutionError") + assert(client.executionError.get.hasUser, "expected UserError kind") + assert(client.executionError.get.getUser.getErrorClass == "SimulatedError") + } + + test("protocol error: second Init is rejected") { + val client = new EngineClient(stub) + client.sendInit("payload".getBytes) + // Send Init again -- protocol violation. + client.sendInit("second-init".getBytes) + + assert(client.awaitDone()) + assert(client.streamError.isDefined, "expected a gRPC error for double Init") + } + + test("Manage: heartbeat is acknowledged") { + val latch = new CountDownLatch(1) + @volatile var response: WorkerResponse = null + + val blockingStub = UdfWorkerGrpc.newBlockingStub(channel) + val resp = blockingStub.manage(WorkerRequest.newBuilder() + .setHeartbeat(Heartbeat.getDefaultInstance) + .build()) + assert(resp.hasHeartbeat, "expected HeartbeatResponse") + } + + test("Manage: ShutdownRequest is acknowledged") { + val blockingStub = UdfWorkerGrpc.newBlockingStub(channel) + val resp = blockingStub.manage(WorkerRequest.newBuilder() + .setShutdown(ShutdownRequest.newBuilder().setReason("test done").build()) + .build()) + assert(resp.hasShutdown, "expected ShutdownResponse") + } +} diff --git a/udf/worker/proto/src/main/protobuf/udf_protocol.proto b/udf/worker/proto/src/main/protobuf/udf_protocol.proto index 9e88eddb308e8..991141e44b540 100644 --- a/udf/worker/proto/src/main/protobuf/udf_protocol.proto +++ b/udf/worker/proto/src/main/protobuf/udf_protocol.proto @@ -47,6 +47,17 @@ service UdfWorker { // eventually arrive. This applies to every in-flight response, not // only [[CancelResponse]]. // + // Stream lifecycle: the engine MUST half-close the request stream + // (call onCompleted() on the gRPC stream) immediately after sending + // [[Finish]] or [[Cancel]]. This is the gRPC-level signal that no + // further requests will arrive and is distinct from the protocol-level + // terminator message. + // + // Response observer threading: gRPC does not permit concurrent calls + // to the response StreamObserver. Worker implementations that dispatch + // processing to a thread pool MUST serialize all writes to the response + // observer. + // // For stateful execution, the state is maintained per bi-directional // stream, mapping to a `WorkerSession` on the engine side. rpc Execute(stream UdfRequest) returns (stream UdfResponse); @@ -74,7 +85,7 @@ message UdfRequest { } // Worker -> Engine. Either a control response ([[InitResponse]] / -// [[FinishResponse]] / [[CancelResponse]] / [[CodeError]]) or a +// [[FinishResponse]] / [[CancelResponse]] / [[ExecutionError]]) or a // data response message. message UdfResponse { // Exactly one branch MUST be set; receivers MUST reject messages @@ -90,7 +101,7 @@ message UdfResponse { // Wire protocol for one Execute stream (both directions): // // Engine -> Worker: Init -> PayloadChunk* -> (DataRequest)* -> (Finish | Cancel) -// Worker -> Engine: InitResponse -> (DataResponse)* -> (CodeError)? -> (FinishResponse | CancelResponse) +// Worker -> Engine: InitResponse -> (DataResponse)* -> (ExecutionError)? -> (FinishResponse | CancelResponse) // // DataRequest and DataResponse are independent streams: the worker // may emit DataResponse messages at any point after InitResponse, @@ -100,20 +111,33 @@ message UdfResponse { // The arrows above denote ordering constraints within each direction, // not a request/response pairing. // +// The engine MAY send DataRequests before receiving InitResponse (pipeline +// mode). The worker MUST buffer such DataRequests and process them in +// arrival order once init succeeds. They MAY be discarded only if init +// fails (i.e. the worker sends ExecutionError before InitResponse). +// // Ordering invariants: -// - PayloadChunk* only after Init and before the first DataRequest; +// - PayloadChunk* only after Init and before the first DataRequest. +// [[Init.is_chunking_payload]] = true signals that chunks will follow; // [[PayloadChunk.last]] = true is the canonical end-of-chunking signal. +// When [[Init.is_chunking_payload]] is false or absent, [[InitResponse]] +// MAY be sent immediately after [[Init]] without waiting for chunks. // - InitResponse MUST be emitted before any DataResponse. -// - CodeError (if any) MUST be emitted after all DataResponse -// messages and before FinishResponse. It is always followed by -// FinishResponse (not CancelResponse) so the stream terminates cleanly. +// - ExecutionError (if any) MUST be emitted after all DataResponse +// messages. After sending it the worker MUST stop processing DataRequests +// and wait for the engine to send Finish or Cancel, then respond with +// FinishResponse or CancelResponse accordingly. At most one +// ExecutionError is sent per stream; the worker aggregates multiple +// errors internally. // - Exactly one terminator pair: Finish -> FinishResponse OR // Cancel -> CancelResponse (never both on the same stream; see [[Finish]] // for the cancel-vs-finish race contract on the engine side). // // A worker that receives messages out of order (e.g. a second Init, // a PayloadChunk after the first DataRequest, a DataRequest before Init, -// or a Cancel before Init) MUST close the stream with an error. +// or a Cancel before Init) MUST send [[ExecutionError]] with a +// [[ProtocolError]] kind, followed by [[FinishResponse]] or +// [[CancelResponse]] to close the stream cleanly. message UdfControlRequest { // Exactly one branch MUST be set; receivers MUST reject messages // with no branch set. @@ -133,7 +157,7 @@ message UdfControlResponse { InitResponse init = 1; FinishResponse finish = 2; CancelResponse cancel = 3; - CodeError error = 4; + ExecutionError error = 4; } } @@ -244,13 +268,21 @@ message Init { // Spark does. optional string timezone = 8; + // (Optional) When true, the UDF payload will be delivered via + // [[PayloadChunk]] messages rather than inline in [[UdfPayload.payload]]. + // The worker MUST wait for [[PayloadChunk.last]] = true before sending + // [[InitResponse]]. When false or absent, the payload is fully contained + // in [[UdfPayload.payload]] and the worker MAY send [[InitResponse]] + // immediately after [[Init]] without waiting for any [[PayloadChunk]]. + optional bool is_chunking_payload = 9; + // Reserved for future typed Init fields, in particular keys // graduated from [[session_conf]] (see the [[timezone]] precedent // above). Numbers >= 100 are intentionally NOT reserved here; if // a future revision needs an opaque escape-hatch field, give it a // number >= 100 alongside [[parameters]] and add a field-level // comment so the convention stays visible. - reserved 9 to 99; + reserved 10 to 99; // (Optional) Engine-packed opaque parameters specific to a // particular kind of UDF execution. The escape hatch for @@ -263,7 +295,7 @@ message Init { // // Numbers >= 100 are reserved by convention for opaque // escape-hatch fields like this one; new typed fields use the - // reserved 9..99 range. + // reserved 10..99 range. // // Client-side init data (anything packed by the layer that // defines and serializes the UDF) does NOT belong here -- it @@ -425,41 +457,70 @@ message Cancel { // Worker -> Engine acknowledgment of [[Cancel]]. message CancelResponse {} -// Worker -> Engine. Signals a non-gRPC error during execution: -// either user code raised an exception, or the worker implementation -// itself detected an application-level error it wants to surface to -// the engine. Distinct from a gRPC stream error: gRPC errors indicate -// transport or protocol failures, whereas [[CodeError]] carries -// application-level errors that should be raised as user-facing -// exceptions rather than transport failures. +// Worker -> Engine. Signals a non-gRPC application-level error. +// Distinct from a gRPC stream error: gRPC errors indicate transport or +// protocol failures; [[ExecutionError]] carries errors that should be +// raised as user-facing exceptions. // // Wire position: emitted at most once, after all [[DataResponse]] -// messages and before [[FinishResponse]]. [[FinishResponse]] is still -// sent after [[CodeError]] so the stream terminates cleanly. -// The engine MUST raise a user-facing exception (not a protocol error) -// upon receiving this message. +// messages. After sending [[ExecutionError]] the worker MUST stop +// processing [[DataRequest]] messages and wait for the engine to send +// [[Finish]] or [[Cancel]], then respond with [[FinishResponse]] or +// [[CancelResponse]] accordingly. +// +// Engine behavior on receipt: the engine records the error and then: +// - If [[Finish]] has already been sent: waits for [[FinishResponse]] +// and raises the error afterwards. +// - If [[Finish]] has not yet been sent: sends [[Cancel]], waits for +// [[CancelResponse]], and raises the error afterwards. +// If more than one [[ExecutionError]] arrives (protocol violation), +// the engine SHOULD surface the first as the primary exception and +// attach subsequent ones as suppressed exceptions. // // Cancel interaction: if [[Cancel]] arrives while the worker is -// preparing to send [[CodeError]], the cancel-vs-finish race rules -// in [[Finish]] apply. The worker emits exactly one terminator: -// [[FinishResponse]] (if [[CodeError]] was already sent or the -// worker chooses to complete) or [[CancelResponse]] (if [[Cancel]] -// wins the race before [[CodeError]] is sent). -message CodeError { +// preparing to send [[ExecutionError]], the cancel-vs-finish race rules +// in [[Finish]] apply. The worker emits exactly one terminator. +message ExecutionError { + // Exactly one kind MUST be set. + oneof kind { + UserError user = 1; + WorkerError worker = 2; + ProtocolError protocol = 3; + } +} + +// Error raised by the user's UDF code. +message UserError { // (Required) Human-readable error message from the UDF. string message = 1; // (Optional) Full stack trace or traceback in the worker's - // language-specific format. Forwarded to the user as-is for - // diagnostics. + // language-specific format. Forwarded to the user as-is. optional string traceback = 2; // (Optional) Language-specific error class name (e.g. "ValueError" - // in Python, "RuntimeException" in Java). Allows the engine to - // surface a more informative exception type to the user. + // in Python, "RuntimeException" in Java). optional string error_class = 3; } +// Error originating from the worker implementation itself, not user code. +message WorkerError { + // (Required) Human-readable description of the worker error. + string message = 1; + + // (Optional) Stack trace for diagnostics. + optional string traceback = 2; +} + +// Protocol violation detected by the worker (e.g. messages received out +// of order, unsupported [[Init.protocol_version]]). Sending this type +// instead of closing with a gRPC error keeps the stream lifecycle intact: +// [[FinishResponse]] or [[CancelResponse]] still follows. +message ProtocolError { + // (Required) Description of the protocol violation. + string message = 1; +} + // The single UDF body delivered to the worker on [[Init]]. Opaque to // the engine: the engine forwards [[payload]] and [[format]] // unchanged, and the worker decodes them per the format the client @@ -579,19 +640,31 @@ message HeartbeatResponse { // streams. OS-level process management is outside the scope of this // protocol and is defined in the worker specification. // -// Interaction with in-flight Execute streams: [[ShutdownRequest]] is a -// cooperative signal that no further Execute streams will be started, -// not an order to abort active ones. In-flight Execute streams MUST be -// allowed to complete normally (via [[Finish]] or [[Cancel]]). The -// engine SHOULD NOT send [[ShutdownRequest]] while it still intends to -// start new Execute streams; the worker is not required to enforce or -// detect this. Once [[ShutdownRequest]] has been received and all -// Execute streams have terminated (cleanly or via gRPC error), the -// worker SHOULD exit. +// Interaction with in-flight Execute streams: when [[cancel_sessions]] +// is false (the default), in-flight Execute streams MUST be allowed to +// complete normally (via [[Finish]] or [[Cancel]]). When [[cancel_sessions]] +// is true, the worker MUST cancel all in-flight streams immediately by +// sending [[ExecutionError]] with [[WorkerError]] kind followed by the +// appropriate terminator on each stream, then exit. The engine SHOULD NOT +// send [[ShutdownRequest]] while it still intends to start new Execute +// streams. Once all Execute streams have terminated (cleanly or via gRPC +// error), the worker SHOULD exit. message ShutdownRequest { // (Optional) Free-form reason for diagnostics. optional string reason = 1; + + // (Optional; defaults to false.) When true, the worker MUST cancel + // all in-flight Execute streams immediately rather than waiting for + // them to complete naturally. + optional bool cancel_sessions = 2; } // Worker -> Engine acknowledgment of [[ShutdownRequest]]. -message ShutdownResponse {} +message ShutdownResponse { + // True when all Execute streams were fully settled (finished or + // cancelled) before this response was sent. False when the worker + // acknowledged the request but streams were still in flight (only + // possible when [[ShutdownRequest.cancel_sessions]] is false and the + // worker sends the response before draining). + bool sessions_settled = 1; +} From 74cb3be76c4e3c88e02f6683eac8d9b75f93eb30 Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Wed, 13 May 2026 12:49:15 +0000 Subject: [PATCH 08/13] updated cancel vs. close semantics. --- .../spark/udf/worker/core/WorkerSession.scala | 17 +- .../udf/worker/core/EchoProtocolSuite.scala | 226 ++++++++++++------ .../src/main/protobuf/udf_protocol.proto | 65 ++--- 3 files changed, 200 insertions(+), 108 deletions(-) diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala index 57544f3f561c5..f60fbca129dd1 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala @@ -59,9 +59,13 @@ import org.apache.spark.udf.worker.Init * interruption listener (which has no view into the session state) * do not need to coordinate with the thread driving [[process]]. * - * The cancel-vs-finish race contract is defined in the `Finish` - * message of `udf_protocol.proto`. Implementations MUST resolve - * that race so the transport never observes both on the same stream. + * [[cancel]] may be called even after all input data has been + * submitted (i.e. after [[Finish]] has been sent on the transport). + * In that case implementations MUST send [[Cancel]] on the transport + * if [[FinishResponse]] has not yet been received, and MUST be + * prepared to receive either [[FinishResponse]] or [[CancelResponse]]. + * See the [[org.apache.spark.udf.worker.Finish]] proto message for + * the full contract. * * The lifecycle is enforced here: [[init]] and [[process]] are `final` * and delegate to [[doInit]] / [[doProcess]] after AtomicBoolean guards. @@ -155,8 +159,11 @@ abstract class WorkerSession extends AutoCloseable { * - between [[init]] and [[process]] -- signals that the session * should be terminated; the caller should not invoke [[process]] * and should call [[close]] to release resources. - * - during [[process]] -- aborts the active stream. - * - after [[process]] / [[close]] has returned -- a no-op. + * - during [[process]] (data flowing or awaiting [[FinishResponse]]) + * -- sends [[Cancel]] and waits for [[CancelResponse]] or + * [[FinishResponse]] (whichever arrives first). + * - after [[FinishResponse]] or [[CancelResponse]] has been received + * -- a no-op; the stream is already terminated. * * Implementations are responsible for the lifecycle-aware behavior * described above (no-op outside the active window; cancellation diff --git a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala index 8ada918e21fb1..e4df0c1a87e22 100644 --- a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala +++ b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala @@ -101,6 +101,9 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { private case class AwaitingData(initPayload: ByteString) extends WorkerState private case class Chunking(accumulated: ByteString) extends WorkerState private case object Data extends WorkerState + // Finish received; FinishResponse not yet sent. Cancel may still win if it + // arrives before the drain completes and FinishResponse is written. + private case object Finishing extends WorkerState private case object Done extends WorkerState private class EchoWorkerService extends UdfWorkerGrpc.UdfWorkerImplBase { @@ -263,33 +266,45 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { case AwaitingData(_) => // Generator-style UDF: engine sends Finish directly after Init. sendInitResponse() - sendControl(UdfControlResponse.newBuilder() - .setFinish(FinishResponse.newBuilder().build()) - .build()) - responseLock.synchronized { responseObserver.onCompleted() } - state = Done + drainAndFinish() case Data => - sendControl(UdfControlResponse.newBuilder() - .setFinish(FinishResponse.newBuilder() - .putMetrics("status", "ok") - .build()) - .build()) - responseLock.synchronized { responseObserver.onCompleted() } - state = Done + drainAndFinish() case _ => closeWithProtocolError(s"Finish received in state $state") } + // Transitions to Finishing, performs any remaining work (for the echo + // worker this is instant), then sends FinishResponse. A real worker with + // async processing would stay in Finishing until the drain completes, + // allowing a concurrent Cancel to win if it arrives in time. + private def drainAndFinish(): Unit = { + state = Finishing + // Echo worker: no buffered state to drain. Immediately complete. + sendControl(UdfControlResponse.newBuilder() + .setFinish(FinishResponse.newBuilder() + .putMetrics("status", "ok") + .build()) + .build()) + responseLock.synchronized { responseObserver.onCompleted() } + state = Done + } + private def handleCancel(cancel: Cancel): Unit = state match { - case Data | AwaitingData(_) => - // Stop producing output and send CancelResponse. + case Data | AwaitingData(_) | Finishing => + // Finishing: Cancel arrived while drain was in progress -- abort. + // (For the synchronous echo worker this race is not observable in + // tests, but the state machine is correct for async implementations.) sendControl(UdfControlResponse.newBuilder() .setCancel(CancelResponse.getDefaultInstance) .build()) responseLock.synchronized { responseObserver.onCompleted() } state = Done + case Done => + // FinishResponse already sent before Cancel arrived -- too late, + // engine will receive FinishResponse and should ignore this Cancel. + case _ => closeWithProtocolError(s"Cancel received in state $state") } @@ -357,9 +372,8 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { private val done = new CountDownLatch(1) @volatile var executionError: Option[ExecutionError] = None @volatile var streamError: Option[Throwable] = None - // Tracks whether Finish has been sent on the request stream so that a - // concurrent Cancel can suppress it (cancel-vs-finish race). - private val finishSent = new AtomicBoolean(false) + // Prevents calling onCompleted() more than once on the request stream. + private val requestCompleted = new AtomicBoolean(false) private val responseObserver = new StreamObserver[UdfResponse] { override def onNext(response: UdfResponse): Unit = { @@ -371,27 +385,21 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { ctrl.getControl match { case UdfControlResponse.Control.Init(_) => // InitResponse received: data phase can start. - // FINDING 6: The proto does not say whether the engine - // should start sending DataRequests before or after receiving - // InitResponse. In this implementation the engine sends all - // DataRequests eagerly after Init without waiting for - // InitResponse; this works because gRPC buffers them. But - // some implementations might want to wait for InitResponse - // before sending data (e.g., to use fields from InitResponse - // to configure the data pipeline). The proto should clarify - // whether the engine MAY pipeline DataRequests before - // receiving InitResponse. + // FINDING 6 (resolved): the engine MAY pipeline DataRequests + // before InitResponse arrives. The worker buffers them and + // processes in arrival order once init succeeds; only discards + // them if init fails (ExecutionError before InitResponse). case UdfControlResponse.Control.Error(err) => - // FINDING 8: record the error; the engine still drives the - // terminator. If Finish was already sent the engine waits for - // FinishResponse. If not, the engine sends Cancel and waits for - // CancelResponse. The exception is raised after the terminator. + // FINDING 8 (resolved): record the error. If the request stream + // is still open, send Cancel so the worker aborts cleanly; + // otherwise (Finish already sent and stream closed) wait for + // FinishResponse. Raise the error after the terminator arrives. executionError = Some(err) - if (!finishSent.get()) { + if (!requestCompleted.get()) { sendCancel("aborting after ExecutionError") } - // Fall through: FinishResponse or CancelResponse will arrive. + // FinishResponse or CancelResponse will arrive next. case UdfControlResponse.Control.Finish(_) => done.countDown() @@ -418,14 +426,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { } } - // FINDING 7: After sending Finish, the engine must also half-close the - // request stream by calling requestObserver.onCompleted(). The proto - // says "Finish is the last message from the engine" but does not mention - // the gRPC half-close. If the engine never calls onCompleted(), the server's - // StreamObserver.onCompleted() is never triggered, which could leave the - // server hanging in some implementations. The proto should say: - // "The engine MUST half-close the request stream (call onCompleted on - // the gRPC stream) immediately after sending Finish or Cancel." private val requestObserver: StreamObserver[UdfRequest] = stub.execute(responseObserver) def sendInit(payloadBytes: Array[Byte], sendChunked: Boolean = false): Unit = { @@ -479,28 +479,32 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { .build()) def sendFinish(): Unit = { - if (finishSent.compareAndSet(false, true)) { - requestObserver.onNext(UdfRequest.newBuilder() - .setControl(UdfControlRequest.newBuilder() - .setFinish(Finish.getDefaultInstance) - .build()) + requestObserver.onNext(UdfRequest.newBuilder() + .setControl(UdfControlRequest.newBuilder() + .setFinish(Finish.getDefaultInstance) .build()) - requestObserver.onCompleted() - } + .build()) + // Do NOT close the request stream here: Cancel may still follow Finish. + // Call completeRequestStream() explicitly when no Cancel will follow. } + // Cancel may be sent either before Finish (mid-stream abort) or after + // Finish (abort of already-submitted data). Always closes the request stream. def sendCancel(reason: String = ""): Unit = { - // Only send Cancel if Finish hasn't already been sent (race guard). - if (finishSent.compareAndSet(false, true)) { - requestObserver.onNext(UdfRequest.newBuilder() - .setControl(UdfControlRequest.newBuilder() - .setCancel(Cancel.newBuilder().setReason(reason).build()) - .build()) + requestObserver.onNext(UdfRequest.newBuilder() + .setControl(UdfControlRequest.newBuilder() + .setCancel(Cancel.newBuilder().setReason(reason).build()) .build()) + .build()) + completeRequestStream() + } + + // Explicitly half-closes the request stream. Call after sendFinish() when + // no Cancel will follow, or rely on sendCancel() which closes it implicitly. + def completeRequestStream(): Unit = { + if (requestCompleted.compareAndSet(false, true)) { requestObserver.onCompleted() } - // If Finish was already sent, suppress Cancel: rely on FinishResponse - // to terminate the stream (per the cancel-vs-finish race contract). } def awaitDone(timeoutMs: Long = 5000): Boolean = @@ -523,10 +527,11 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { client.sendInit("dummy-payload".getBytes) client.sendData("hello".getBytes) client.sendFinish() + client.completeRequestStream() assert(client.awaitDone(), "stream did not complete in time") assert(client.streamError.isEmpty, s"unexpected stream error: ${client.streamError}") - assert(client.executionError.isEmpty, s"unexpected code error: ${client.executionError}") + assert(client.executionError.isEmpty, s"unexpected execution error: ${client.executionError}") val results = client.drainResults() assert(results.length == 1) assert(new String(results.head) == "hello") @@ -537,6 +542,7 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { client.sendInit("dummy-payload".getBytes) Seq("batch1", "batch2", "batch3").foreach(b => client.sendData(b.getBytes)) client.sendFinish() + client.completeRequestStream() assert(client.awaitDone()) assert(client.streamError.isEmpty) @@ -544,11 +550,72 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { assert(results == Seq("batch1", "batch2", "batch3")) } + // In a real engine the producer (sending DataRequests) and consumer + // (collecting DataResponses) run concurrently on separate threads; gRPC's + // bidirectional streaming and HTTP/2 flow control manage the interleaving + // automatically. The engine drives the request side from a producer thread + // while the response observer fires on a gRPC-managed callback thread. + // No explicit coordination is needed between the two sides beyond the + // protocol-level ordering (InitResponse before DataResponse, etc.). + test("echo: concurrent sending and receiving (producer/consumer pattern)") { + // Use a non-direct executor so request and response handlers run on + // separate threads, demonstrating the concurrent interleaving. + val asyncStub = UdfWorkerGrpc.newStub(channel) // uses gRPC thread pool + + val receivedCount = new java.util.concurrent.atomic.AtomicInteger(0) + val doneLatch = new CountDownLatch(1) + @volatile var streamErr: Option[Throwable] = None + + val responseObs = new StreamObserver[UdfResponse] { + override def onNext(r: UdfResponse): Unit = r.getResponse match { + case UdfResponse.Response.Data(_) => receivedCount.incrementAndGet() + case UdfResponse.Response.Control(c) => + if (c.getControl.isInstanceOf[UdfControlResponse.Control.Finish]) doneLatch.countDown() + case _ => + } + override def onError(t: Throwable): Unit = { streamErr = Some(t); doneLatch.countDown() } + override def onCompleted(): Unit = doneLatch.countDown() + } + val reqObs = asyncStub.execute(responseObs) + + // Producer thread sends Init + batches concurrently with response callbacks. + val producer = new Thread(() => { + reqObs.onNext(UdfRequest.newBuilder() + .setControl(UdfControlRequest.newBuilder() + .setInit(Init.newBuilder() + .setProtocolVersion(SUPPORTED_VERSION) + .setDataFormat(UDFWorkerDataFormat.ARROW) + .setUdf(UdfPayload.newBuilder() + .setPayload(ByteString.copyFromUtf8("payload")) + .setFormat("echo").build()) + .build()) + .build()) + .build()) + (1 to 5).foreach { i => + reqObs.onNext(UdfRequest.newBuilder() + .setData(DataRequest.newBuilder() + .setData(ByteString.copyFromUtf8(s"batch-$i")).build()) + .build()) + } + reqObs.onNext(UdfRequest.newBuilder() + .setControl(UdfControlRequest.newBuilder() + .setFinish(Finish.getDefaultInstance).build()) + .build()) + reqObs.onCompleted() + }, "producer") + producer.start() + + assert(doneLatch.await(10, TimeUnit.SECONDS), "stream did not complete") + assert(streamErr.isEmpty, s"unexpected error: $streamErr") + assert(receivedCount.get() == 5, s"expected 5 echoes, got ${receivedCount.get()}") + } + test("echo: chunked payload delivery") { val client = new EngineClient(stub) client.sendInit("chunked-payload".getBytes, sendChunked = true) client.sendData("data".getBytes) client.sendFinish() + client.completeRequestStream() assert(client.awaitDone()) assert(client.streamError.isEmpty) @@ -558,43 +625,52 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { test("echo: generator-style UDF (zero DataRequests, engine sends Finish after Init)") { val client = new EngineClient(stub) client.sendInit("generator-payload".getBytes) - // No DataRequests at all -- engine sends Finish directly after Init. client.sendFinish() + client.completeRequestStream() assert(client.awaitDone()) assert(client.streamError.isEmpty) assert(client.drainResults().isEmpty) } - test("cancel: engine cancels mid-stream") { + test("cancel: engine cancels mid-stream before sending Finish") { val client = new EngineClient(stub) client.sendInit("dummy-payload".getBytes) client.sendData("batch1".getBytes) - client.sendCancel("task interrupted") + client.sendCancel("task interrupted") // closes request stream implicitly assert(client.awaitDone()) assert(client.streamError.isEmpty) - // Results before Cancel may or may not have arrived; no guarantee. } - test("cancel-vs-finish race: second call is suppressed") { + // Cancel MAY follow Finish. The engine sends all data, then sends Finish, + // then sends Cancel (e.g. task interrupted while waiting for FinishResponse). + // The worker responds with CancelResponse if Cancel arrives before + // FinishResponse is sent, or with FinishResponse if it arrived too late. + // The engine MUST accept either response. + test("cancel: engine sends Cancel after Finish -- accepts FinishResponse or CancelResponse") { val client = new EngineClient(stub) client.sendInit("dummy-payload".getBytes) client.sendData("data".getBytes) client.sendFinish() - client.sendCancel("late cancel -- should be suppressed") - - assert(client.awaitDone()) - // Only FinishResponse should arrive; the Cancel was suppressed by finishSent. - assert(client.streamError.isEmpty) + // Cancel follows Finish immediately. With the synchronous in-process + // executor the worker will have already sent FinishResponse, so the + // engine receives FinishResponse. In an async worker Cancel could win. + client.sendCancel("task interrupted after finish") // also closes stream + + assert(client.awaitDone(), "stream did not complete") + assert(client.streamError.isEmpty, + s"Cancel-after-Finish must not cause a gRPC error: ${client.streamError}") + // Either FinishResponse (if worker already completed) or CancelResponse + // (if Cancel won the race) is a correct outcome. } test("ExecutionError: error trigger causes UserError, engine sends Cancel, gets CancelResponse") { val client = new EngineClient(stub) client.sendInit("dummy-payload".getBytes) client.sendData(ERROR_TRIGGER.toByteArray) - // Engine has NOT sent Finish yet; on receiving ExecutionError it will send - // Cancel and wait for CancelResponse before raising the error. + // Engine has NOT closed the request stream; on receiving ExecutionError it + // will send Cancel. The engine raises the error after CancelResponse. assert(client.awaitDone()) assert(client.streamError.isEmpty, s"expected no gRPC error, got ${client.streamError}") @@ -603,20 +679,18 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { assert(client.executionError.get.getUser.getErrorClass == "SimulatedError") } - test("protocol error: second Init is rejected") { + test("protocol error: second Init is rejected with ProtocolError + FinishResponse") { val client = new EngineClient(stub) client.sendInit("payload".getBytes) - // Send Init again -- protocol violation. - client.sendInit("second-init".getBytes) + client.sendInit("second-init".getBytes) // protocol violation assert(client.awaitDone()) - assert(client.streamError.isDefined, "expected a gRPC error for double Init") + assert(client.streamError.isEmpty, "expected ProtocolError, not a gRPC stream error") + assert(client.executionError.isDefined, "expected an ExecutionError") + assert(client.executionError.get.hasProtocol, "expected ProtocolError kind") } test("Manage: heartbeat is acknowledged") { - val latch = new CountDownLatch(1) - @volatile var response: WorkerResponse = null - val blockingStub = UdfWorkerGrpc.newBlockingStub(channel) val resp = blockingStub.manage(WorkerRequest.newBuilder() .setHeartbeat(Heartbeat.getDefaultInstance) diff --git a/udf/worker/proto/src/main/protobuf/udf_protocol.proto b/udf/worker/proto/src/main/protobuf/udf_protocol.proto index 991141e44b540..153d99eec7326 100644 --- a/udf/worker/proto/src/main/protobuf/udf_protocol.proto +++ b/udf/worker/proto/src/main/protobuf/udf_protocol.proto @@ -48,9 +48,11 @@ service UdfWorker { // only [[CancelResponse]]. // // Stream lifecycle: the engine MUST half-close the request stream - // (call onCompleted() on the gRPC stream) immediately after sending - // [[Finish]] or [[Cancel]]. This is the gRPC-level signal that no - // further requests will arrive and is distinct from the protocol-level + // (call onCompleted() on the gRPC stream) after its last request + // message. When [[Cancel]] does not follow [[Finish]], the engine + // calls onCompleted() after [[Finish]]. When [[Cancel]] follows + // [[Finish]], the engine calls onCompleted() after [[Cancel]]. + // This gRPC-level half-close is distinct from the protocol-level // terminator message. // // Response observer threading: gRPC does not permit concurrent calls @@ -100,7 +102,8 @@ message UdfResponse { // // Wire protocol for one Execute stream (both directions): // -// Engine -> Worker: Init -> PayloadChunk* -> (DataRequest)* -> (Finish | Cancel) +// Engine -> Worker: Init -> PayloadChunk* -> (DataRequest)* -> Finish (Cancel)? +// | Cancel // Worker -> Engine: InitResponse -> (DataResponse)* -> (ExecutionError)? -> (FinishResponse | CancelResponse) // // DataRequest and DataResponse are independent streams: the worker @@ -129,9 +132,12 @@ message UdfResponse { // FinishResponse or CancelResponse accordingly. At most one // ExecutionError is sent per stream; the worker aggregates multiple // errors internally. -// - Exactly one terminator pair: Finish -> FinishResponse OR -// Cancel -> CancelResponse (never both on the same stream; see [[Finish]] -// for the cancel-vs-finish race contract on the engine side). +// - The engine terminates with one of: +// (a) Finish alone -> worker sends FinishResponse. +// (b) Cancel alone -> worker sends CancelResponse. +// (c) Finish then Cancel -> worker sends CancelResponse if it has not +// yet sent FinishResponse, otherwise FinishResponse (see [[Finish]]). +// Cancel MUST NOT precede Finish on the same stream. // // A worker that receives messages out of order (e.g. a second Init, // a PayloadChunk after the first DataRequest, a DataRequest before Init, @@ -368,6 +374,10 @@ message PayloadChunk { // the payload. The wire format (Arrow IPC etc.) is declared once per // session via [[Init.data_format]] and stays the same for the life // of the stream. +// +// Backpressure: this protocol currently relies on gRPC's transport-level +// (HTTP/2) flow control for backpressure. Application-level backpressure +// is not yet defined and may be introduced in a future revision. // ===================================================================== // Engine -> Worker per-batch payload. @@ -396,23 +406,23 @@ message DataResponse { // Finish / Cancel phase // ===================================================================== -// Sent by the engine when no more input batches will arrive. The -// worker MUST drain any remaining output, then emit -// [[FinishResponse]] and close the response stream. +// Sent by the engine when all input data has been submitted and normal +// completion is expected. The worker MUST drain any remaining output, +// then emit [[FinishResponse]] and close the response stream. // -// Exactly one of [[Finish]] or [[Cancel]] is sent per Execute stream; -// they are mutually exclusive. If the engine has already sent -// [[Finish]] it MUST NOT send [[Cancel]] afterwards (and vice versa). +// [[Cancel]] MAY follow [[Finish]] on the same stream if the engine +// wants to abort processing of already-submitted data (e.g. a Spark +// task is interrupted after all input batches were sent). [[Cancel]] +// MUST NOT precede [[Finish]]; if the engine cancels before sending +// all data it sends [[Cancel]] without [[Finish]]. // -// The mutual-exclusion rule is a wire-level contract: workers never -// observe both messages on the same Execute stream. Engine-side -// implementations that drive [[Finish]] and [[Cancel]] from different -// threads (e.g. a session driver vs. a task-interruption listener) are -// responsible for ensuring that at most one of the two reaches the -// transport. A typical implementation tracks whether [[Finish]] has -// already been written and, if so, suppresses any subsequent [[Cancel]] -// (relying on [[FinishResponse]] to terminate the stream); equivalently, -// a [[Cancel]] that wins the race suppresses the implicit [[Finish]]. +// Worker behavior when [[Cancel]] follows [[Finish]]: +// - If [[FinishResponse]] has not yet been sent, the worker MUST +// abort output, run cleanup, and send [[CancelResponse]]. +// - If [[FinishResponse]] has already been sent, [[Cancel]] arrives +// too late and is ignored; the engine receives [[FinishResponse]]. +// The engine MUST therefore be prepared to receive either +// [[FinishResponse]] or [[CancelResponse]] when it sends both. message Finish {} // Worker -> Engine completion message. May carry summary metrics. @@ -437,11 +447,12 @@ message FinishResponse { // worker MUST stop emitting [[DataResponse]] messages, run cleanup, // and emit [[CancelResponse]] before closing. // -// Exactly one of [[Finish]] or [[Cancel]] is sent per Execute stream; -// see [[Finish]]. [[Cancel]] is the cooperative cancellation path; a -// broken gRPC connection is the involuntary fallback -- in that case -// gRPC surfaces an error on the stream (see the error contract on -// [[Execute]]). +// [[Cancel]] is the cooperative cancellation path and may be sent +// either instead of [[Finish]] (engine cancels before all data is +// submitted) or after [[Finish]] (engine aborts processing of +// already-submitted data -- see [[Finish]] for the full contract). +// A broken gRPC connection is the involuntary fallback -- in that +// case gRPC surfaces an error on the stream (see [[Execute]]). // // Worker behavior on involuntary stream error: when the worker observes // a gRPC error on the Execute stream (i.e. the engine-side connection From 4e2cd66ebe6fc871494a272386b57ab2615b4c5b Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Wed, 13 May 2026 13:33:30 +0000 Subject: [PATCH 09/13] cleanup the onComplete location. --- .../udf/worker/core/EchoProtocolSuite.scala | 34 ++++++++++++------- .../src/main/protobuf/udf_protocol.proto | 12 +++---- 2 files changed, 27 insertions(+), 19 deletions(-) diff --git a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala index e4df0c1a87e22..44bba226b9859 100644 --- a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala +++ b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala @@ -402,9 +402,14 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { // FinishResponse or CancelResponse will arrive next. case UdfControlResponse.Control.Finish(_) => + // Session terminated cleanly. Now safe to half-close the + // request stream -- no further Cancel can follow. + completeRequestStream() done.countDown() case UdfControlResponse.Control.Cancel(_) => + // CancelResponse received: outcome is known, half-close now. + completeRequestStream() done.countDown() case _ => @@ -415,13 +420,14 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { } override def onError(t: Throwable): Unit = { + // gRPC error: connection broken. Half-close the request stream + // (idempotent if sendCancel already did it) and signal done. streamError = Some(t) + completeRequestStream() done.countDown() } override def onCompleted(): Unit = { - // Server closed the response stream. If done hasn't been counted down - // yet (no FinishResponse / CancelResponse seen), treat as unexpected. done.countDown() } } @@ -484,23 +490,29 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { .setFinish(Finish.getDefaultInstance) .build()) .build()) - // Do NOT close the request stream here: Cancel may still follow Finish. - // Call completeRequestStream() explicitly when no Cancel will follow. + // Do NOT half-close the request stream here. The engine does not yet + // know whether Cancel will follow, and even if it does, the rule is + // uniform: completeRequestStream() is always called by the response + // observer upon receiving FinishResponse, CancelResponse, or a gRPC + // error -- never based on what the engine sent. } - // Cancel may be sent either before Finish (mid-stream abort) or after - // Finish (abort of already-submitted data). Always closes the request stream. + // Cancel may be sent instead of Finish (mid-stream abort) or after Finish + // (abort of already-submitted data). The request stream is NOT half-closed + // here: completeRequestStream() is called by the response observer when + // CancelResponse, FinishResponse, or a gRPC error arrives, keeping the + // half-close rule uniform across all termination paths. def sendCancel(reason: String = ""): Unit = { requestObserver.onNext(UdfRequest.newBuilder() .setControl(UdfControlRequest.newBuilder() .setCancel(Cancel.newBuilder().setReason(reason).build()) .build()) .build()) - completeRequestStream() } - // Explicitly half-closes the request stream. Call after sendFinish() when - // no Cancel will follow, or rely on sendCancel() which closes it implicitly. + // Half-closes the request stream. Always called by the response observer + // upon receiving FinishResponse, CancelResponse, or a gRPC error -- + // never triggered by what the engine sent. Idempotent. def completeRequestStream(): Unit = { if (requestCompleted.compareAndSet(false, true)) { requestObserver.onCompleted() @@ -527,7 +539,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { client.sendInit("dummy-payload".getBytes) client.sendData("hello".getBytes) client.sendFinish() - client.completeRequestStream() assert(client.awaitDone(), "stream did not complete in time") assert(client.streamError.isEmpty, s"unexpected stream error: ${client.streamError}") @@ -542,7 +553,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { client.sendInit("dummy-payload".getBytes) Seq("batch1", "batch2", "batch3").foreach(b => client.sendData(b.getBytes)) client.sendFinish() - client.completeRequestStream() assert(client.awaitDone()) assert(client.streamError.isEmpty) @@ -615,7 +625,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { client.sendInit("chunked-payload".getBytes, sendChunked = true) client.sendData("data".getBytes) client.sendFinish() - client.completeRequestStream() assert(client.awaitDone()) assert(client.streamError.isEmpty) @@ -626,7 +635,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { val client = new EngineClient(stub) client.sendInit("generator-payload".getBytes) client.sendFinish() - client.completeRequestStream() assert(client.awaitDone()) assert(client.streamError.isEmpty) diff --git a/udf/worker/proto/src/main/protobuf/udf_protocol.proto b/udf/worker/proto/src/main/protobuf/udf_protocol.proto index 153d99eec7326..d96e55aeca5a7 100644 --- a/udf/worker/proto/src/main/protobuf/udf_protocol.proto +++ b/udf/worker/proto/src/main/protobuf/udf_protocol.proto @@ -48,12 +48,12 @@ service UdfWorker { // only [[CancelResponse]]. // // Stream lifecycle: the engine MUST half-close the request stream - // (call onCompleted() on the gRPC stream) after its last request - // message. When [[Cancel]] does not follow [[Finish]], the engine - // calls onCompleted() after [[Finish]]. When [[Cancel]] follows - // [[Finish]], the engine calls onCompleted() after [[Cancel]]. - // This gRPC-level half-close is distinct from the protocol-level - // terminator message. + // (call onCompleted() on the gRPC stream) after the session + // terminates: on receiving [[FinishResponse]] or [[CancelResponse]] + // (clean termination) or on receiving a gRPC error (connection + // broke). Deferring the half-close until the outcome is known keeps + // the request stream open long enough for [[Cancel]] to follow + // [[Finish]] if needed. // // Response observer threading: gRPC does not permit concurrent calls // to the response StreamObserver. Worker implementations that dispatch From 62948d8817f21c1969bb1810907e0f161a29dced Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Wed, 13 May 2026 18:42:11 +0000 Subject: [PATCH 10/13] retire some outdated comments. --- .../udf/worker/core/EchoProtocolSuite.scala | 214 +++++------------- 1 file changed, 58 insertions(+), 156 deletions(-) diff --git a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala index 44bba226b9859..b06d509206b1f 100644 --- a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala +++ b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala @@ -21,11 +21,8 @@ import java.util.concurrent.atomic.AtomicBoolean import com.google.protobuf.ByteString -// NOTE: These imports require adding grpc-stub + grpc-inprocess to the pom. -// The proto module currently only runs protoc (no grpc-java plugin), so -// UdfWorkerGrpc does not exist yet. Adding it requires: -// 1. grpc-java codegen plugin in udf/worker/proto/pom.xml -// 2. grpc-stub + grpc-netty (or grpc-inprocess) deps in udf/worker/core/pom.xml +// Requires grpc-stub and grpc-inprocess dependencies, plus grpc-java codegen +// in udf/worker/proto/pom.xml to generate UdfWorkerGrpc. import io.grpc.stub.StreamObserver import io.grpc.{ManagedChannel, Server, Status} import io.grpc.inprocess.{InProcessChannelBuilder, InProcessServerBuilder} @@ -45,21 +42,19 @@ import org.scalatest.funsuite.AnyFunSuite import org.scalatest.BeforeAndAfterEach /** - * Validates the UDF gRPC protocol by implementing a dummy echo worker and - * an engine client. The worker echoes each DataRequest batch back as a - * DataResponse. Error paths (ExecutionError, Cancel) are exercised with fake - * triggers. + * Protocol validation test for the UDF gRPC execution protocol. * - * FINDINGS -- things that were unclear or missing from the proto when - * writing this implementation: - * See the FINDING comments throughout the file. + * Implements a minimal echo worker (gRPC server) and engine client to verify + * the full Execute stream lifecycle: init, data streaming, finish, cancel, + * error handling, and the Manage RPC. The worker echoes each DataRequest + * batch back as a DataResponse; error paths are triggered by a sentinel + * payload value. */ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { // scalastyle:on funsuite private val SUPPORTED_VERSION: Int = 1 - // Trigger word: a DataRequest whose payload equals this string causes the - // worker to emit ExecutionError + FinishResponse instead of echoing. + // A DataRequest whose payload equals this value triggers an ExecutionError. private val ERROR_TRIGGER: ByteString = ByteString.copyFromUtf8("ERROR") private var server: Server = _ @@ -89,15 +84,9 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { /** Worker state machine for one Execute stream. */ private sealed trait WorkerState private case object AwaitingInit extends WorkerState - // FINDING 1: The proto does not state explicitly what the worker does - // between Init and the first data message when no chunks are expected. - // The chunking section says "when the engine uses PayloadChunk at all, - // it MUST set last=true." So the worker enters Chunking only after it - // actually receives a PayloadChunk. Until then it stays in AwaitingData, - // ready to send InitResponse immediately when the first DataRequest or - // Finish arrives. The proto should state this explicitly: - // "If no PayloadChunk arrives, InitResponse MUST be sent before the - // first DataRequest or Finish is processed." + // After Init: stays here until the first PayloadChunk or DataRequest arrives. + // If is_chunking_payload is set, waits for PayloadChunk(last=true) before + // sending InitResponse. Otherwise sends InitResponse on the first DataRequest. private case class AwaitingData(initPayload: ByteString) extends WorkerState private case class Chunking(accumulated: ByteString) extends WorkerState private case object Data extends WorkerState @@ -122,14 +111,9 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { .build()) responseObserver.onCompleted() - case WorkerRequest.Manage.Shutdown(req) => - // FINDING 2: The proto says the worker SHOULD exit after all Execute - // streams terminate. But ShutdownResponse gives no way to indicate - // "acknowledged, draining" vs "acknowledged, already idle." A boolean - // or enum field on ShutdownResponse would make the worker's state - // visible to the engine without requiring a separate Heartbeat probe. + case WorkerRequest.Manage.Shutdown(_) => responseObserver.onNext(WorkerResponse.newBuilder() - .setShutdown(ShutdownResponse.getDefaultInstance) + .setShutdown(ShutdownResponse.newBuilder().setSessionsSettled(true).build()) .build()) responseObserver.onCompleted() @@ -145,13 +129,8 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { responseObserver: StreamObserver[UdfResponse]) extends StreamObserver[UdfRequest] { @volatile private var state: WorkerState = AwaitingInit - // Guards responseObserver: gRPC does not allow concurrent onNext calls. - // FINDING 3: The proto says DataRequest and DataResponse are "independent - // streams" and the worker may emit DataResponse at any time. In practice, - // if the worker dispatches processing to a thread pool, multiple threads - // could race to call responseObserver.onNext(). The proto does not mention - // this constraint. Worker implementations must serialize all writes to the - // response observer themselves. + // gRPC does not permit concurrent calls to the response StreamObserver. + // All writes are serialized through this lock. private val responseLock = new Object override def onNext(request: UdfRequest): Unit = { @@ -174,26 +153,11 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { private def handleInit(init: Init): Unit = state match { case AwaitingInit => - // FINDING 4 (resolved): unsupported protocol_version is now surfaced - // via ExecutionError(ProtocolError). The worker sends ProtocolError - // + FinishResponse, keeping the stream lifecycle intact. if (init.hasProtocolVersion && init.getProtocolVersion != SUPPORTED_VERSION) { - sendControl(UdfControlResponse.newBuilder() - .setError(ExecutionError.newBuilder() - .setProtocol(ProtocolError.newBuilder() - .setMessage(s"unsupported protocol version: ${init.getProtocolVersion}") - .build()) - .build()) - .build()) - sendControl(UdfControlResponse.newBuilder() - .setFinish(FinishResponse.getDefaultInstance) - .build()) - responseLock.synchronized { responseObserver.onCompleted() } - state = Done + closeWithProtocolError(s"unsupported protocol version: ${init.getProtocolVersion}") return } - val inlinePayload = init.getUdf.getPayload - state = AwaitingData(inlinePayload) + state = AwaitingData(init.getUdf.getPayload) case _ => closeWithProtocolError(s"Init received in state $state") } @@ -203,7 +167,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { case AwaitingData(existing) => val updated = existing.concat(chunk.getData) if (chunk.hasLast && chunk.getLast) { - // Payload is complete. Send InitResponse and move to Data. sendInitResponse() state = Data } else { @@ -224,9 +187,7 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { private def handleDataRequest(data: DataRequest): Unit = state match { case AwaitingData(_) => - // No chunks were sent. Send InitResponse before handling data, - // per the ordering invariant "InitResponse MUST be emitted before - // any DataResponse." + // No chunks were sent: send InitResponse before the first DataResponse. sendInitResponse() state = Data processEcho(data) @@ -238,8 +199,8 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { private def processEcho(data: DataRequest): Unit = { if (data.getData == ERROR_TRIGGER) { - // Simulate a user-code error: send ExecutionError(UserError), then - // wait for the engine's Finish or Cancel before sending the terminator. + // Signal a user-code error. Stay in Data so the engine's Finish or + // Cancel is still received and drives the stream terminator. sendControl(UdfControlResponse.newBuilder() .setError(ExecutionError.newBuilder() .setUser(UserError.newBuilder() @@ -248,12 +209,7 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { .build()) .build()) .build()) - // Transition to a "post-error, awaiting terminator" state reusing Done - // to stop further DataRequest processing. The actual FinishResponse or - // CancelResponse is sent when Finish or Cancel arrives from the engine. - state = Data // stay in Data to receive Finish/Cancel normally } else { - // Echo the raw bytes back. responseLock.synchronized { responseObserver.onNext(UdfResponse.newBuilder() .setData(DataResponse.newBuilder().setData(data.getData).build()) @@ -274,13 +230,11 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { case _ => closeWithProtocolError(s"Finish received in state $state") } - // Transitions to Finishing, performs any remaining work (for the echo - // worker this is instant), then sends FinishResponse. A real worker with - // async processing would stay in Finishing until the drain completes, - // allowing a concurrent Cancel to win if it arrives in time. + // Transitions to Finishing, drains any remaining output (instant for the + // echo worker), then sends FinishResponse. An async worker would remain in + // Finishing until the drain completes, allowing a concurrent Cancel to win. private def drainAndFinish(): Unit = { state = Finishing - // Echo worker: no buffered state to drain. Immediately complete. sendControl(UdfControlResponse.newBuilder() .setFinish(FinishResponse.newBuilder() .putMetrics("status", "ok") @@ -293,8 +247,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { private def handleCancel(cancel: Cancel): Unit = state match { case Data | AwaitingData(_) | Finishing => // Finishing: Cancel arrived while drain was in progress -- abort. - // (For the synchronous echo worker this race is not observable in - // tests, but the state machine is correct for async implementations.) sendControl(UdfControlResponse.newBuilder() .setCancel(CancelResponse.getDefaultInstance) .build()) @@ -302,31 +254,19 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { state = Done case Done => - // FinishResponse already sent before Cancel arrived -- too late, - // engine will receive FinishResponse and should ignore this Cancel. + // FinishResponse already sent; Cancel arrived too late and is ignored. case _ => closeWithProtocolError(s"Cancel received in state $state") } override def onError(t: Throwable): Unit = { - // gRPC transport error from the engine side (connection dropped). - // Per the protocol: treat as equivalent to Cancel for cleanup purposes; - // do NOT attempt to send CancelResponse (stream is dead). + // gRPC transport error (connection dropped). Treat as involuntary cancel: + // clean up and do not attempt to send any response. state = Done } - // FINDING 5: The proto does not specify what the worker should do when - // the gRPC request stream is half-closed by the engine (onCompleted) if - // the worker is still in an active state. In the normal flow the engine - // half-closes after sending Finish, so onCompleted arrives after the - // worker has already sent FinishResponse and moved to Done. But if - // onCompleted arrives unexpectedly (e.g. engine bug), the worker has no - // way to distinguish "normal half-close after Finish" from - // "engine closed stream early." The proto should say: if onCompleted is - // received while the worker is still in an active state it SHOULD treat - // it as a protocol error and close with a gRPC error. override def onCompleted(): Unit = state match { - case Done => // normal: Finish or Cancel was already processed + case Done => // normal: engine half-closed after session terminated case _ => closeWithProtocolError( s"request stream closed by engine in unexpected state $state") @@ -344,8 +284,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { } private def closeWithProtocolError(msg: String): Unit = { - // FINDING 5 (resolved): use ProtocolError instead of a gRPC error so - // the stream lifecycle (FinishResponse/CancelResponse) stays intact. state = Done sendControl(UdfControlResponse.newBuilder() .setError(ExecutionError.newBuilder() @@ -364,15 +302,18 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { // =========================================================================== /** - * Minimal engine client. Collects received DataResponse bytes and exposes - * latch-based synchronization so tests can wait for stream completion. + * Minimal engine client that drives the Execute stream and collects results. + * + * The request stream is half-closed (onCompleted) only after the session + * outcome is known from the server: on receiving FinishResponse, + * CancelResponse, or a gRPC error. This keeps the stream open long enough + * for Cancel to follow Finish when needed. */ private class EngineClient(stub: UdfWorkerGrpc.UdfWorkerStub) { private val results = new LinkedBlockingQueue[Array[Byte]]() private val done = new CountDownLatch(1) @volatile var executionError: Option[ExecutionError] = None @volatile var streamError: Option[Throwable] = None - // Prevents calling onCompleted() more than once on the request stream. private val requestCompleted = new AtomicBoolean(false) private val responseObserver = new StreamObserver[UdfResponse] { @@ -384,31 +325,22 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { case UdfResponse.Response.Control(ctrl) => ctrl.getControl match { case UdfControlResponse.Control.Init(_) => - // InitResponse received: data phase can start. - // FINDING 6 (resolved): the engine MAY pipeline DataRequests - // before InitResponse arrives. The worker buffers them and - // processes in arrival order once init succeeds; only discards - // them if init fails (ExecutionError before InitResponse). + // InitResponse received: data phase can begin. case UdfControlResponse.Control.Error(err) => - // FINDING 8 (resolved): record the error. If the request stream - // is still open, send Cancel so the worker aborts cleanly; - // otherwise (Finish already sent and stream closed) wait for - // FinishResponse. Raise the error after the terminator arrives. + // Record the error. If the request stream is still open, send + // Cancel so the worker can abort cleanly. The error is surfaced + // after the response terminator (FinishResponse or CancelResponse). executionError = Some(err) if (!requestCompleted.get()) { sendCancel("aborting after ExecutionError") } - // FinishResponse or CancelResponse will arrive next. case UdfControlResponse.Control.Finish(_) => - // Session terminated cleanly. Now safe to half-close the - // request stream -- no further Cancel can follow. completeRequestStream() done.countDown() case UdfControlResponse.Control.Cancel(_) => - // CancelResponse received: outcome is known, half-close now. completeRequestStream() done.countDown() @@ -420,8 +352,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { } override def onError(t: Throwable): Unit = { - // gRPC error: connection broken. Half-close the request stream - // (idempotent if sendCancel already did it) and signal done. streamError = Some(t) completeRequestStream() done.countDown() @@ -436,23 +366,19 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { def sendInit(payloadBytes: Array[Byte], sendChunked: Boolean = false): Unit = { if (sendChunked) { - // Send Init with empty inline payload, then PayloadChunk. - val initReq = UdfRequest.newBuilder() + requestObserver.onNext(UdfRequest.newBuilder() .setControl(UdfControlRequest.newBuilder() .setInit(Init.newBuilder() .setProtocolVersion(SUPPORTED_VERSION) + .setIsChunkingPayload(true) .setDataFormat(UDFWorkerDataFormat.ARROW) .setUdf(UdfPayload.newBuilder() - // Intentionally empty: payload arrives via chunks. .setPayload(ByteString.EMPTY) .setFormat("echo") .build()) .build()) .build()) - .build() - requestObserver.onNext(initReq) - - // Send payload as a single chunk with last=true. + .build()) requestObserver.onNext(UdfRequest.newBuilder() .setControl(UdfControlRequest.newBuilder() .setPayload(org.apache.spark.udf.worker.PayloadChunk.newBuilder() @@ -490,29 +416,20 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { .setFinish(Finish.getDefaultInstance) .build()) .build()) - // Do NOT half-close the request stream here. The engine does not yet - // know whether Cancel will follow, and even if it does, the rule is - // uniform: completeRequestStream() is always called by the response - // observer upon receiving FinishResponse, CancelResponse, or a gRPC - // error -- never based on what the engine sent. + // Request stream stays open: Cancel may still follow Finish. + // completeRequestStream() is called by the response observer. } - // Cancel may be sent instead of Finish (mid-stream abort) or after Finish - // (abort of already-submitted data). The request stream is NOT half-closed - // here: completeRequestStream() is called by the response observer when - // CancelResponse, FinishResponse, or a gRPC error arrives, keeping the - // half-close rule uniform across all termination paths. def sendCancel(reason: String = ""): Unit = { requestObserver.onNext(UdfRequest.newBuilder() .setControl(UdfControlRequest.newBuilder() .setCancel(Cancel.newBuilder().setReason(reason).build()) .build()) .build()) + // Request stream stays open until the response terminator arrives; + // completeRequestStream() is called by the response observer. } - // Half-closes the request stream. Always called by the response observer - // upon receiving FinishResponse, CancelResponse, or a gRPC error -- - // never triggered by what the engine sent. Idempotent. def completeRequestStream(): Unit = { if (requestCompleted.compareAndSet(false, true)) { requestObserver.onCompleted() @@ -560,17 +477,12 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { assert(results == Seq("batch1", "batch2", "batch3")) } - // In a real engine the producer (sending DataRequests) and consumer - // (collecting DataResponses) run concurrently on separate threads; gRPC's - // bidirectional streaming and HTTP/2 flow control manage the interleaving - // automatically. The engine drives the request side from a producer thread - // while the response observer fires on a gRPC-managed callback thread. - // No explicit coordination is needed between the two sides beyond the - // protocol-level ordering (InitResponse before DataResponse, etc.). + // The engine drives the request side from a producer thread while the + // response observer fires on a gRPC-managed callback thread. gRPC's + // bidirectional streaming and HTTP/2 flow control manage the interleaving; + // no explicit coordination is needed beyond the protocol ordering invariants. test("echo: concurrent sending and receiving (producer/consumer pattern)") { - // Use a non-direct executor so request and response handlers run on - // separate threads, demonstrating the concurrent interleaving. - val asyncStub = UdfWorkerGrpc.newStub(channel) // uses gRPC thread pool + val asyncStub = UdfWorkerGrpc.newStub(channel) val receivedCount = new java.util.concurrent.atomic.AtomicInteger(0) val doneLatch = new CountDownLatch(1) @@ -578,7 +490,7 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { val responseObs = new StreamObserver[UdfResponse] { override def onNext(r: UdfResponse): Unit = r.getResponse match { - case UdfResponse.Response.Data(_) => receivedCount.incrementAndGet() + case UdfResponse.Response.Data(_) => receivedCount.incrementAndGet() case UdfResponse.Response.Control(c) => if (c.getControl.isInstanceOf[UdfControlResponse.Control.Finish]) doneLatch.countDown() case _ => @@ -588,7 +500,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { } val reqObs = asyncStub.execute(responseObs) - // Producer thread sends Init + batches concurrently with response callbacks. val producer = new Thread(() => { reqObs.onNext(UdfRequest.newBuilder() .setControl(UdfControlRequest.newBuilder() @@ -645,40 +556,31 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { val client = new EngineClient(stub) client.sendInit("dummy-payload".getBytes) client.sendData("batch1".getBytes) - client.sendCancel("task interrupted") // closes request stream implicitly + client.sendCancel("task interrupted") assert(client.awaitDone()) assert(client.streamError.isEmpty) } - // Cancel MAY follow Finish. The engine sends all data, then sends Finish, - // then sends Cancel (e.g. task interrupted while waiting for FinishResponse). - // The worker responds with CancelResponse if Cancel arrives before - // FinishResponse is sent, or with FinishResponse if it arrived too late. - // The engine MUST accept either response. + // Cancel MAY follow Finish. The worker sends CancelResponse if Cancel arrives + // before FinishResponse is sent, or FinishResponse if it arrived too late. + // The engine must accept either outcome. test("cancel: engine sends Cancel after Finish -- accepts FinishResponse or CancelResponse") { val client = new EngineClient(stub) client.sendInit("dummy-payload".getBytes) client.sendData("data".getBytes) client.sendFinish() - // Cancel follows Finish immediately. With the synchronous in-process - // executor the worker will have already sent FinishResponse, so the - // engine receives FinishResponse. In an async worker Cancel could win. - client.sendCancel("task interrupted after finish") // also closes stream + client.sendCancel("task interrupted after finish") assert(client.awaitDone(), "stream did not complete") assert(client.streamError.isEmpty, s"Cancel-after-Finish must not cause a gRPC error: ${client.streamError}") - // Either FinishResponse (if worker already completed) or CancelResponse - // (if Cancel won the race) is a correct outcome. } - test("ExecutionError: error trigger causes UserError, engine sends Cancel, gets CancelResponse") { + test("ExecutionError: worker signals UserError, engine sends Cancel and receives CancelResponse") { val client = new EngineClient(stub) client.sendInit("dummy-payload".getBytes) client.sendData(ERROR_TRIGGER.toByteArray) - // Engine has NOT closed the request stream; on receiving ExecutionError it - // will send Cancel. The engine raises the error after CancelResponse. assert(client.awaitDone()) assert(client.streamError.isEmpty, s"expected no gRPC error, got ${client.streamError}") @@ -690,7 +592,7 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { test("protocol error: second Init is rejected with ProtocolError + FinishResponse") { val client = new EngineClient(stub) client.sendInit("payload".getBytes) - client.sendInit("second-init".getBytes) // protocol violation + client.sendInit("second-init".getBytes) assert(client.awaitDone()) assert(client.streamError.isEmpty, "expected ProtocolError, not a gRPC stream error") From 458f0f9f958742448040f029a0cbe846290461a4 Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Wed, 13 May 2026 20:32:52 +0000 Subject: [PATCH 11/13] address comments. --- udf/worker/README.md | 56 ++++++++++++++--- .../spark/udf/worker/core/WorkerSession.scala | 19 +++--- .../core/direct/DirectWorkerDispatcher.scala | 3 +- .../core/direct/DirectWorkerSession.scala | 2 +- .../udf/worker/core/EchoProtocolSuite.scala | 36 +++++++---- .../src/main/protobuf/udf_protocol.proto | 63 ++++++++++--------- 6 files changed, 117 insertions(+), 62 deletions(-) diff --git a/udf/worker/README.md b/udf/worker/README.md index 4946425826328..a121a4b66a0dc 100644 --- a/udf/worker/README.md +++ b/udf/worker/README.md @@ -19,7 +19,7 @@ WorkerDispatcher -- manages workers, creates sessions | v WorkerSession -- one UDF execution - | 1. session.init(Init proto: udf payload + data format + schemas) + | 1. session.init(Init proto: protocol version + udf payload + data format + schemas) | 2. val results = session.process(inputBatches) | 3. session.close() ``` @@ -58,20 +58,57 @@ obtaining workers from a provisioning service or daemon. ## Wire protocol -Each UDF execution uses a single bidirectional `Execute` gRPC stream: +Each UDF execution uses a single bidirectional `Execute` gRPC stream. + +### Message flow ``` -Engine -> Worker: Init -> PayloadChunk* -> (DataRequest)* -> (Finish | Cancel) +Engine -> Worker: Init -> PayloadChunk* -> (DataRequest)* -> Finish (Cancel)? + | Cancel Worker -> Engine: InitResponse -> (DataResponse)* -> (ExecutionError)? -> (FinishResponse | CancelResponse) ``` -`DataRequest` and `DataResponse` are independent streams: the worker may emit -`DataResponse` messages at any point after `InitResponse`, including before the -first `DataRequest` arrives. Generator-style UDFs may have zero `DataRequest` -messages, with the engine sending `Finish` directly after `Init`. -`PayloadChunk.last = true` is the canonical end-of-chunking signal. +- `Init.is_chunking_payload = true` signals that `PayloadChunk` messages will + follow. `PayloadChunk.last = true` is the canonical end-of-chunking signal. + When `is_chunking_payload` is absent or false, `InitResponse` is sent + immediately after `Init`. +- `DataRequest` and `DataResponse` are **independent streams**: the worker may + emit `DataResponse` at any point after `InitResponse`, including before any + `DataRequest` arrives. Generator-style UDFs may have zero `DataRequest` + messages. +- The engine MAY pipeline `DataRequests` before `InitResponse` arrives. The + worker MUST buffer them and process in order once init succeeds. +- **Cancel-after-Finish**: `Cancel` MAY follow `Finish`. The worker sends + `CancelResponse` if `Cancel` arrives before `FinishResponse` is sent; + otherwise `FinishResponse`. The engine must accept either. +- **`ExecutionError`** carries application-level errors (`UserError`, + `WorkerError`, `ProtocolError`). After sending it the worker waits for + the engine's `Finish` or `Cancel` before sending the terminator. +- **gRPC errors** indicate transport/connection failures only. Hanging UDFs + (connection alive, no response) require worker-side per-batch timeouts and + engine-side client timeouts. +- **Backpressure** is handled by gRPC's HTTP/2 transport-level flow control. + Application-level backpressure is not yet defined. + See `udf/worker/proto/src/main/protobuf/udf_protocol.proto` for the complete -ordering invariants, gRPC error contract, and cancel-vs-finish race contract. +ordering invariants and error contract. + +### Worker state machine (Execute stream) + +A worker implementation tracks one of these states per `Execute` stream: + +| State | Meaning | Enters on | Exits on | +|-------|---------|-----------|----------| +| `AwaitingInit` | Stream opened, no `Init` received yet | stream open | `Init` received | +| `AwaitingData` | Chunked-payload mode: waiting for `PayloadChunk` messages | `Init` with `is_chunking_payload=true` | `PayloadChunk(last=true)` → send `InitResponse` | +| `Chunking` | Non-final chunks accumulating | `PayloadChunk(last=false)` | `PayloadChunk(last=true)` → send `InitResponse` | +| `Data` | Init complete; processing `DataRequest` / emitting `DataResponse` | `Init` (inline payload) or last `PayloadChunk` | `Finish` or `Cancel` | +| `Finishing` | `Finish` received; draining remaining output | `Finish` | drain complete → send `FinishResponse`; or `Cancel` wins → send `CancelResponse` | +| `Done` | Stream closed | `FinishResponse` or `CancelResponse` sent, or `onError` | — | + +Any protocol violation in an active state sends `ExecutionError(ProtocolError)` + +`FinishResponse` and transitions to `Done`. A gRPC transport error (`onError`) +transitions to `Done` without sending any response. ### Direct worker creation @@ -133,6 +170,7 @@ val session = dispatcher.createSession(securityScope = None) try { // 4. Initialize with the serialized function and schemas. session.init(Init.newBuilder() + .setProtocolVersion(1) .setUdf(UdfPayload.newBuilder() .setPayload(ByteString.copyFrom(serializedFunction)) .setFormat(payloadFormat) // worker-recognised tag diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala index f60fbca129dd1..bef5a88c45122 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala @@ -39,6 +39,7 @@ import org.apache.spark.udf.worker.Init * val session = dispatcher.createSession(securityScope = None) * try { * session.init(Init.newBuilder() + * .setProtocolVersion(1) * .setUdf(UdfPayload.newBuilder().setPayload(callable).setFormat(fmt).build()) * .setDataFormat(UDFWorkerDataFormat.ARROW) * .build()) @@ -60,12 +61,12 @@ import org.apache.spark.udf.worker.Init * do not need to coordinate with the thread driving [[process]]. * * [[cancel]] may be called even after all input data has been - * submitted (i.e. after [[Finish]] has been sent on the transport). - * In that case implementations MUST send [[Cancel]] on the transport - * if [[FinishResponse]] has not yet been received, and MUST be - * prepared to receive either [[FinishResponse]] or [[CancelResponse]]. - * See the [[org.apache.spark.udf.worker.Finish]] proto message for - * the full contract. + * submitted (i.e. after `Finish` has been sent on the transport). + * In that case implementations MUST send `Cancel` on the transport + * if `FinishResponse` has not yet been received, and MUST be + * prepared to receive either `FinishResponse` or `CancelResponse`. + * See the `Finish` proto message in `udf_protocol.proto` for the + * full contract. * * The lifecycle is enforced here: [[init]] and [[process]] are `final` * and delegate to [[doInit]] / [[doProcess]] after AtomicBoolean guards. @@ -166,10 +167,8 @@ abstract class WorkerSession extends AutoCloseable { * -- a no-op; the stream is already terminated. * * Implementations are responsible for the lifecycle-aware behavior - * described above (no-op outside the active window; cancellation - * thrown from a subsequent [[process]] when applicable) so that - * callers (e.g. task interruption listeners) do not need to - * coordinate with the thread driving [[process]]. + * described above so that callers (e.g. task interruption listeners) + * do not need to coordinate with the thread driving [[process]]. */ def cancel(): Unit diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerDispatcher.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerDispatcher.scala index afaf23791d80f..14db8da7ac89e 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerDispatcher.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerDispatcher.scala @@ -373,7 +373,8 @@ abstract class DirectWorkerDispatcher( "DirectWorker.runner must have at least one entry in command or arguments") val workerId = UUID.randomUUID().toString val address = newEndpointAddress(workerId) - // Proto contract: the engine must pass --id and --connection. + // The engine injects --connection (the socket address it manages) and + // --id (an internal correlation identifier) into the worker command. val cmd = baseCmd ++ Seq("--id", workerId, "--connection", address) val env = runner.getEnvironmentVariablesMap.asScala.toMap val outputFile = Files.createTempFile("udf-worker-", ".log") diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerSession.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerSession.scala index 5de1f486e832e..de1dc45b5a8d3 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerSession.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/direct/DirectWorkerSession.scala @@ -27,7 +27,7 @@ import org.apache.spark.udf.worker.core.{WorkerConnection, WorkerSession} * * This is the session type returned by [[DirectWorkerDispatcher]]. It ties * the session lifecycle to the worker's ref-count: the dispatcher increments - * the count before construction, and [[close]] decrements it, so the + * the count before construction, and [[doClose]] decrements it, so the * dispatcher knows when a worker process is idle and can be terminated or * reused. * diff --git a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala index b06d509206b1f..8aac8e565c1af 100644 --- a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala +++ b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala @@ -84,9 +84,8 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { /** Worker state machine for one Execute stream. */ private sealed trait WorkerState private case object AwaitingInit extends WorkerState - // After Init: stays here until the first PayloadChunk or DataRequest arrives. - // If is_chunking_payload is set, waits for PayloadChunk(last=true) before - // sending InitResponse. Otherwise sends InitResponse on the first DataRequest. + // Chunking mode only: Init received with is_chunking_payload=true. + // Accumulates PayloadChunk messages until last=true, then sends InitResponse. private case class AwaitingData(initPayload: ByteString) extends WorkerState private case class Chunking(accumulated: ByteString) extends WorkerState private case object Data extends WorkerState @@ -157,7 +156,15 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { closeWithProtocolError(s"unsupported protocol version: ${init.getProtocolVersion}") return } - state = AwaitingData(init.getUdf.getPayload) + if (init.getIsChunkingPayload) { + // Payload will arrive via PayloadChunk messages; wait for last=true + // before sending InitResponse. + state = AwaitingData(init.getUdf.getPayload) + } else { + // Payload is fully inline; send InitResponse immediately. + sendInitResponse() + state = Data + } case _ => closeWithProtocolError(s"Init received in state $state") } @@ -186,12 +193,6 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { } private def handleDataRequest(data: DataRequest): Unit = state match { - case AwaitingData(_) => - // No chunks were sent: send InitResponse before the first DataResponse. - sendInitResponse() - state = Data - processEcho(data) - case Data => processEcho(data) case _ => closeWithProtocolError(s"DataRequest received in state $state") @@ -220,7 +221,9 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { private def handleFinish(): Unit = state match { case AwaitingData(_) => - // Generator-style UDF: engine sends Finish directly after Init. + // Chunking path: engine sends Finish before all chunks arrived. + // Send InitResponse now (treating the partial payload as complete) + // then finish normally. sendInitResponse() drainAndFinish() @@ -260,8 +263,15 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { } override def onError(t: Throwable): Unit = { - // gRPC transport error (connection dropped). Treat as involuntary cancel: - // clean up and do not attempt to send any response. + // gRPC transport error: the engine-side connection dropped. + // The stream is already dead so the worker MUST NOT attempt to send + // CancelResponse or any other message. However, the worker MUST still + // run the same cleanup it would perform on an explicit Cancel: + // - stop any in-progress UDF execution + // - release file handles, locks, and temporary state + // - free any buffers holding input batches not yet processed + // A real worker would invoke its internal cancel/cleanup path here. + // The echo worker has no such resources, so only the state is updated. state = Done } diff --git a/udf/worker/proto/src/main/protobuf/udf_protocol.proto b/udf/worker/proto/src/main/protobuf/udf_protocol.proto index d96e55aeca5a7..f2277f469c643 100644 --- a/udf/worker/proto/src/main/protobuf/udf_protocol.proto +++ b/udf/worker/proto/src/main/protobuf/udf_protocol.proto @@ -34,26 +34,21 @@ option java_multiple_files = true; // The default UDF gRPC service. A worker that exposes this service // MUST do so over the default connection of the worker specification. // -// In future, additional connections (e.g. a separate channel) may be -// reserved by the worker spec for other purposes. +// Future revisions of the worker specification may introduce additional +// dedicated connections for specific purposes (e.g. a separate channel +// for streaming state store access in stateful UDFs). service UdfWorker { // Per-execution stream. See [[UdfControlRequest]] for the complete // wire protocol and ordering invariants. // - // Error contract: if the gRPC connection breaks at any point, gRPC - // surfaces an error on the stream. The engine therefore never needs - // to poll or time out waiting for a response -- the absence of a - // gRPC error guarantees that a proper protocol response will - // eventually arrive. This applies to every in-flight response, not - // only [[CancelResponse]]. - // - // Stream lifecycle: the engine MUST half-close the request stream - // (call onCompleted() on the gRPC stream) after the session - // terminates: on receiving [[FinishResponse]] or [[CancelResponse]] - // (clean termination) or on receiving a gRPC error (connection - // broke). Deferring the half-close until the outcome is known keeps - // the request stream open long enough for [[Cancel]] to follow - // [[Finish]] if needed. + // Error contract: a gRPC error on this stream indicates a transport + // or connection failure. Application-level errors (user code exceptions, + // worker errors, protocol violations) are communicated via + // [[ExecutionError]] so the stream lifecycle remains intact. + // Note: a hanging UDF keeps the connection alive and therefore does + // not produce a gRPC error. Worker-side per-batch timeouts are the + // appropriate mechanism for detecting and surfacing hung user code; + // the engine should also apply client-side timeouts as a safety net. // // Response observer threading: gRPC does not permit concurrent calls // to the response StreamObserver. Worker implementations that dispatch @@ -64,10 +59,11 @@ service UdfWorker { // stream, mapping to a `WorkerSession` on the engine side. rpc Execute(stream UdfRequest) returns (stream UdfResponse); - // Worker-scoped management RPC, independent of any per-execution - // stream. Used for heartbeat, capability query, and graceful - // shutdown. Kept unary so it does not depend on the lifecycle of an - // active Execute stream. + // Worker-scoped management RPC for heartbeat, capability query, and + // graceful shutdown. Workers MUST ensure this RPC remains serviceable + // regardless of how many [[Execute]] streams are in flight; failing to + // do so can prevent the engine from detecting a hung worker or + // initiating a clean shutdown. rpc Manage(WorkerRequest) returns (WorkerResponse); } @@ -137,7 +133,6 @@ message UdfResponse { // (b) Cancel alone -> worker sends CancelResponse. // (c) Finish then Cancel -> worker sends CancelResponse if it has not // yet sent FinishResponse, otherwise FinishResponse (see [[Finish]]). -// Cancel MUST NOT precede Finish on the same stream. // // A worker that receives messages out of order (e.g. a second Init, // a PayloadChunk after the first DataRequest, a DataRequest before Init, @@ -425,10 +420,21 @@ message DataResponse { // [[FinishResponse]] or [[CancelResponse]] when it sends both. message Finish {} -// Worker -> Engine completion message. May carry summary metrics. +// Worker -> Engine completion message. Carries per-execution summary metrics. +// +// Metrics design: +// - [[FinishResponse.metrics]]: per-execution metrics accumulated over the +// full session (e.g. rows in/out, time per phase). Emitted once at the +// end of an [[Execute]] stream. +// - [[HeartbeatResponse.metrics]]: worker-global metrics aggregated across +// all sessions (e.g. total rows processed, memory usage). Emitted +// periodically via the [[Manage]] RPC. +// - Real-time per-execution metrics (e.g. incremental progress during a +// long-running UDF) are not yet defined. They may be introduced as an +// optional message in a future revision without changing this contract. message FinishResponse { - // Final metrics aggregated over the whole session (e.g. rows - // in/out, time per phase). Free-form; names are worker-defined. + // Per-execution metrics accumulated over the session. Free-form; + // names are worker-defined. map metrics = 1; // (Optional) Inline finish result returned by the worker. @@ -640,10 +646,11 @@ message Heartbeat { // Acknowledgment for [[Heartbeat]]. message HeartbeatResponse { - // Reserved for future additive fields (e.g. a server-side load - // hint to inform scheduling decisions, or a matching sequence - // number echoed from [[Heartbeat]]). - reserved 1; + // (Optional) Worker-global metrics aggregated across all active sessions + // (e.g. total rows processed, active session count, memory usage). + // Complements the per-execution metrics in [[FinishResponse.metrics]]. + // Free-form; names are worker-defined. + map metrics = 1; } // Engine-initiated graceful shutdown request. This lets the worker From d9b284de1c676ab99029151b3c530b42ad1eaac1 Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Thu, 14 May 2026 10:08:39 +0000 Subject: [PATCH 12/13] more fixes. --- udf/worker/README.md | 19 ++- .../spark/udf/worker/core/WorkerSession.scala | 2 + .../udf/worker/core/EchoProtocolSuite.scala | 80 ++++++---- .../src/main/protobuf/udf_protocol.proto | 138 ++++++++++++------ 4 files changed, 164 insertions(+), 75 deletions(-) diff --git a/udf/worker/README.md b/udf/worker/README.md index a121a4b66a0dc..c28eeef3e6009 100644 --- a/udf/worker/README.md +++ b/udf/worker/README.md @@ -82,8 +82,12 @@ Worker -> Engine: InitResponse -> (DataResponse)* -> (ExecutionError)? `CancelResponse` if `Cancel` arrives before `FinishResponse` is sent; otherwise `FinishResponse`. The engine must accept either. - **`ExecutionError`** carries application-level errors (`UserError`, - `WorkerError`, `ProtocolError`). After sending it the worker waits for - the engine's `Finish` or `Cancel` before sending the terminator. + `WorkerError`, `ProtocolError`) that occur during data processing + (before the finish or cancel callback is invoked). The terminator is + always `CancelResponse` — the engine MUST send `Cancel` (or + Cancel-after-Finish) and the worker responds with `CancelResponse`. + Errors in the finish or cancel callback are reported via + `FinishResponse.error` / `CancelResponse.error` instead. - **gRPC errors** indicate transport/connection failures only. Hanging UDFs (connection alive, no response) require worker-side per-batch timeouts and engine-side client timeouts. @@ -100,14 +104,15 @@ A worker implementation tracks one of these states per `Execute` stream: | State | Meaning | Enters on | Exits on | |-------|---------|-----------|----------| | `AwaitingInit` | Stream opened, no `Init` received yet | stream open | `Init` received | -| `AwaitingData` | Chunked-payload mode: waiting for `PayloadChunk` messages | `Init` with `is_chunking_payload=true` | `PayloadChunk(last=true)` → send `InitResponse` | -| `Chunking` | Non-final chunks accumulating | `PayloadChunk(last=false)` | `PayloadChunk(last=true)` → send `InitResponse` | -| `Data` | Init complete; processing `DataRequest` / emitting `DataResponse` | `Init` (inline payload) or last `PayloadChunk` | `Finish` or `Cancel` | -| `Finishing` | `Finish` received; draining remaining output | `Finish` | drain complete → send `FinishResponse`; or `Cancel` wins → send `CancelResponse` | +| `AwaitingData` | Chunked-payload mode: `Init` received, waiting for first `PayloadChunk` | `Init` with `is_chunking_payload=true` | `PayloadChunk(last=true)` → send `InitResponse` → `Data`; `PayloadChunk(last=false)` → `Chunking`; `Cancel` → `Done`; `Finish` → `Finishing` | +| `Chunking` | Non-final chunks accumulating | `PayloadChunk(last=false)` | `PayloadChunk(last=true)` → send `InitResponse` → `Data`; `Cancel` → `Done`; `Finish` → `Finishing` | +| `Data` | Init complete; processing `DataRequest` / emitting `DataResponse` | `Init` (inline payload) or last `PayloadChunk` | `ExecutionError` sent → `PostError`; `Finish` → `Finishing`; `Cancel` → `Done` | +| `PostError` | Data-phase error sent; finish callback will not run | `ExecutionError` sent | `Finish` or `Cancel` received → run cancel callback (if any) → send `CancelResponse` → `Done` | +| `Finishing` | Finish callback (if any) running after all input consumed | `Finish` (from `Data`, `AwaitingData`, or `Chunking`) | callback completes → send `FinishResponse` → `Done`; or `Cancel` wins → run cancel callback (if any) → send `CancelResponse` → `Done` | | `Done` | Stream closed | `FinishResponse` or `CancelResponse` sent, or `onError` | — | Any protocol violation in an active state sends `ExecutionError(ProtocolError)` + -`FinishResponse` and transitions to `Done`. A gRPC transport error (`onError`) +`CancelResponse` and transitions to `Done`. A gRPC transport error (`onError`) transitions to `Done` without sending any response. ### Direct worker creation diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala index bef5a88c45122..a20eb0ca3996e 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala @@ -160,6 +160,8 @@ abstract class WorkerSession extends AutoCloseable { * - between [[init]] and [[process]] -- signals that the session * should be terminated; the caller should not invoke [[process]] * and should call [[close]] to release resources. + * Implementations SHOULD throw `CancellationException` (or equivalent) + * if [[process]] is subsequently invoked despite the cancellation. * - during [[process]] (data flowing or awaiting [[FinishResponse]]) * -- sends [[Cancel]] and waits for [[CancelResponse]] or * [[FinishResponse]] (whichever arrives first). diff --git a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala index 8aac8e565c1af..cb4773fe41133 100644 --- a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala +++ b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala @@ -89,8 +89,12 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { private case class AwaitingData(initPayload: ByteString) extends WorkerState private case class Chunking(accumulated: ByteString) extends WorkerState private case object Data extends WorkerState - // Finish received; FinishResponse not yet sent. Cancel may still win if it - // arrives before the drain completes and FinishResponse is written. + // ExecutionError sent during data processing. The finish callback will not + // run; the terminator is always CancelResponse. + private case object PostError extends WorkerState + // Finish received; finish callback running after all input consumed. + // FinishResponse not yet sent. Cancel may still win if it arrives before + // the callback completes and FinishResponse is written. private case object Finishing extends WorkerState private case object Done extends WorkerState @@ -172,21 +176,19 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { private def handleChunk(chunk: org.apache.spark.udf.worker.PayloadChunk): Unit = state match { case AwaitingData(existing) => - val updated = existing.concat(chunk.getData) if (chunk.hasLast && chunk.getLast) { sendInitResponse() state = Data } else { - state = Chunking(updated) + state = Chunking(existing.concat(chunk.getData)) } case Chunking(existing) => - val updated = existing.concat(chunk.getData) if (chunk.hasLast && chunk.getLast) { sendInitResponse() state = Data } else { - state = Chunking(updated) + state = Chunking(existing.concat(chunk.getData)) } case _ => closeWithProtocolError(s"PayloadChunk received in state $state") @@ -200,8 +202,9 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { private def processEcho(data: DataRequest): Unit = { if (data.getData == ERROR_TRIGGER) { - // Signal a user-code error. Stay in Data so the engine's Finish or - // Cancel is still received and drives the stream terminator. + // Data-phase error: send ExecutionError and enter PostError. + // The finish callback will not run; the engine must send Cancel + // (or Cancel-after-Finish) and the terminator is CancelResponse. sendControl(UdfControlResponse.newBuilder() .setError(ExecutionError.newBuilder() .setUser(UserError.newBuilder() @@ -210,6 +213,7 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { .build()) .build()) .build()) + state = PostError } else { responseLock.synchronized { responseObserver.onNext(UdfResponse.newBuilder() @@ -220,7 +224,7 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { } private def handleFinish(): Unit = state match { - case AwaitingData(_) => + case AwaitingData(_) | Chunking(_) => // Chunking path: engine sends Finish before all chunks arrived. // Send InitResponse now (treating the partial payload as complete) // then finish normally. @@ -230,6 +234,11 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { case Data => drainAndFinish() + case PostError => + // ExecutionError was already sent; finish callback does not run. + // Respond with CancelResponse to complete the stream. + sendCancelResponse() + case _ => closeWithProtocolError(s"Finish received in state $state") } @@ -243,18 +252,15 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { .putMetrics("status", "ok") .build()) .build()) - responseLock.synchronized { responseObserver.onCompleted() } state = Done + responseLock.synchronized { responseObserver.onCompleted() } } private def handleCancel(cancel: Cancel): Unit = state match { - case Data | AwaitingData(_) | Finishing => - // Finishing: Cancel arrived while drain was in progress -- abort. - sendControl(UdfControlResponse.newBuilder() - .setCancel(CancelResponse.getDefaultInstance) - .build()) - responseLock.synchronized { responseObserver.onCompleted() } - state = Done + case Data | AwaitingData(_) | Chunking(_) | PostError | Finishing => + // Finishing: Cancel arrived while finish callback was in progress -- abort. + // PostError: engine sent Cancel after receiving ExecutionError (expected). + sendCancelResponse() case Done => // FinishResponse already sent; Cancel arrived too late and is ignored. @@ -287,6 +293,14 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { .setInit(InitResponse.getDefaultInstance) .build()) + private def sendCancelResponse(): Unit = { + sendControl(UdfControlResponse.newBuilder() + .setCancel(CancelResponse.getDefaultInstance) + .build()) + state = Done + responseLock.synchronized { responseObserver.onCompleted() } + } + private def sendControl(ctrl: UdfControlResponse): Unit = responseLock.synchronized { responseObserver.onNext( @@ -294,16 +308,12 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { } private def closeWithProtocolError(msg: String): Unit = { - state = Done sendControl(UdfControlResponse.newBuilder() .setError(ExecutionError.newBuilder() .setProtocol(ProtocolError.newBuilder().setMessage(msg).build()) .build()) .build()) - sendControl(UdfControlResponse.newBuilder() - .setFinish(FinishResponse.getDefaultInstance) - .build()) - responseLock.synchronized { responseObserver.onCompleted() } + sendCancelResponse() } } @@ -497,18 +507,35 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { val receivedCount = new java.util.concurrent.atomic.AtomicInteger(0) val doneLatch = new CountDownLatch(1) @volatile var streamErr: Option[Throwable] = None + val requestCompleted = new AtomicBoolean(false) + // reqObs is assigned after responseObs is created; @volatile ensures + // the response observer (which runs on a gRPC callback thread) sees the + // assignment made by the test thread. + @volatile var reqObs: StreamObserver[UdfRequest] = _ val responseObs = new StreamObserver[UdfResponse] { + private def completeRequestStream(): Unit = + if (requestCompleted.compareAndSet(false, true)) reqObs.onCompleted() + override def onNext(r: UdfResponse): Unit = r.getResponse match { case UdfResponse.Response.Data(_) => receivedCount.incrementAndGet() case UdfResponse.Response.Control(c) => - if (c.getControl.isInstanceOf[UdfControlResponse.Control.Finish]) doneLatch.countDown() + c.getControl match { + case UdfControlResponse.Control.Finish(_) => + completeRequestStream() + doneLatch.countDown() + case _ => + } case _ => } - override def onError(t: Throwable): Unit = { streamErr = Some(t); doneLatch.countDown() } + override def onError(t: Throwable): Unit = { + streamErr = Some(t) + completeRequestStream() + doneLatch.countDown() + } override def onCompleted(): Unit = doneLatch.countDown() } - val reqObs = asyncStub.execute(responseObs) + reqObs = asyncStub.execute(responseObs) val producer = new Thread(() => { reqObs.onNext(UdfRequest.newBuilder() @@ -532,7 +559,8 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { .setControl(UdfControlRequest.newBuilder() .setFinish(Finish.getDefaultInstance).build()) .build()) - reqObs.onCompleted() + // Request stream stays open; completeRequestStream() is called by + // the response observer on FinishResponse or gRPC error. }, "producer") producer.start() diff --git a/udf/worker/proto/src/main/protobuf/udf_protocol.proto b/udf/worker/proto/src/main/protobuf/udf_protocol.proto index f2277f469c643..1377a5bb68cdf 100644 --- a/udf/worker/proto/src/main/protobuf/udf_protocol.proto +++ b/udf/worker/proto/src/main/protobuf/udf_protocol.proto @@ -50,10 +50,12 @@ service UdfWorker { // appropriate mechanism for detecting and surfacing hung user code; // the engine should also apply client-side timeouts as a safety net. // - // Response observer threading: gRPC does not permit concurrent calls - // to the response StreamObserver. Worker implementations that dispatch - // processing to a thread pool MUST serialize all writes to the response - // observer. + // Stream lifecycle: the engine MUST half-close the request stream + // (call onCompleted() on the gRPC request side) only after the session + // outcome is known: on receiving [[FinishResponse]] or [[CancelResponse]] + // (clean termination), or on receiving a gRPC error. Deferring the + // half-close keeps the stream open long enough for [[Cancel]] to follow + // [[Finish]] when needed (see [[Finish]] for the full contract). // // For stateful execution, the state is maintained per bi-directional // stream, mapping to a `WorkerSession` on the engine side. @@ -123,16 +125,26 @@ message UdfResponse { // MAY be sent immediately after [[Init]] without waiting for chunks. // - InitResponse MUST be emitted before any DataResponse. // - ExecutionError (if any) MUST be emitted after all DataResponse -// messages. After sending it the worker MUST stop processing DataRequests -// and wait for the engine to send Finish or Cancel, then respond with -// FinishResponse or CancelResponse accordingly. At most one -// ExecutionError is sent per stream; the worker aggregates multiple -// errors internally. +// messages and only from the init or data-processing phase (not from +// the finish or cancel callback). It signals that the execution is +// aborted; the terminator is always CancelResponse. When sent before +// [[InitResponse]] it indicates an init failure (no [[InitResponse]] +// will follow). +// The engine MUST send Cancel upon receiving ExecutionError (using +// Cancel-after-Finish if Finish was already sent), and the worker +// MUST respond with CancelResponse. At most one ExecutionError is +// sent per stream; the worker aggregates multiple errors internally. +// - Errors in the finish callback (if any) are reported via +// [[FinishResponse.error]]; errors in the cancel callback (if any) +// via [[CancelResponse.error]]. These are distinct from +// ExecutionError, which covers data-processing errors only. // - The engine terminates with one of: // (a) Finish alone -> worker sends FinishResponse. // (b) Cancel alone -> worker sends CancelResponse. // (c) Finish then Cancel -> worker sends CancelResponse if it has not // yet sent FinishResponse, otherwise FinishResponse (see [[Finish]]). +// - Cancel MUST NOT precede Finish on the same stream; if the engine +// cancels before all data is submitted, it sends Cancel alone (case b). // // A worker that receives messages out of order (e.g. a second Init, // a PayloadChunk after the first DataRequest, a DataRequest before Init, @@ -304,15 +316,24 @@ message Init { optional bytes parameters = 100; } -// Acknowledgment for [[Init]]. The worker MUST send exactly one -// [[InitResponse]] before any [[DataResponse]]. When [[PayloadChunk]] +// Acknowledgment for [[Init]] on success. The worker MUST send exactly +// one [[InitResponse]] before any [[DataResponse]]. When [[PayloadChunk]] // is used to deliver the UDF payload, the worker MUST also wait until // end-of-chunking to emit it (see [[PayloadChunk]]). // -// The init phase allows the engine to interact with the UDF before -// data starts flowing -- the worker can return inline bytes here for -// the engine (or higher-level code on the engine side) to consume -// during setup. The semantics of those bytes are agreed between the +// On init failure (e.g. unsupported payload format, failed deserialization), +// the worker sends [[ExecutionError]] instead -- no [[InitResponse]] is +// emitted. The engine treats a received [[ExecutionError]] without a +// prior [[InitResponse]] as an init failure, sends [[Cancel]], and waits +// for [[CancelResponse]]. +// +// The init phase is a bidirectional handshake: the worker can return +// inline bytes for the engine to consume before data starts flowing. +// This enables certain UDF execution types to communicate init-time +// results back to the engine early -- for example, signalling that +// execution should be skipped entirely (e.g. the UDF determined during +// init that its output is empty), or returning an output schema derived +// from the payload. The semantics of those bytes are agreed between the // client side of the protocol and the worker; this message itself is // otherwise opaque. message InitResponse { @@ -327,8 +348,9 @@ message InitResponse { // inline on [[UdfPayload.payload]]; chunking is only needed when a // payload exceeds the gRPC message size limit. // -// When used, chunks are sent zero or more times after [[Init]] and -// before the first [[DataRequest]]. The worker concatenates the +// When used, at least one chunk MUST be sent after [[Init]] and +// before the first [[DataRequest]], with the final chunk carrying +// [[PayloadChunk.last]] = true. The worker concatenates the // inline [[UdfPayload.payload]] (if any) followed by all chunks in // arrival order to form the final payload. // @@ -371,8 +393,7 @@ message PayloadChunk { // of the stream. // // Backpressure: this protocol currently relies on gRPC's transport-level -// (HTTP/2) flow control for backpressure. Application-level backpressure -// is not yet defined and may be introduced in a future revision. +// (HTTP/2) flow control for backpressure. // ===================================================================== // Engine -> Worker per-batch payload. @@ -423,9 +444,10 @@ message Finish {} // Worker -> Engine completion message. Carries per-execution summary metrics. // // Metrics design: -// - [[FinishResponse.metrics]]: per-execution metrics accumulated over the -// full session (e.g. rows in/out, time per phase). Emitted once at the -// end of an [[Execute]] stream. +// - [[FinishResponse.metrics]] / [[CancelResponse.metrics]]: per-execution +// metrics accumulated up to the point of stream termination (e.g. rows +// processed, time per phase). Emitted once per [[Execute]] stream, +// regardless of whether the stream ended cleanly or was cancelled. // - [[HeartbeatResponse.metrics]]: worker-global metrics aggregated across // all sessions (e.g. total rows processed, memory usage). Emitted // periodically via the [[Manage]] RPC. @@ -445,6 +467,13 @@ message FinishResponse { // of those bytes are agreed between the client side of the // protocol and the worker. optional bytes data = 2; + + // (Optional) Error raised by the finish callback (if any), invoked + // after all input data has been consumed. This field is only set on + // the non-error execution path: if [[ExecutionError]] was sent during + // data processing, the terminator is [[CancelResponse]], not + // [[FinishResponse]]. The engine SHOULD surface this as an exception. + optional ExecutionError error = 3; } // Engine -> Worker explicit cancel. Distinct from a gRPC stream error @@ -460,6 +489,20 @@ message FinishResponse { // A broken gRPC connection is the involuntary fallback -- in that // case gRPC surfaces an error on the stream (see [[Execute]]). // +// Cancellation latency: [[Cancel]] is delivered in-order on the same +// stream as [[DataRequest]] messages, so it takes effect only after all +// preceding batches in the gRPC receive queue have been processed. The +// delay is therefore (queue depth) × (batch processing time), where +// the queue depth is bounded by HTTP/2 flow control. Workers SHOULD +// set a cancellation flag as soon as [[Cancel]] is received so that +// any in-flight processing thread can abort at the next safe checkpoint. +// +// Future: for use cases that require lower-latency cancellation (e.g. +// interrupting a long-running batch), an out-of-band cancel signal may +// be added to the [[Manage]] RPC. Because [[Manage]] uses a separate +// connection it bypasses the Execute stream queue and can signal a +// processing thread immediately, independent of buffered batches. +// // Worker behavior on involuntary stream error: when the worker observes // a gRPC error on the Execute stream (i.e. the engine-side connection // dropped), it MUST treat this as equivalent to [[Cancel]] for cleanup @@ -471,32 +514,43 @@ message Cancel { optional string reason = 1; } -// Worker -> Engine acknowledgment of [[Cancel]]. -message CancelResponse {} +// Worker -> Engine acknowledgment of [[Cancel]], or the terminator after +// [[ExecutionError]] (the execution is always considered aborted when an +// error occurred during data processing). Carries any per-execution metrics +// accumulated up to the point of cancellation -- even a partial execution +// may produce useful diagnostics (e.g. rows processed before abort). +message CancelResponse { + // Per-execution metrics accumulated up to cancellation. Free-form; + // names are worker-defined. See [[FinishResponse.metrics]] for the + // full metrics design. + map metrics = 1; -// Worker -> Engine. Signals a non-gRPC application-level error. -// Distinct from a gRPC stream error: gRPC errors indicate transport or -// protocol failures; [[ExecutionError]] carries errors that should be -// raised as user-facing exceptions. + // (Optional) Error raised by the cancel callback (if any), invoked + // when [[Cancel]] is received. The engine SHOULD surface this + // alongside any prior [[ExecutionError]] (e.g. as a suppressed + // exception). + optional ExecutionError error = 2; +} + +// Worker -> Engine. Signals an application-level error that occurred +// during data processing. Distinct from a gRPC stream error: gRPC errors +// indicate transport or connection failures; [[ExecutionError]] carries +// errors that should be raised as user-facing exceptions. // -// Wire position: emitted at most once, after all [[DataResponse]] -// messages. After sending [[ExecutionError]] the worker MUST stop -// processing [[DataRequest]] messages and wait for the engine to send -// [[Finish]] or [[Cancel]], then respond with [[FinishResponse]] or -// [[CancelResponse]] accordingly. +// Wire position: emitted at most once, after all [[DataResponse]] messages, +// and only from the data-processing phase. Errors in the finish or cancel +// callback (if any) are reported via [[FinishResponse.error]] or +// [[CancelResponse.error]] respectively, not as a separate ExecutionError. +// +// Terminator: after [[ExecutionError]] the terminator is always +// [[CancelResponse]] -- the execution is considered aborted regardless of +// whether the engine already sent [[Finish]]. The engine MUST send +// [[Cancel]] (using Cancel-after-Finish if [[Finish]] was already sent) +// and wait for [[CancelResponse]], then raise the error. // -// Engine behavior on receipt: the engine records the error and then: -// - If [[Finish]] has already been sent: waits for [[FinishResponse]] -// and raises the error afterwards. -// - If [[Finish]] has not yet been sent: sends [[Cancel]], waits for -// [[CancelResponse]], and raises the error afterwards. // If more than one [[ExecutionError]] arrives (protocol violation), // the engine SHOULD surface the first as the primary exception and // attach subsequent ones as suppressed exceptions. -// -// Cancel interaction: if [[Cancel]] arrives while the worker is -// preparing to send [[ExecutionError]], the cancel-vs-finish race rules -// in [[Finish]] apply. The worker emits exactly one terminator. message ExecutionError { // Exactly one kind MUST be set. oneof kind { From 266f102df22d4710d87df2450140623839996ed9 Mon Sep 17 00:00:00 2001 From: Haiyang Sun Date: Thu, 14 May 2026 11:20:15 +0000 Subject: [PATCH 13/13] update docs. --- udf/worker/README.md | 50 +----------- .../udf/worker/core/WorkerDispatcher.scala | 15 ++-- .../spark/udf/worker/core/WorkerSession.scala | 77 ++++++------------- .../udf/worker/core/EchoProtocolSuite.scala | 25 ++++-- 4 files changed, 50 insertions(+), 117 deletions(-) diff --git a/udf/worker/README.md b/udf/worker/README.md index c28eeef3e6009..861f69b898aa0 100644 --- a/udf/worker/README.md +++ b/udf/worker/README.md @@ -19,7 +19,7 @@ WorkerDispatcher -- manages workers, creates sessions | v WorkerSession -- one UDF execution - | 1. session.init(Init proto: protocol version + udf payload + data format + schemas) + | 1. session.init(Init proto) | 2. val results = session.process(inputBatches) | 3. session.close() ``` @@ -60,60 +60,14 @@ obtaining workers from a provisioning service or daemon. Each UDF execution uses a single bidirectional `Execute` gRPC stream. -### Message flow - ``` Engine -> Worker: Init -> PayloadChunk* -> (DataRequest)* -> Finish (Cancel)? | Cancel Worker -> Engine: InitResponse -> (DataResponse)* -> (ExecutionError)? -> (FinishResponse | CancelResponse) ``` -- `Init.is_chunking_payload = true` signals that `PayloadChunk` messages will - follow. `PayloadChunk.last = true` is the canonical end-of-chunking signal. - When `is_chunking_payload` is absent or false, `InitResponse` is sent - immediately after `Init`. -- `DataRequest` and `DataResponse` are **independent streams**: the worker may - emit `DataResponse` at any point after `InitResponse`, including before any - `DataRequest` arrives. Generator-style UDFs may have zero `DataRequest` - messages. -- The engine MAY pipeline `DataRequests` before `InitResponse` arrives. The - worker MUST buffer them and process in order once init succeeds. -- **Cancel-after-Finish**: `Cancel` MAY follow `Finish`. The worker sends - `CancelResponse` if `Cancel` arrives before `FinishResponse` is sent; - otherwise `FinishResponse`. The engine must accept either. -- **`ExecutionError`** carries application-level errors (`UserError`, - `WorkerError`, `ProtocolError`) that occur during data processing - (before the finish or cancel callback is invoked). The terminator is - always `CancelResponse` — the engine MUST send `Cancel` (or - Cancel-after-Finish) and the worker responds with `CancelResponse`. - Errors in the finish or cancel callback are reported via - `FinishResponse.error` / `CancelResponse.error` instead. -- **gRPC errors** indicate transport/connection failures only. Hanging UDFs - (connection alive, no response) require worker-side per-batch timeouts and - engine-side client timeouts. -- **Backpressure** is handled by gRPC's HTTP/2 transport-level flow control. - Application-level backpressure is not yet defined. - See `udf/worker/proto/src/main/protobuf/udf_protocol.proto` for the complete -ordering invariants and error contract. - -### Worker state machine (Execute stream) - -A worker implementation tracks one of these states per `Execute` stream: - -| State | Meaning | Enters on | Exits on | -|-------|---------|-----------|----------| -| `AwaitingInit` | Stream opened, no `Init` received yet | stream open | `Init` received | -| `AwaitingData` | Chunked-payload mode: `Init` received, waiting for first `PayloadChunk` | `Init` with `is_chunking_payload=true` | `PayloadChunk(last=true)` → send `InitResponse` → `Data`; `PayloadChunk(last=false)` → `Chunking`; `Cancel` → `Done`; `Finish` → `Finishing` | -| `Chunking` | Non-final chunks accumulating | `PayloadChunk(last=false)` | `PayloadChunk(last=true)` → send `InitResponse` → `Data`; `Cancel` → `Done`; `Finish` → `Finishing` | -| `Data` | Init complete; processing `DataRequest` / emitting `DataResponse` | `Init` (inline payload) or last `PayloadChunk` | `ExecutionError` sent → `PostError`; `Finish` → `Finishing`; `Cancel` → `Done` | -| `PostError` | Data-phase error sent; finish callback will not run | `ExecutionError` sent | `Finish` or `Cancel` received → run cancel callback (if any) → send `CancelResponse` → `Done` | -| `Finishing` | Finish callback (if any) running after all input consumed | `Finish` (from `Data`, `AwaitingData`, or `Chunking`) | callback completes → send `FinishResponse` → `Done`; or `Cancel` wins → run cancel callback (if any) → send `CancelResponse` → `Done` | -| `Done` | Stream closed | `FinishResponse` or `CancelResponse` sent, or `onError` | — | - -Any protocol violation in an active state sends `ExecutionError(ProtocolError)` + -`CancelResponse` and transitions to `Done`. A gRPC transport error (`onError`) -transitions to `Done` without sending any response. +protocol definition, ordering invariants, and error contract. ### Direct worker creation diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerDispatcher.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerDispatcher.scala index 8218a2f8b30eb..e938c3e04be5b 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerDispatcher.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerDispatcher.scala @@ -28,14 +28,13 @@ import org.apache.spark.udf.worker.UDFWorkerSpecification * handling pooling, reuse, and lifecycle behind the scenes. Spark interacts with * workers exclusively through the [[WorkerSession]]s returned by [[createSession]]. * - * '''Worker invalidation:''' if a session's Execute stream terminates with a gRPC - * transport error the worker that backed it MUST NOT be returned to any reuse pool. - * A transport error leaves the worker in an unknown state; only workers that - * complete sessions cleanly (via [[org.apache.spark.udf.worker.FinishResponse]] or - * [[org.apache.spark.udf.worker.CancelResponse]]) are eligible for reuse. - * Implementations are responsible for tracking this condition -- typically - * [[WorkerSession.doProcess]] flags the worker as invalid before [[WorkerSession.doClose]] - * releases it, so the dispatcher can distinguish a clean release from a failed one. + * '''Worker invalidation:''' if a session terminates with a transport error the + * worker that backed it MUST NOT be returned to any reuse pool. A transport + * error leaves the worker in an unknown state; only workers that complete + * sessions cleanly are eligible for reuse. Implementations are responsible for + * tracking this condition -- typically [[WorkerSession.doProcess]] flags the + * worker as invalid before [[WorkerSession.doClose]] releases it, so the + * dispatcher can distinguish a clean release from a failed one. */ @Experimental trait WorkerDispatcher extends AutoCloseable { diff --git a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala index a20eb0ca3996e..fa063a7963996 100644 --- a/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala +++ b/udf/worker/core/src/main/scala/org/apache/spark/udf/worker/core/WorkerSession.scala @@ -54,19 +54,8 @@ import org.apache.spark.udf.worker.Init * - [[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, 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]] may be called even after all input data has been - * submitted (i.e. after `Finish` has been sent on the transport). - * In that case implementations MUST send `Cancel` on the transport - * if `FinishResponse` has not yet been received, and MUST be - * prepared to receive either `FinishResponse` or `CancelResponse`. - * See the `Finish` proto message in `udf_protocol.proto` for the - * full contract. + * - [[cancel]] may be called at any time from any execution context. + * See [[cancel]] for the full contract. * * The lifecycle is enforced here: [[init]] and [[process]] are `final` * and delegate to [[doInit]] / [[doProcess]] after AtomicBoolean guards. @@ -85,10 +74,9 @@ abstract class WorkerSession extends AutoCloseable { * * Throws `IllegalStateException` if called more than once. * - * @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. + * @param message the [[Init]] message carrying the UDF body, data + * format, optional schemas, and any session context + * the worker needs to start processing. */ final def init(message: Init): Unit = { if (!initialized.compareAndSet(false, true)) { @@ -102,7 +90,7 @@ abstract class WorkerSession extends AutoCloseable { * * Follows Spark's Iterator-to-Iterator pattern: input batches are streamed * to the worker, and result batches are lazily pulled from the returned - * iterator. The session sends a Finish signal to the worker when the input + * iterator. The session sends a finish signal to the worker when the input * iterator is exhausted. * * Must be called after [[init]] and at most once per session. @@ -123,35 +111,19 @@ abstract class WorkerSession extends AutoCloseable { /** * Subclass hook for [[init]]. Called once, after the guard. - * Implementations MUST NOT open the Execute gRPC stream before - * this call: [[cancel]] before [[init]] is contractually a no-op - * at the transport level, which only holds if no stream has been - * opened yet. + * The session MUST NOT be activated before this call, since + * [[cancel]] before [[init]] is contractually a no-op. */ protected def doInit(message: Init): Unit - /** - * Subclass hook for [[process]]. Called at most once, after the guard. - * - * If the Execute stream terminates with a gRPC transport error (i.e. - * the connection broke rather than the worker sending a protocol - * response), the implementation MUST: - * - throw an appropriate exception so the caller observes a failure - * rather than a silent empty result; and - * - ensure the underlying worker is not returned to any reuse pool, - * since a transport error leaves the worker in an unknown state. - * Implementations signal this to the [[WorkerDispatcher]] via - * whatever mechanism the dispatcher provides (e.g. flagging the - * worker as invalid before calling [[doClose]]). - */ + /** Subclass hook for [[process]]. Called at most once, after the guard. */ protected def doProcess(input: Iterator[Array[Byte]]): Iterator[Array[Byte]] /** * Requests cancellation of the current UDF execution. * - * '''Thread-safety:''' implementations must allow [[cancel]] to be called - * from a thread different from the one driving [[process]] (typically a - * task interruption thread). + * '''Thread-safety:''' [[cancel]] may be called concurrently with + * [[process]] from any execution context. * * '''Lifecycle:''' [[cancel]] is idempotent and safe at any point in * the session's life: @@ -160,17 +132,16 @@ abstract class WorkerSession extends AutoCloseable { * - between [[init]] and [[process]] -- signals that the session * should be terminated; the caller should not invoke [[process]] * and should call [[close]] to release resources. - * Implementations SHOULD throw `CancellationException` (or equivalent) - * if [[process]] is subsequently invoked despite the cancellation. - * - during [[process]] (data flowing or awaiting [[FinishResponse]]) - * -- sends [[Cancel]] and waits for [[CancelResponse]] or - * [[FinishResponse]] (whichever arrives first). - * - after [[FinishResponse]] or [[CancelResponse]] has been received - * -- a no-op; the stream is already terminated. + * Implementations SHOULD surface this as an error if [[process]] + * is subsequently invoked despite the cancellation. + * - during [[process]] (data flowing or awaiting completion) + * -- requests the worker to abort on a best-effort basis. + * - after [[process]] has returned (session already terminated) + * -- a no-op. * * Implementations are responsible for the lifecycle-aware behavior - * described above so that callers (e.g. task interruption listeners) - * do not need to coordinate with the thread driving [[process]]. + * described above so that the caller does not need to coordinate + * with the execution context driving [[process]]. */ def cancel(): Unit @@ -180,11 +151,11 @@ abstract class WorkerSession extends AutoCloseable { * [[process]], or [[cancel]] have been invoked. * * If [[init]] was called but [[process]] was not (e.g. an exception - * was thrown between the two), [[close]] sends `Cancel` on the - * Execute stream before releasing resources, so the worker can clean - * up deterministically rather than observing a gRPC transport error. - * Subclasses implement [[doClose]] for resource teardown; the base - * class handles the cancel-before-close guarantee automatically. + * was thrown between the two), [[close]] signals cancellation to the + * worker before releasing resources so it can clean up + * deterministically. Subclasses implement [[doClose]] for resource + * teardown; the base class handles the cancel-before-close guarantee + * automatically. */ final override def close(): Unit = { if (initialized.get() && !processed.get()) { diff --git a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala index cb4773fe41133..1684a5e789b25 100644 --- a/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala +++ b/udf/worker/core/src/test/scala/org/apache/spark/udf/worker/core/EchoProtocolSuite.scala @@ -81,20 +81,29 @@ class EchoProtocolSuite extends AnyFunSuite with BeforeAndAfterEach { // WORKER SIDE (gRPC server) // =========================================================================== - /** Worker state machine for one Execute stream. */ + /** + * Worker state machine for one Execute stream. + * + * State Meaning Enters on Exits on + * AwaitingInit Stream open, no Init received yet stream open Init received + * AwaitingData Chunking: Init received, awaiting first chunk Init(is_chunking_payload=true) PayloadChunk(last=true) → Data; PayloadChunk(last=false) → Chunking; Finish → Finishing; Cancel → Done + * Chunking Non-final chunks accumulating PayloadChunk(last=false) PayloadChunk(last=true) → Data; Finish → Finishing; Cancel → Done + * Data Processing DataRequests / emitting DataResponse Init (inline) or last chunk ExecutionError sent → PostError; Finish → Finishing; Cancel → Done + * PostError Data-phase error sent; finish callback skipped ExecutionError sent Finish or Cancel → CancelResponse → Done + * Finishing Finish callback (if any) running Finish (from Data/AwaitingData/ callback completes → FinishResponse → Done; + * Chunking) or Cancel wins → CancelResponse → Done + * Done Stream closed FinishResponse or CancelResponse — + * sent, or onError + * + * Any protocol violation sends ExecutionError(ProtocolError) + CancelResponse → Done. + * A gRPC transport error (onError) transitions to Done without sending any response. + */ private sealed trait WorkerState private case object AwaitingInit extends WorkerState - // Chunking mode only: Init received with is_chunking_payload=true. - // Accumulates PayloadChunk messages until last=true, then sends InitResponse. private case class AwaitingData(initPayload: ByteString) extends WorkerState private case class Chunking(accumulated: ByteString) extends WorkerState private case object Data extends WorkerState - // ExecutionError sent during data processing. The finish callback will not - // run; the terminator is always CancelResponse. private case object PostError extends WorkerState - // Finish received; finish callback running after all input consumed. - // FinishResponse not yet sent. Cancel may still win if it arrives before - // the callback completes and FinishResponse is written. private case object Finishing extends WorkerState private case object Done extends WorkerState