grundprinzip commented on code in PR #42235: URL: https://github.com/apache/spark/pull/42235#discussion_r1281096015
########## python/pyspark/sql/connect/client/reattach.py: ########## @@ -0,0 +1,228 @@ +# +# 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. +# +from pyspark.sql.connect.utils import check_dependencies + +check_dependencies(__name__) + +import uuid +from collections.abc import Generator +from typing import Optional, Dict, Any, Iterator +import threading + +import pyspark.sql.connect.proto as pb2 +import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib + + +class ExecutePlanResponseReattachableIterator(Generator): + """ + 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 + """ + + def __init__( + self, + request: pb2.ExecutePlanRequest, + stub: grpc_lib.SparkConnectServiceStub, + retry_policy: Dict[str, Any], + ): + self._request = request + self._retry_policy = retry_policy + if request.operation_id: + self._operation_id = request.operation_id + 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. + self._operation_id = str(uuid.uuid4()) + + self._stub = stub + request.request_options.append( + pb2.ExecutePlanRequest.RequestOption( + reattach_options=pb2.ReattachOptions(reattachable=True) + ) + ) + self._initial_request = request + + # ResponseId of the last response returned by next() + self._last_returned_response_id: Optional[str] = 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. + self._result_complete = False + + # Initial iterator comes from ExecutePlan request. + self._iterator: Iterator[pb2.ExecutePlanResponse] = self._stub.ExecutePlan( + self._initial_request + ) + + # Current item from this iterator. + self._current: Optional[pb2.ExecutePlanResponse] = None + + def send(self, value: Any) -> pb2.ExecutePlanResponse: + # will trigger reattach in case the stream completed without result_complete + if not self._has_next(): + raise StopIteration() + + ret = self._current + assert ret is not None + + self._last_returned_response_id = ret.response_id + if ret.result_complete: + self._result_complete = True + self._release_execute(None) # release all + else: + self._release_execute(self._last_returned_response_id) + self._current = None + return ret + + def _has_next(self) -> bool: + from pyspark.sql.connect.client.core import SparkConnectClient + from pyspark.sql.connect.client.core import Retrying + + if self._result_complete: + # After response complete response + return False + else: + first_try = True + for attempt in Retrying( Review Comment: attempt is of type `AttemptManager` which contains a `_retry_state` that has a _count. We can make the retry state visible in the attempt manager if you want to avoid having a first try variable ########## python/pyspark/sql/connect/client/reattach.py: ########## @@ -0,0 +1,228 @@ +# +# 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. +# +from pyspark.sql.connect.utils import check_dependencies + +check_dependencies(__name__) + +import uuid +from collections.abc import Generator +from typing import Optional, Dict, Any, Iterator +import threading + +import pyspark.sql.connect.proto as pb2 +import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib + + +class ExecutePlanResponseReattachableIterator(Generator): + """ + 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 + """ + + def __init__( + self, + request: pb2.ExecutePlanRequest, + stub: grpc_lib.SparkConnectServiceStub, + retry_policy: Dict[str, Any], + ): + self._request = request + self._retry_policy = retry_policy + if request.operation_id: + self._operation_id = request.operation_id + 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. + self._operation_id = str(uuid.uuid4()) + + self._stub = stub + request.request_options.append( + pb2.ExecutePlanRequest.RequestOption( + reattach_options=pb2.ReattachOptions(reattachable=True) + ) + ) + self._initial_request = request + + # ResponseId of the last response returned by next() + self._last_returned_response_id: Optional[str] = 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. + self._result_complete = False + + # Initial iterator comes from ExecutePlan request. + self._iterator: Iterator[pb2.ExecutePlanResponse] = self._stub.ExecutePlan( + self._initial_request + ) + + # Current item from this iterator. + self._current: Optional[pb2.ExecutePlanResponse] = None + + def send(self, value: Any) -> pb2.ExecutePlanResponse: + # will trigger reattach in case the stream completed without result_complete + if not self._has_next(): + raise StopIteration() + + ret = self._current + assert ret is not None + + self._last_returned_response_id = ret.response_id + if ret.result_complete: + self._result_complete = True + self._release_execute(None) # release all + else: + self._release_execute(self._last_returned_response_id) + self._current = None + return ret + + def _has_next(self) -> bool: + from pyspark.sql.connect.client.core import SparkConnectClient Review Comment: why the import here? ########## python/pyspark/sql/connect/client/core.py: ########## @@ -606,26 +622,35 @@ def __init__( "initial_backoff": 50, "max_backoff": 60000, } - if retryPolicy: - self._retry_policy.update(retryPolicy) + if retry_policy: + self._retry_policy.update(retry_policy) # Generate a unique session ID for this client. This UUID must be unique to allow # concurrent Spark sessions of the same user. If the channel is closed, creating # a new client will create a new session ID. self._session_id = str(uuid.uuid4()) if self._builder.userId is not None: self._user_id = self._builder.userId - elif userId is not None: - self._user_id = userId + elif user_id is not None: + self._user_id = user_id else: self._user_id = os.getenv("USER", None) self._channel = self._builder.toChannel() self._closed = False self._stub = grpc_lib.SparkConnectServiceStub(self._channel) self._artifact_manager = ArtifactManager(self._user_id, self._session_id, self._channel) + self._use_reattachable_execute = use_reattachable_execute # Configure logging for the SparkConnect client. + def disable_reattachable_execute(self) -> "SparkConnectClient": Review Comment: I think this will probably need some flagging to avoid that this is done while a query is currently being executed so that we know what to do. Or if the information returned from the query is enough to independently understand if it needs to be resumed. ########## python/pyspark/sql/connect/client/reattach.py: ########## @@ -0,0 +1,228 @@ +# +# 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. +# +from pyspark.sql.connect.utils import check_dependencies + +check_dependencies(__name__) + +import uuid +from collections.abc import Generator +from typing import Optional, Dict, Any, Iterator +import threading + +import pyspark.sql.connect.proto as pb2 +import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib + + +class ExecutePlanResponseReattachableIterator(Generator): + """ + 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 + """ + + def __init__( + self, + request: pb2.ExecutePlanRequest, + stub: grpc_lib.SparkConnectServiceStub, + retry_policy: Dict[str, Any], + ): + self._request = request + self._retry_policy = retry_policy + if request.operation_id: + self._operation_id = request.operation_id + 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. + self._operation_id = str(uuid.uuid4()) + + self._stub = stub + request.request_options.append( + pb2.ExecutePlanRequest.RequestOption( + reattach_options=pb2.ReattachOptions(reattachable=True) + ) + ) + self._initial_request = request + + # ResponseId of the last response returned by next() + self._last_returned_response_id: Optional[str] = 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. + self._result_complete = False + + # Initial iterator comes from ExecutePlan request. + self._iterator: Iterator[pb2.ExecutePlanResponse] = self._stub.ExecutePlan( Review Comment: So in the case of the target resource throwing a retriable error, this will already raise an exception and break the retry logic. It looks like the current retry handling can only deal with subsequent errors in the handling of the response stream but not the initial error itself. ########## python/pyspark/sql/connect/client/reattach.py: ########## @@ -0,0 +1,228 @@ +# +# 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. +# +from pyspark.sql.connect.utils import check_dependencies + +check_dependencies(__name__) + +import uuid +from collections.abc import Generator +from typing import Optional, Dict, Any, Iterator +import threading + +import pyspark.sql.connect.proto as pb2 +import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib + + +class ExecutePlanResponseReattachableIterator(Generator): + """ + 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 + """ + + def __init__( + self, + request: pb2.ExecutePlanRequest, + stub: grpc_lib.SparkConnectServiceStub, + retry_policy: Dict[str, Any], + ): + self._request = request + self._retry_policy = retry_policy + if request.operation_id: + self._operation_id = request.operation_id + 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. + self._operation_id = str(uuid.uuid4()) + + self._stub = stub + request.request_options.append( + pb2.ExecutePlanRequest.RequestOption( + reattach_options=pb2.ReattachOptions(reattachable=True) + ) + ) + self._initial_request = request + + # ResponseId of the last response returned by next() + self._last_returned_response_id: Optional[str] = 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. + self._result_complete = False + + # Initial iterator comes from ExecutePlan request. + self._iterator: Iterator[pb2.ExecutePlanResponse] = self._stub.ExecutePlan( Review Comment: @juliuszsompolski is this the same for the current scala client? ########## python/pyspark/sql/connect/client/reattach.py: ########## @@ -0,0 +1,228 @@ +# +# 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. +# +from pyspark.sql.connect.utils import check_dependencies + +check_dependencies(__name__) + +import uuid +from collections.abc import Generator +from typing import Optional, Dict, Any, Iterator +import threading + +import pyspark.sql.connect.proto as pb2 +import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib + + +class ExecutePlanResponseReattachableIterator(Generator): + """ + 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 + """ + + def __init__( + self, + request: pb2.ExecutePlanRequest, + stub: grpc_lib.SparkConnectServiceStub, + retry_policy: Dict[str, Any], + ): + self._request = request + self._retry_policy = retry_policy + if request.operation_id: + self._operation_id = request.operation_id + 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. + self._operation_id = str(uuid.uuid4()) + + self._stub = stub + request.request_options.append( + pb2.ExecutePlanRequest.RequestOption( + reattach_options=pb2.ReattachOptions(reattachable=True) + ) + ) + self._initial_request = request + + # ResponseId of the last response returned by next() + self._last_returned_response_id: Optional[str] = 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. + self._result_complete = False + + # Initial iterator comes from ExecutePlan request. + self._iterator: Iterator[pb2.ExecutePlanResponse] = self._stub.ExecutePlan( + self._initial_request + ) + + # Current item from this iterator. + self._current: Optional[pb2.ExecutePlanResponse] = None + + def send(self, value: Any) -> pb2.ExecutePlanResponse: + # will trigger reattach in case the stream completed without result_complete + if not self._has_next(): + raise StopIteration() + + ret = self._current + assert ret is not None + + self._last_returned_response_id = ret.response_id + if ret.result_complete: + self._result_complete = True + self._release_execute(None) # release all + else: + self._release_execute(self._last_returned_response_id) + self._current = None + return ret + + def _has_next(self) -> bool: + from pyspark.sql.connect.client.core import SparkConnectClient + from pyspark.sql.connect.client.core import Retrying + + if self._result_complete: + # After response complete response + return False + else: + first_try = True + for attempt in Retrying( + can_retry=SparkConnectClient.retry_exception, **self._retry_policy + ): + with attempt: + if first_try: + # on first try, we use the existing iterator. + first_try = False + else: + # on retry, the iterator is borked, so we need a new one + self._iterator = self._stub.ReattachExecute( + self._create_reattach_execute_request() + ) + + if self._current is None: + try: + self._current = next(self._iterator) + except StopIteration: + pass + + has_next = self._current is not None + + # Graceful reattach: + # If iterator ended, but there was no ResponseComplete, it means that + # there is more, and we need to reattach. While ResponseComplete didn't + # arrive, we keep reattaching. + first_loop = True + if not has_next and not self._result_complete: Review Comment: Just readability. ```suggestion if not self._result_complete and not has_next: ``` ########## python/pyspark/sql/connect/client/reattach.py: ########## @@ -0,0 +1,228 @@ +# +# 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. +# +from pyspark.sql.connect.utils import check_dependencies + +check_dependencies(__name__) + +import uuid +from collections.abc import Generator +from typing import Optional, Dict, Any, Iterator +import threading + +import pyspark.sql.connect.proto as pb2 +import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib + + +class ExecutePlanResponseReattachableIterator(Generator): + """ + 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 + """ + + def __init__( + self, + request: pb2.ExecutePlanRequest, + stub: grpc_lib.SparkConnectServiceStub, + retry_policy: Dict[str, Any], + ): + self._request = request + self._retry_policy = retry_policy + if request.operation_id: + self._operation_id = request.operation_id + 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. + self._operation_id = str(uuid.uuid4()) + + self._stub = stub + request.request_options.append( + pb2.ExecutePlanRequest.RequestOption( + reattach_options=pb2.ReattachOptions(reattachable=True) + ) + ) + self._initial_request = request + + # ResponseId of the last response returned by next() + self._last_returned_response_id: Optional[str] = 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. + self._result_complete = False + + # Initial iterator comes from ExecutePlan request. + self._iterator: Iterator[pb2.ExecutePlanResponse] = self._stub.ExecutePlan( + self._initial_request + ) + + # Current item from this iterator. + self._current: Optional[pb2.ExecutePlanResponse] = None + + def send(self, value: Any) -> pb2.ExecutePlanResponse: + # will trigger reattach in case the stream completed without result_complete + if not self._has_next(): + raise StopIteration() + + ret = self._current + assert ret is not None + + self._last_returned_response_id = ret.response_id + if ret.result_complete: + self._result_complete = True + self._release_execute(None) # release all + else: + self._release_execute(self._last_returned_response_id) + self._current = None + return ret + + def _has_next(self) -> bool: + from pyspark.sql.connect.client.core import SparkConnectClient + from pyspark.sql.connect.client.core import Retrying + + if self._result_complete: + # After response complete response + return False + else: + first_try = True + for attempt in Retrying( + can_retry=SparkConnectClient.retry_exception, **self._retry_policy + ): + with attempt: + if first_try: Review Comment: or maybe `attempt.first_try` (added to attempt manager) ########## python/pyspark/sql/connect/client/reattach.py: ########## @@ -0,0 +1,228 @@ +# +# 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. +# +from pyspark.sql.connect.utils import check_dependencies + +check_dependencies(__name__) + +import uuid +from collections.abc import Generator +from typing import Optional, Dict, Any, Iterator +import threading + +import pyspark.sql.connect.proto as pb2 +import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib + + +class ExecutePlanResponseReattachableIterator(Generator): + """ + 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 + """ + + def __init__( + self, + request: pb2.ExecutePlanRequest, + stub: grpc_lib.SparkConnectServiceStub, + retry_policy: Dict[str, Any], + ): + self._request = request + self._retry_policy = retry_policy + if request.operation_id: + self._operation_id = request.operation_id + 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. + self._operation_id = str(uuid.uuid4()) + + self._stub = stub + request.request_options.append( + pb2.ExecutePlanRequest.RequestOption( + reattach_options=pb2.ReattachOptions(reattachable=True) + ) + ) + self._initial_request = request + + # ResponseId of the last response returned by next() + self._last_returned_response_id: Optional[str] = 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. + self._result_complete = False + + # Initial iterator comes from ExecutePlan request. + self._iterator: Iterator[pb2.ExecutePlanResponse] = self._stub.ExecutePlan( + self._initial_request + ) + + # Current item from this iterator. + self._current: Optional[pb2.ExecutePlanResponse] = None + + def send(self, value: Any) -> pb2.ExecutePlanResponse: + # will trigger reattach in case the stream completed without result_complete + if not self._has_next(): + raise StopIteration() + + ret = self._current + assert ret is not None + + self._last_returned_response_id = ret.response_id + if ret.result_complete: + self._result_complete = True + self._release_execute(None) # release all + else: + self._release_execute(self._last_returned_response_id) + self._current = None + return ret + + def _has_next(self) -> bool: + from pyspark.sql.connect.client.core import SparkConnectClient + from pyspark.sql.connect.client.core import Retrying + + if self._result_complete: + # After response complete response + return False + else: + first_try = True + for attempt in Retrying( + can_retry=SparkConnectClient.retry_exception, **self._retry_policy + ): + with attempt: + if first_try: + # on first try, we use the existing iterator. + first_try = False + else: + # on retry, the iterator is borked, so we need a new one + self._iterator = self._stub.ReattachExecute( + self._create_reattach_execute_request() + ) + + if self._current is None: + try: + self._current = next(self._iterator) + except StopIteration: + pass + + has_next = self._current is not None + + # Graceful reattach: + # If iterator ended, but there was no ResponseComplete, it means that + # there is more, and we need to reattach. While ResponseComplete didn't + # arrive, we keep reattaching. + first_loop = True + if not has_next and not self._result_complete: + while not has_next or first_loop: + self._iterator = self._stub.ReattachExecute( Review Comment: This could raise itself with a retriable error. ########## python/pyspark/sql/connect/client/reattach.py: ########## @@ -0,0 +1,228 @@ +# +# 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. +# +from pyspark.sql.connect.utils import check_dependencies + +check_dependencies(__name__) + +import uuid +from collections.abc import Generator +from typing import Optional, Dict, Any, Iterator +import threading + +import pyspark.sql.connect.proto as pb2 +import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib + + +class ExecutePlanResponseReattachableIterator(Generator): + """ + 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 + """ + + def __init__( + self, + request: pb2.ExecutePlanRequest, + stub: grpc_lib.SparkConnectServiceStub, + retry_policy: Dict[str, Any], + ): + self._request = request + self._retry_policy = retry_policy + if request.operation_id: + self._operation_id = request.operation_id + 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. + self._operation_id = str(uuid.uuid4()) + + self._stub = stub + request.request_options.append( + pb2.ExecutePlanRequest.RequestOption( + reattach_options=pb2.ReattachOptions(reattachable=True) + ) + ) + self._initial_request = request + + # ResponseId of the last response returned by next() + self._last_returned_response_id: Optional[str] = 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. + self._result_complete = False + + # Initial iterator comes from ExecutePlan request. + self._iterator: Iterator[pb2.ExecutePlanResponse] = self._stub.ExecutePlan( + self._initial_request + ) + + # Current item from this iterator. + self._current: Optional[pb2.ExecutePlanResponse] = None + + def send(self, value: Any) -> pb2.ExecutePlanResponse: + # will trigger reattach in case the stream completed without result_complete + if not self._has_next(): + raise StopIteration() + + ret = self._current + assert ret is not None + + self._last_returned_response_id = ret.response_id + if ret.result_complete: Review Comment: I would add a comment that this assumes that this is part of the oneof and set to none if it's not set in this response -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
