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


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala:
##########
@@ -51,13 +51,14 @@ import org.apache.spark.util.{SparkFileUtils, 
SparkThreadUtils}
 class ArtifactManager(
     userContext: proto.UserContext,
     sessionId: String,
-    channel: ManagedChannel) {
+    channel: ManagedChannel,
+    retryPolicy: GrpcRetryHandler.RetryPolicy) {
   // Using the midpoint recommendation of 32KiB for chunk size as specified in
   // https://github.com/grpc/grpc.github.io/issues/371.
   private val CHUNK_SIZE: Int = 32 * 1024
 
-  private[this] val stub = proto.SparkConnectServiceGrpc.newStub(channel)
-  private[this] val bstub = 
proto.SparkConnectServiceGrpc.newBlockingStub(channel)
+  private[this] val stub = new CustomSparkConnectStub(channel, retryPolicy)
+  private[this] val bstub = new CustomSparkConnectBlockingStub(channel, 
retryPolicy)

Review Comment:
   Could we add functions that ArtifactManager to SparkConnectClient, and pass 
client here to the constructor, and let it use the client?



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala:
##########
@@ -18,34 +18,52 @@ package org.apache.spark.sql.connect.client
 
 import io.grpc.ManagedChannel
 
-import org.apache.spark.connect.proto.{AnalyzePlanRequest, 
AnalyzePlanResponse, ConfigRequest, ConfigResponse, ExecutePlanRequest, 
ExecutePlanResponse, InterruptRequest, InterruptResponse}
+import org.apache.spark.connect.proto.{AnalyzePlanRequest, 
AnalyzePlanResponse, ArtifactStatusesRequest, ArtifactStatusesResponse, 
ConfigRequest, ConfigResponse, ExecutePlanRequest, ExecutePlanResponse, 
InterruptRequest, InterruptResponse}

Review Comment:
   nit: could import `_` at this point (style guide mentions 7 elements as 
threshold for _ import)



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 firstOnNext = false // only retries on first onNext call
+    private var streamObserver = call(request)
+    override def onNext(v: U): Unit = {
+      if (!firstOnNext) {
+        firstOnNext = false
+        var firstTry = true
+        retry {
+          if (firstTry) {
+            // on first try, we use the initial streamObserver.
+            firstTry = false
+          } else {
+            // on retry, we need to call the RPC again.
+            streamObserver = call(request)
+          }
+          streamObserver.onNext(v)
+        }
+      } else {
+        streamObserver.onNext(v)
+      }
+    }
+    override def onError(throwable: Throwable): Unit = {
+      firstOnNext = false
+      streamObserver.onError(throwable)
+    }
+    override def onCompleted(): Unit = {
+      firstOnNext = false
+      streamObserver.onCompleted()
+    }
+  }
+
+  object RetryStreamObserver {
+    def apply[T, U](request: T, call: T => StreamObserver[U]): 
RetryStreamObserver[T, U] =
+      new RetryStreamObserver(request, call)
+  }
+}
+
+private[client] object GrpcRetryHandler {
+  def apply(retryPolicy: RetryPolicy): GrpcRetryHandler = new 
GrpcRetryHandler(retryPolicy)

Review Comment:
   do we need this apply? I think it would be fine for the users to use the 
normal constructor.



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 firstOnNext = false // only retries on first onNext call
+    private var streamObserver = call(request)
+    override def onNext(v: U): Unit = {
+      if (!firstOnNext) {

Review Comment:
   ```suggestion
         if (firstOnNext) {
   ```
   ??



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 firstOnNext = false // only retries on first onNext call

Review Comment:
   ocd nit: would name it `opened`, because it only retries on first onNext 
call, but also if it's the first call at all (as seen by setting firstOnNext = 
false also in calls other than next.



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