Skip to content

Commit

Permalink
[SPARK-44421][SPARK-44423][CONNECT] Reattachable execution in Spark C…
Browse files Browse the repository at this point in the history
…onnect

### What changes were proposed in this pull request?

Implement server support and scala client for reattachable execution with two new RPCs:
* ReattachExecute
* ReleaseExecute

With reattachable execution, if the response stream of ExecutePlan RPC gets broken, it can be reattached to with ReattachExecute RPC. The server now holds a bit of a backtrack buffer, in case the client coming back with reattach after error did not receive the latest sent responses, and have to backtrack. The client can sends ReleaseExecute responses to let the server know what has already been consumed and can be freed.

#### Server: ExecuteResponseObserver

If ExecuteHolder is reattachable, ExecuteResponseSender is modified to not throw away responses that were returned to ExecuteGrpcResponseSender immediately, but keep a buffer of `spark.connect.execute.reattachable.observerRetryBufferSize` (default 1 MB) responses behind the last returned response. This facilitates returning consumer being able to backtrack in case the last returned response did not successfully arrive at the client.

#### Server: ExecuteGrpcResponseSender

If ExecuteHolder is reattachable, ExecuteGrpcResponseSender is modified to gracefully end the RPC response stream after a given time (`spark.connect.execute.reattachable.senderMaxStreamDuration`, default 5 minutes) or size of responses (`spark.connect.execute.reattachable.senderMaxStreamSize`, default 1 GB). After that the response is completed, and client needs to come back with a ReattachExecute RPC to continue. Clients know whether it's the real end of the results, because ExecuteThreadRunner now puts a marker ResultComplete message at the real end of the stream.

Cutting the stream manually in a graceful way like that can help prevent errors related to long open streams. It also helps with long held idle stream that may get broken because of idleness. The client is able to handle these errors, but this also provides a cleaner way to minimize the number of network errors.

ExecuteGrpcResponseSender also implements flow control of responses sent to the GRPC stream. Normally, grpcResponseObserver.onNext would never block. This doesn't however mean that it would send all responses immediately. The responses could be queued if the client does not keep up receiving and processing them. This queue can grow ubounded, and we don't know what was actually sent, so we don't know how much of a buffer ExecuteResponseObserver really needs to keep. Flow control utilizes functions of ServerCallStreamObserver, which provides means to check if the stream is actually ready to send data, and callbacks to notify when it becomes ready. ExecuteGrpcResponseSender will wait with grpcResponseObserver.onNext until it becomes ready.

This also helps with clients that would abandon their result stream. Now we will block from sending the responses, and time out the RPC after senderMaxStreamDuration, instead of pushing the responses to a GRPC outbound queue that is outside our control. The TODO server mechanism would then remove the execution with no RPC attached. Note: in such a situation, responses will still pile up in ExecuteResponseObserver; there is unfortunately no flow control of the Spark task results being pushed to it.

#### Client: ExecutePlanResponseReattachableIterator

Client wraps the whole implementation inside ExecutePlanResponseReattachableIterator. It's a wrapper that starts with an ExecutePlan result iterator, but if this iterator either finishes without a ResultComplete message, or an intermittent network error occurs, it can acquire a new iterator to continue where it left off: by tracking the last received response, it can request a new iterator from ReattachExecute that starts after that.

The client sends ReleaseExecute RPCs asynchronously to release the responses it already consumed (potentially faster than the ExecuteResponseObserver would release from its backtrack buffer). The client doesn't need to check their result or block on them. If something fails, the executions will get cleaned by the TODO cleanup mechanism in the server.

#### TODO as followup

* What remains TODO for a followup PR is server mechanism for dealing with executions that have been abandoned - the client did not come back with ReatachExecute, and didn't free the execution with ReleaseExecute. This will be done by an operation manager that monitors all active executions, and interrupts and removes ones that were not released, but don't have any client RPC attached to them for a while.
* Python client implementation is a followup.

### Why are the changes needed?

Robust handling of long running queries in face of intermittent network glitches.

### Does this PR introduce _any_ user-facing change?

