|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.connect.pipelines |
| 19 | + |
| 20 | +import java.util.concurrent.ThreadPoolExecutor |
| 21 | +import java.util.concurrent.atomic.AtomicBoolean |
| 22 | + |
| 23 | +import scala.util.control.NonFatal |
| 24 | + |
| 25 | +import com.google.protobuf.{Timestamp => ProtoTimestamp} |
| 26 | +import io.grpc.stub.StreamObserver |
| 27 | + |
| 28 | +import org.apache.spark.connect.proto |
| 29 | +import org.apache.spark.connect.proto.ExecutePlanResponse |
| 30 | +import org.apache.spark.internal.{Logging, LogKeys} |
| 31 | +import org.apache.spark.sql.connect.service.SessionHolder |
| 32 | +import org.apache.spark.sql.internal.SQLConf |
| 33 | +import org.apache.spark.sql.pipelines.common.FlowStatus |
| 34 | +import org.apache.spark.sql.pipelines.logging.{FlowProgress, PipelineEvent, RunProgress} |
| 35 | +import org.apache.spark.util.ThreadUtils |
| 36 | + |
| 37 | +/** |
| 38 | + * Handles sending pipeline events to the client in a background thread. This prevents pipeline |
| 39 | + * execution from blocking on streaming events. |
| 40 | + */ |
| 41 | +class PipelineEventSender( |
| 42 | + responseObserver: StreamObserver[ExecutePlanResponse], |
| 43 | + sessionHolder: SessionHolder) |
| 44 | + extends Logging |
| 45 | + with AutoCloseable { |
| 46 | + |
| 47 | + private final val queueCapacity: Int = |
| 48 | + sessionHolder.session.conf |
| 49 | + .get(SQLConf.PIPELINES_EVENT_QUEUE_CAPACITY.key) |
| 50 | + .toInt |
| 51 | + |
| 52 | + // ExecutorService for background event processing |
| 53 | + private val executor: ThreadPoolExecutor = |
| 54 | + ThreadUtils.newDaemonSingleThreadExecutor(threadName = |
| 55 | + s"PipelineEventSender-${sessionHolder.sessionId}") |
| 56 | + |
| 57 | + /* |
| 58 | + * Atomic flags to track the state of the sender |
| 59 | + * - `isShutdown`: Indicates if the sender has been shut down, if true, no new events |
| 60 | + * can be accepted, and the executor will be shut down after processing all submitted events. |
| 61 | + */ |
| 62 | + private val isShutdown = new AtomicBoolean(false) |
| 63 | + |
| 64 | + /** |
| 65 | + * Send an event async by submitting it to the executor, if the sender is not shut down. |
| 66 | + * Otherwise, throws an IllegalStateException, to raise awareness of the shutdown state. |
| 67 | + * |
| 68 | + * For RunProgress events, we ensure they are always queued even if the queue is full. For other |
| 69 | + * events, we may drop them if the queue is at capacity to prevent blocking. |
| 70 | + */ |
| 71 | + def sendEvent(event: PipelineEvent): Unit = synchronized { |
| 72 | + if (!isShutdown.get()) { |
| 73 | + if (shouldEnqueueEvent(event)) { |
| 74 | + executor.submit(new Runnable { |
| 75 | + override def run(): Unit = { |
| 76 | + try { |
| 77 | + sendEventToClient(event) |
| 78 | + } catch { |
| 79 | + case NonFatal(e) => |
| 80 | + logError( |
| 81 | + log"Failed to send pipeline event to client: " + |
| 82 | + log"${MDC(LogKeys.ERROR, event.message)}", |
| 83 | + e) |
| 84 | + } |
| 85 | + } |
| 86 | + }) |
| 87 | + } |
| 88 | + } else { |
| 89 | + throw new IllegalStateException( |
| 90 | + s"Cannot send event after shutdown for session ${sessionHolder.sessionId}") |
| 91 | + } |
| 92 | + } |
| 93 | + |
| 94 | + private def shouldEnqueueEvent(event: PipelineEvent): Boolean = { |
| 95 | + event.details match { |
| 96 | + case _: RunProgress => |
| 97 | + // For RunProgress events, always enqueue event |
| 98 | + true |
| 99 | + case flowProgress: FlowProgress if FlowStatus.isTerminal(flowProgress.status) => |
| 100 | + // For FlowProgress events that are terminal, always enqueue event |
| 101 | + true |
| 102 | + case _ => |
| 103 | + // For other events, check if we have capacity |
| 104 | + executor.getQueue.size() < queueCapacity |
| 105 | + } |
| 106 | + } |
| 107 | + |
| 108 | + // Implementing AutoCloseable to allow for try-with-resources usage |
| 109 | + // This will ensure that the sender is properly shut down and all resources are released |
| 110 | + // without requiring explicit shutdown calls in user code. |
| 111 | + override def close(): Unit = shutdown() |
| 112 | + |
| 113 | + /** |
| 114 | + * Shutdown the event sender, stop taking new events and wait for processing to complete. This |
| 115 | + * method blocks until all queued events have been processed. Idempotent operation: calling this |
| 116 | + * multiple times has no effect after the first call. |
| 117 | + */ |
| 118 | + def shutdown(): Unit = { |
| 119 | + if (isShutdown.compareAndSet(false, true)) { |
| 120 | + // Request a shutdown of the executor which waits for all tasks to complete |
| 121 | + executor.shutdown() |
| 122 | + // Blocks until all tasks have completed execution after a shutdown request, |
| 123 | + // disregard the timeout since we want all events to be processed |
| 124 | + if (!executor.awaitTermination(Long.MaxValue, java.util.concurrent.TimeUnit.MILLISECONDS)) { |
| 125 | + logError( |
| 126 | + log"Pipeline event sender for session " + |
| 127 | + log"${MDC(LogKeys.SESSION_ID, sessionHolder.sessionId)} failed to terminate") |
| 128 | + executor.shutdownNow() |
| 129 | + } |
| 130 | + logInfo( |
| 131 | + log"Pipeline event sender shutdown completed for session " + |
| 132 | + log"${MDC(LogKeys.SESSION_ID, sessionHolder.sessionId)}") |
| 133 | + } |
| 134 | + } |
| 135 | + |
| 136 | + /** |
| 137 | + * Send a single event to the client |
| 138 | + */ |
| 139 | + private[connect] def sendEventToClient(event: PipelineEvent): Unit = { |
| 140 | + try { |
| 141 | + val protoEvent = constructProtoEvent(event) |
| 142 | + responseObserver.onNext( |
| 143 | + proto.ExecutePlanResponse |
| 144 | + .newBuilder() |
| 145 | + .setSessionId(sessionHolder.sessionId) |
| 146 | + .setServerSideSessionId(sessionHolder.serverSessionId) |
| 147 | + .setPipelineEventResult(proto.PipelineEventResult.newBuilder |
| 148 | + .setEvent(protoEvent) |
| 149 | + .build()) |
| 150 | + .build()) |
| 151 | + } catch { |
| 152 | + case NonFatal(e) => |
| 153 | + logError( |
| 154 | + log"Failed to send pipeline event to client: " + |
| 155 | + log"${MDC(LogKeys.ERROR, event.message)}", |
| 156 | + e) |
| 157 | + } |
| 158 | + } |
| 159 | + |
| 160 | + private def constructProtoEvent(event: PipelineEvent): proto.PipelineEvent = { |
| 161 | + val message = if (event.error.nonEmpty) { |
| 162 | + // Returns the message associated with a Throwable and all its causes |
| 163 | + def getExceptionMessages(throwable: Throwable): Seq[String] = { |
| 164 | + throwable.getMessage +: |
| 165 | + Option(throwable.getCause).map(getExceptionMessages).getOrElse(Nil) |
| 166 | + } |
| 167 | + val errorMessages = getExceptionMessages(event.error.get) |
| 168 | + s"""${event.message} |
| 169 | + |Error: ${errorMessages.mkString("\n")}""".stripMargin |
| 170 | + } else { |
| 171 | + event.message |
| 172 | + } |
| 173 | + val protoEventBuilder = proto.PipelineEvent |
| 174 | + .newBuilder() |
| 175 | + .setTimestamp( |
| 176 | + ProtoTimestamp |
| 177 | + .newBuilder() |
| 178 | + // java.sql.Timestamp normalizes its internal fields: getTime() returns |
| 179 | + // the full timestamp in milliseconds, while getNanos() returns the |
| 180 | + // fractional seconds (0-999,999,999 ns). This ensures no precision is |
| 181 | + // lost or double-counted. |
| 182 | + .setSeconds(event.timestamp.getTime / 1000) |
| 183 | + .setNanos(event.timestamp.getNanos) |
| 184 | + .build()) |
| 185 | + .setMessage(message) |
| 186 | + protoEventBuilder.build() |
| 187 | + } |
| 188 | +} |
0 commit comments