Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-44421][SPARK-44423][CONNECT] Reattachable execution in Spark Connect #42228

Closed
wants to merge 18 commits into from
Closed
Show file tree
Hide file tree
Changes from 17 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
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 @@ -1299,6 +1299,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 @@ -1447,14 +1475,29 @@
"The handle <handle> is invalid."
],
"subClass" : {
"ALREADY_EXISTS" : {
juliuszsompolski marked this conversation as resolved.
Show resolved Hide resolved
"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(
juliuszsompolski marked this conversation as resolved.
Show resolved Hide resolved
request: proto.ExecutePlanRequest,
channel: ManagedChannel,
retryPolicy: GrpcRetryHandler.RetryPolicy)
extends java.util.Iterator[proto.ExecutePlanResponse]
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should it be CloseableIterator instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The regular iterator returned from regular executePlan is not a closeable iterator, and grpc-java doesn't seem to play nicely with closeable iterators in general: grpc/grpc-java#2409
I think the best I can do in Java is make sure that internal users in the client consume it... and over places that give control over it to user (like Dataset.toLocalIterator) there's no good control.
If an iterator is left open and idle:

  • the server will close the RPC stream after 5 minutes (STREAM_MAX_TIME)
  • then the server will tear down executions without attached RPC stream after another 5 minutes (TODO)

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)
}