The re-attachable execution mechanism is not exposed in any public API. Several internal configs have been added:

Connect server configs:
* `spark.connect.execute.reattachable.senderMaxStreamDuration`
* `spark.connect.execute.reattachable.senderMaxStreamSize`
* `spark.connect.execute.reattachable.observerRetryBufferSize`

Connect scala client configuration element: `useReattachableExecute`

### How was this patch tested?

Existing E2E suite utilize re-attachable execution. The tests have been configure to exercise reattaching by using an extremely low senderMaxStreamDuration and senderMaxStreamSize.

Closes #42228 from juliuszsompolski/SPARK-44421.

Lead-authored-by: Juliusz Sompolski <julek@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
(cherry picked from commit 3e5203c)
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
  • Loading branch information
juliuszsompolski and HyukjinKwon committed Aug 1, 2023
1 parent ff382ac commit 562068b
Show file tree
Hide file tree
Showing 29 changed files with 1,660 additions and 213 deletions.
51 changes: 47 additions & 4 deletions common/utils/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -1240,6 +1240,34 @@
],
"sqlState" : "42000"
},
"INVALID_CURSOR" : {
"message" : [
"The cursor is invalid."
],
"subClass" : {
"DISCONNECTED" : {
"message" : [
"The cursor has been disconnected by the server."
]
},
"NOT_REATTACHABLE" : {
"message" : [
"The cursor is not reattachable."
]
},
"POSITION_NOT_AVAILABLE" : {
"message" : [
"The cursor position id <responseId> is no longer available at index <index>."
]
},
"POSITION_NOT_FOUND" : {
"message" : [
"The cursor position id <responseId> is not found."
]
}
},
"sqlState" : "HY109"
},
"INVALID_DEFAULT_VALUE" : {
"message" : [
"Failed to execute <statement> command because the destination table column <colName> has a DEFAULT value <defaultValue>,"
Expand Down Expand Up @@ -1388,14 +1416,29 @@
"The handle <handle> is invalid."
],
"subClass" : {
"ALREADY_EXISTS" : {
"FORMAT" : {
"message" : [
"Handle already exists."
"Handle must be an UUID string of the format '00112233-4455-6677-8899-aabbccddeeff'"
]
},
"FORMAT" : {
"OPERATION_ALREADY_EXISTS" : {
"message" : [
"Operation already exists."
]
},
"OPERATION_NOT_FOUND" : {
"message" : [
"Operation not found."
]
},
"SESSION_ALREADY_EXISTS" : {
"message" : [
"Session already exists."
]
},
"SESSION_NOT_FOUND" : {
"message" : [
"Handle has invalid format. Handle must an UUID string of the format '00112233-4455-6677-8899-aabbccddeeff'"
"Session not found."
]
}
},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -254,7 +254,8 @@ class SparkSession private[sql] (
val plan = proto.Plan.newBuilder().setCommand(cmd)
val responseIter = client.execute(plan.build())

val response = responseIter.asScala
// Note: .toSeq makes the stream be consumed and closed.
val response = responseIter.asScala.toSeq
.find(_.hasSqlCommandResult)
.getOrElse(throw new RuntimeException("SQLCommandResult must be present"))

Expand Down Expand Up @@ -310,7 +311,8 @@ class SparkSession private[sql] (
val plan = proto.Plan.newBuilder().setCommand(cmd)
val responseIter = client.execute(plan.build())

val response = responseIter.asScala
// Note: .toSeq makes the stream be consumed and closed.
val response = responseIter.asScala.toSeq
.find(_.hasSqlCommandResult)
.getOrElse(throw new RuntimeException("SQLCommandResult must be present"))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,15 @@ private[client] class CustomSparkConnectBlockingStub(
}
}

def executePlanReattachable(
request: ExecutePlanRequest): java.util.Iterator[ExecutePlanResponse] = {
GrpcExceptionConverter.convert {
GrpcExceptionConverter.convertIterator[ExecutePlanResponse](
// Don't use retryHandler - own retry handling is inside.
new ExecutePlanResponseReattachableIterator(request, channel, retryPolicy))
}
}

def analyzePlan(request: AnalyzePlanRequest): AnalyzePlanResponse = {
GrpcExceptionConverter.convert {
retryHandler.retry {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,244 @@
/*
* 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.sql.connect.client

import java.util.UUID

import scala.util.control.NonFatal

import io.grpc.ManagedChannel
import io.grpc.stub.StreamObserver

import org.apache.spark.connect.proto
import org.apache.spark.internal.Logging

/**
* Retryable iterator of ExecutePlanResponses to an ExecutePlan call.
*
* It can handle situations when:
* - the ExecutePlanResponse stream was broken by retryable network error (governed by
* retryPolicy)
* - the ExecutePlanResponse was gracefully ended by the server without a ResultComplete
* message; this tells the client that there is more, and it should reattach to continue.
*
* Initial iterator is the result of an ExecutePlan on the request, but it can be reattached with
* ReattachExecute request. ReattachExecute request is provided the responseId of last returned
* ExecutePlanResponse on the iterator to return a new iterator from server that continues after
* that.
*
* Since in reattachable execute the server does buffer some responses in case the client needs to
* backtrack
*/
class ExecutePlanResponseReattachableIterator(
request: proto.ExecutePlanRequest,
channel: ManagedChannel,
retryPolicy: GrpcRetryHandler.RetryPolicy)
extends java.util.Iterator[proto.ExecutePlanResponse]
with Logging {

val operationId = if (request.hasOperationId) {
request.getOperationId
} else {
// Add operation id, if not present.
// with operationId set by the client, the client can use it to try to reattach on error
// even before getting the first response. If the operation in fact didn't even reach the
// server, that will end with INVALID_HANDLE.OPERATION_NOT_FOUND error.
UUID.randomUUID.toString
}

// Need raw stubs, don't want retry handling or error conversion done by the custom stubs.
// - this does it's own custom retry handling
// - error conversion is wrapped around this in CustomSparkConnectBlockingStub,
// this needs raw GRPC errors for retries.
private val rawBlockingStub = proto.SparkConnectServiceGrpc.newBlockingStub(channel)
private val rawAsyncStub = proto.SparkConnectServiceGrpc.newStub(channel)

private val initialRequest: proto.ExecutePlanRequest = request
.toBuilder()
.addRequestOptions(
proto.ExecutePlanRequest.RequestOption
.newBuilder()
.setReattachOptions(proto.ReattachOptions.newBuilder().setReattachable(true).build())
.build())
.setOperationId(operationId)
.build()

// ResponseId of the last response returned by next()
private var lastReturnedResponseId: Option[String] = None

// True after ResponseComplete message was seen in the stream.
// Server will always send this message at the end of the stream, if the underlying iterator
// finishes without producing one, another iterator needs to be reattached.
private var responseComplete: Boolean = false

// Initial iterator comes from ExecutePlan request.
private var iterator: java.util.Iterator[proto.ExecutePlanResponse] =
rawBlockingStub.executePlan(initialRequest)

override def next(): proto.ExecutePlanResponse = synchronized {
// hasNext will trigger reattach in case the stream completed without responseComplete
if (!hasNext()) {
throw new java.util.NoSuchElementException()
}

// Get next response, possibly triggering reattach in case of stream error.
var firstTry = true
val ret = retry {
if (firstTry) {
// on first try, we use the existing iterator.
firstTry = false
} else {
// on retry, the iterator is borked, so we need a new one
iterator = rawBlockingStub.reattachExecute(createReattachExecuteRequest())
}
iterator.next()
}

// Record last returned response, to know where to restart in case of reattach.
lastReturnedResponseId = Some(ret.getResponseId)
if (ret.hasResultComplete) {
responseComplete = true
releaseExecute(None) // release all
} else {
releaseExecute(lastReturnedResponseId) // release until this response
}
ret
}

override def hasNext(): Boolean = synchronized {
if (responseComplete) {
// After response complete response
return false
}
var firstTry = true
retry {
if (firstTry) {
// on first try, we use the existing iterator.
firstTry = false
} else {
// on retry, the iterator is borked, so we need a new one
iterator = rawBlockingStub.reattachExecute(createReattachExecuteRequest())
}
var hasNext = iterator.hasNext()
// Graceful reattach:
// If iterator ended, but there was no ResultComplete, it means that there is more,
// and we need to reattach.
if (!hasNext && !responseComplete) {
do {
iterator = rawBlockingStub.reattachExecute(createReattachExecuteRequest())
assert(!responseComplete) // shouldn't change...
hasNext = iterator.hasNext()
// It's possible that the new iterator will be empty, so we need to loop to get another.
// Eventually, there will be a non empty iterator, because there's always a ResultComplete
// at the end of the stream.
} while (!hasNext)
}
hasNext
}
}

/**
* Inform the server to release the execution.
*
* This will send an asynchronous RPC which will not block this iterator, the iterator can
* continue to be consumed.
*
* Release with untilResponseId informs the server that the iterator has been consumed until and
* including response with that responseId, and these responses can be freed.
*
* Release with None means that the responses have been completely consumed and informs the
* server that the completed execution can be completely freed.
*/
private def releaseExecute(untilResponseId: Option[String]): Unit = {
val request = createReleaseExecuteRequest(untilResponseId)
rawAsyncStub.releaseExecute(request, createRetryingReleaseExecuteResponseObserer(request))
}

/**
* Create result callback to the asynchronouse ReleaseExecute. The client does not block on
* ReleaseExecute and continues with iteration, but if it fails with a retryable error, the
* callback will retrigger the asynchronous ReleaseExecute.
*/
private def createRetryingReleaseExecuteResponseObserer(
requestForRetry: proto.ReleaseExecuteRequest,
currentRetryNum: Int = 0): StreamObserver[proto.ReleaseExecuteResponse] = {
new StreamObserver[proto.ReleaseExecuteResponse] {
override def onNext(v: proto.ReleaseExecuteResponse): Unit = {}
override def onCompleted(): Unit = {}
override def onError(t: Throwable): Unit = t match {
case NonFatal(e) if retryPolicy.canRetry(e) && currentRetryNum < retryPolicy.maxRetries =>
Thread.sleep(
(retryPolicy.maxBackoff min retryPolicy.initialBackoff * Math
.pow(retryPolicy.backoffMultiplier, currentRetryNum)).toMillis)
rawAsyncStub.releaseExecute(
requestForRetry,
createRetryingReleaseExecuteResponseObserer(requestForRetry, currentRetryNum + 1))
case _ =>
logWarning(s"ReleaseExecute failed with exception: $t.")
}
}
}

private def createReattachExecuteRequest() = {
val reattach = proto.ReattachExecuteRequest
.newBuilder()
.setSessionId(initialRequest.getSessionId)
.setUserContext(initialRequest.getUserContext)
.setOperationId(initialRequest.getOperationId)

if (initialRequest.hasClientType) {
reattach.setClientType(initialRequest.getClientType)
}

if (lastReturnedResponseId.isDefined) {
reattach.setLastResponseId(lastReturnedResponseId.get)
}
reattach.build()
}

private def createReleaseExecuteRequest(untilResponseId: Option[String]) = {
val release = proto.ReleaseExecuteRequest
.newBuilder()
.setSessionId(initialRequest.getSessionId)
.setUserContext(initialRequest.getUserContext)
.setOperationId(initialRequest.getOperationId)

if (initialRequest.hasClientType) {
release.setClientType(initialRequest.getClientType)
}

untilResponseId match {
case None =>
release.setReleaseAll(proto.ReleaseExecuteRequest.ReleaseAll.newBuilder().build())
case Some(responseId) =>
release
.setReleaseUntil(
proto.ReleaseExecuteRequest.ReleaseUntil
.newBuilder()
.setResponseId(responseId)
.build())
}

release.build()
}

/**
* Retries the given function with exponential backoff according to the client's retryPolicy.
*/
private def retry[T](fn: => T, currentRetryNum: Int = 0): T =
GrpcRetryHandler.retry(retryPolicy)(fn, currentRetryNum)
}

0 comments on commit 562068b

Please sign in to comment.