juliuszsompolski commented on code in PR #42235:
URL: https://github.com/apache/spark/pull/42235#discussion_r1281120432


##########
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:
   At least in scala, calling the `executePlan` never throws error, even if the 
service is absolutely not available and one could expect immediate error; I 
tested that with completely killing the server, then calling executePlan, and 
never getting an actual error before trying to call next()... The error is only 
thrown on the first `next()` or `hasNext()` call of the resulting iterator. So 
in scala not catching and retrying here, but at the first iterator invocation 
is fine. So in 
https://github.com/apache/spark/pull/42228/files#diff-2efc03ab3edd5e5f78652b1454b31fd4917113de09a64fbdc7352f150c685628R89
 and also in 
https://github.com/apache/spark/pull/41829/files#diff-79d952ace19b8b9d4b04e010653aba35e4aeaca957946cc68414b7a766deea99R68
 doesn't wrap the first call
   we do not retry.
   
   But it's a good callout that it needs checking if python GRPC behaves the 
same, or maybe just defensively wrap it in retry anyway (and do the same in 
scala just in case it's a GRPC internal quirk that we shouldn't depend on), or 
at least it deserves a comment.
   
   
   



-- 
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]

Reply via email to