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


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala:
##########
@@ -60,14 +64,66 @@ private[sql] class SparkConnectClient(
     new ArtifactManager(userContext, sessionId, channel)
   }
 
+  private val retryPolicy: SparkConnectClient.RetryPolicy = 
configuration.retryPolicy
+
+  @tailrec private[client] final def retry[T](fn: => T, currentRetryNum: Int = 
0): T = {
+    if (currentRetryNum > retryPolicy.maxRetries) {
+      throw new IllegalArgumentException(
+        s"The number of retries ($currentRetryNum) must not exceed " +
+          s"the maximum number of retires (${retryPolicy.maxRetries}).")
+    }
+    try {
+      return fn
+    } catch {
+      case NonFatal(e) if retryPolicy.canRetry(e) && currentRetryNum < 
retryPolicy.maxRetries =>
+        Thread.sleep(
+          (retryPolicy.maxBackoff min retryPolicy.initialBackoff * Math
+            .pow(retryPolicy.backoffMultiplier, currentRetryNum)).toMillis)
+    }
+    retry(fn, currentRetryNum + 1)
+  }
+
   /**
    * Dispatch the [[proto.AnalyzePlanRequest]] to the Spark Connect server.
    * @return
    *   A [[proto.AnalyzePlanResponse]] from the Spark Connect server.
    */
   def analyze(request: proto.AnalyzePlanRequest): proto.AnalyzePlanResponse = {
     artifactManager.uploadAllClassFileArtifacts()
-    stub.analyzePlan(request)
+    retry {
+      stub.analyzePlan(request)
+    }
+  }
+
+  private class executeRetryIterator(
+      request: proto.ExecutePlanRequest,
+      origIterator: java.util.Iterator[proto.ExecutePlanResponse])

Review Comment:
   nit: Class name starts with Capital.
   You could pass
   ```
   request: T
   call: T => java.util.Iterator[U]
   ```
   and have `var iterator = call(request)` at constructor time
   to generalize it ad `RetryIterator`



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala:
##########
@@ -60,14 +64,66 @@ private[sql] class SparkConnectClient(
     new ArtifactManager(userContext, sessionId, channel)
   }
 
+  private val retryPolicy: SparkConnectClient.RetryPolicy = 
configuration.retryPolicy
+
+  @tailrec private[client] final def retry[T](fn: => T, currentRetryNum: Int = 
0): T = {
+    if (currentRetryNum > retryPolicy.maxRetries) {
+      throw new IllegalArgumentException(
+        s"The number of retries ($currentRetryNum) must not exceed " +
+          s"the maximum number of retires (${retryPolicy.maxRetries}).")
+    }
+    try {
+      return fn
+    } catch {
+      case NonFatal(e) if retryPolicy.canRetry(e) && currentRetryNum < 
retryPolicy.maxRetries =>
+        Thread.sleep(
+          (retryPolicy.maxBackoff min retryPolicy.initialBackoff * Math
+            .pow(retryPolicy.backoffMultiplier, currentRetryNum)).toMillis)
+    }
+    retry(fn, currentRetryNum + 1)
+  }
+
   /**
    * Dispatch the [[proto.AnalyzePlanRequest]] to the Spark Connect server.
    * @return
    *   A [[proto.AnalyzePlanResponse]] from the Spark Connect server.
    */
   def analyze(request: proto.AnalyzePlanRequest): proto.AnalyzePlanResponse = {
     artifactManager.uploadAllClassFileArtifacts()
-    stub.analyzePlan(request)
+    retry {
+      stub.analyzePlan(request)
+    }
+  }
+
+  private class executeRetryIterator(
+      request: proto.ExecutePlanRequest,
+      origIterator: java.util.Iterator[proto.ExecutePlanResponse])
+      extends java.util.Iterator[proto.ExecutePlanResponse] {
+
+    private var hasNextCalled = false
+    private var iterator = origIterator
+
+    override def next(): proto.ExecutePlanResponse = {
+      iterator.next()

Review Comment:
   we should set `hasNextCalled = true` also here. Otherwise user can call a 
couple of `next()` before calling `hasNext()` and we'll treat it as if we were 
still at the beginning of iterator. Maybe rename `hasNextCalled` to `opened`
   
   Actually, you could generalize
   ```
   def retryIter[T](f: java.util.Iterator[proto.ExecutePlanResponse] => T) = {
         if (!opened) {
           opened = true
           var firstTry = true
           retry {
             if (firstTry) {
               // on first try, we use the iterator provided by constructor
               firstTry = false
               f(iterator)
             } else {
               // on retry, we need to call the RPC again.
               iterator = stub.executePlan(request)
               f(iterator)
             }
           }
         } else {
           f(iterator)
         }
       }
   }
   ```
   
   and have both next() and hasNext() be wrapped for retry.
   We could also generalize the stub.executePlan(request), and have a solution 
for any future streaming iter RPC.



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala:
##########
@@ -60,14 +64,66 @@ private[sql] class SparkConnectClient(
     new ArtifactManager(userContext, sessionId, channel)
   }
 
+  private val retryPolicy: SparkConnectClient.RetryPolicy = 
configuration.retryPolicy
+
+  @tailrec private[client] final def retry[T](fn: => T, currentRetryNum: Int = 
0): T = {
+    if (currentRetryNum > retryPolicy.maxRetries) {
+      throw new IllegalArgumentException(
+        s"The number of retries ($currentRetryNum) must not exceed " +
+          s"the maximum number of retires (${retryPolicy.maxRetries}).")
+    }
+    try {
+      return fn
+    } catch {
+      case NonFatal(e) if retryPolicy.canRetry(e) && currentRetryNum < 
retryPolicy.maxRetries =>
+        Thread.sleep(
+          (retryPolicy.maxBackoff min retryPolicy.initialBackoff * Math
+            .pow(retryPolicy.backoffMultiplier, currentRetryNum)).toMillis)
+    }
+    retry(fn, currentRetryNum + 1)
+  }
+
   /**
    * Dispatch the [[proto.AnalyzePlanRequest]] to the Spark Connect server.
    * @return
    *   A [[proto.AnalyzePlanResponse]] from the Spark Connect server.
    */
   def analyze(request: proto.AnalyzePlanRequest): proto.AnalyzePlanResponse = {
     artifactManager.uploadAllClassFileArtifacts()
-    stub.analyzePlan(request)
+    retry {
+      stub.analyzePlan(request)
+    }
+  }
+
+  private class executeRetryIterator(
+      request: proto.ExecutePlanRequest,
+      origIterator: java.util.Iterator[proto.ExecutePlanResponse])
+      extends java.util.Iterator[proto.ExecutePlanResponse] {
+
+    private var hasNextCalled = false

Review Comment:
   Rename to `opened` with a comment `// we only retry if it fails on first 
call when using the iterator`



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