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


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala:
##########
@@ -44,20 +43,23 @@ import org.apache.spark.util.{SparkFileUtils, 
SparkThreadUtils}
  * The Artifact Manager is responsible for handling and transferring artifacts 
from the local
  * client to the server (local/remote).
  * @param userContext
+ *   The user context the artifact manager operates in.
  * @param sessionId
  *   An unique identifier of the session which the artifact manager belongs to.
- * @param channel
+ * @param bstub
+ *   A blocking stub to the server.
+ * @param stub
+ *   An async stub to the server.
  */
 class ArtifactManager(
-    userContext: proto.UserContext,
-    sessionId: String,
-    channel: ManagedChannel) {
+    private val userContext: proto.UserContext,
+    private val sessionId: String,
+    private val bstub: CustomSparkConnectBlockingStub,
+    private val stub: CustomSparkConnectStub) {

Review Comment:
   nit: why do you need `private val` for all of these?



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 scala.annotation.tailrec
+import scala.concurrent.duration.FiniteDuration
+import scala.util.control.NonFatal
+
+import io.grpc.{Status, StatusRuntimeException}
+import io.grpc.stub.StreamObserver
+
+private[client] class GrpcRetryHandler(private val retryPolicy: 
GrpcRetryHandler.RetryPolicy) {
+
+  /**
+   * Retries the given function with exponential backoff according to the 
client's retryPolicy.
+   * @param fn
+   *   The function to retry.
+   * @param currentRetryNum
+   *   Current number of retries.
+   * @tparam T
+   *   The return type of the function.
+   * @return
+   *   The result of the function.
+   */
+  @tailrec 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)
+  }
+
+  /**
+   * Generalizes the retry logic for RPC calls that return an iterator.
+   * @param request
+   *   The request to send to the server.
+   * @param call
+   *   The function that calls the RPC.
+   * @tparam T
+   *   The type of the request.
+   * @tparam U
+   *   The type of the response.
+   */
+  class RetryIterator[T, U](request: T, call: T => java.util.Iterator[U])
+      extends java.util.Iterator[U] {
+
+    private var opened = false // we only retry if it fails on first call when 
using the iterator
+    private var iterator = call(request)
+
+    private def retryIter[V](f: java.util.Iterator[U] => V) = {
+      if (!opened) {
+        opened = true
+        var firstTry = true
+        retry {
+          if (firstTry) {
+            // on first try, we use the initial iterator.
+            firstTry = false
+          } else {
+            // on retry, we need to call the RPC again.
+            iterator = call(request)
+          }
+          f(iterator)
+        }
+      } else {
+        f(iterator)
+      }
+    }
+
+    override def next: U = {
+      retryIter(_.next())
+    }
+
+    override def hasNext: Boolean = {
+      retryIter(_.hasNext())
+    }
+  }
+
+  object RetryIterator {
+    def apply[T, U](request: T, call: T => java.util.Iterator[U]): 
RetryIterator[T, U] =
+      new RetryIterator(request, call)
+  }
+
+  /**
+   * Generalizes the retry logic for RPC calls that return a StreamObserver.
+   * @param request
+   *   The request to send to the server.
+   * @param call
+   *   The function that calls the RPC.
+   * @tparam T
+   *   The type of the request.
+   * @tparam U
+   *   The type of the response.
+   */
+  class RetryStreamObserver[T, U](request: T, call: T => StreamObserver[U])
+      extends StreamObserver[U] {
+
+    private var opened = false // only retries on first call
+    private var streamObserver = call(request)
+    override def onNext(v: U): Unit = {

Review Comment:
   nit: empty line between methods (here and below)



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