You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "nija-at (via GitHub)" <gi...@apache.org> on 2023/07/03 14:25:08 UTC

[GitHub] [spark] nija-at commented on a diff in pull request #41829: [SPARK-44275][CONNECT] Add configurable retry mechanism to Scala Spark Connect

nija-at commented on code in PR #41829:
URL: https://github.com/apache/spark/pull/41829#discussion_r1250959678


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala:
##########
@@ -564,4 +607,33 @@ object SparkConnectClient {
       }
     }
   }
+
+  private[client] def retryException(e: Throwable): Boolean = {
+    if (e.isInstanceOf[StatusRuntimeException]) {
+      e.asInstanceOf[StatusRuntimeException].getStatus().getCode() == Status.Code.UNAVAILABLE
+    } else {
+      false
+    }
+  }
+
+  /**
+   * [[RetryParameters]] configure the retry mechanism in [[SparkConnectClient]]
+   *
+   * @param max_retries
+   *   Maximum number of retries.
+   * @param initial_backoff
+   *   Start value of the exponential backoff (ms).
+   * @param max_backoff
+   *   Maximal value of the exponential backoff (ms).
+   * @param backoff_multiplier
+   *   Multiplicative base of the exponential backoff.
+   * @param can_retry
+   *   Function that determines whether a retry is to be performed in the event of an error.
+   */
+  private[client] case class RetryParameters(
+      max_retries: Int = 15,
+      initial_backoff: Int = 50,
+      max_backoff: Int = 60000,
+      backoff_multiplier: Double = 4.0,
+      can_retry: Throwable => Boolean = retryException) {}

Review Comment:
   minor naming: "should_retry" sounds better.



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala:
##########
@@ -564,4 +607,33 @@ object SparkConnectClient {
       }
     }
   }
+
+  private[client] def retryException(e: Throwable): Boolean = {
+    if (e.isInstanceOf[StatusRuntimeException]) {
+      e.asInstanceOf[StatusRuntimeException].getStatus().getCode() == Status.Code.UNAVAILABLE
+    } else {
+      false
+    }
+  }
+
+  /**
+   * [[RetryParameters]] configure the retry mechanism in [[SparkConnectClient]]
+   *
+   * @param max_retries
+   *   Maximum number of retries.
+   * @param initial_backoff
+   *   Start value of the exponential backoff (ms).
+   * @param max_backoff
+   *   Maximal value of the exponential backoff (ms).
+   * @param backoff_multiplier
+   *   Multiplicative base of the exponential backoff.
+   * @param can_retry
+   *   Function that determines whether a retry is to be performed in the event of an error.
+   */
+  private[client] case class RetryParameters(
+      max_retries: Int = 15,
+      initial_backoff: Int = 50,
+      max_backoff: Int = 60000,

Review Comment:
   Consider switching to FiniteDuration.
   
   https://docs.oracle.com/javase/8/docs/api/java/time/Duration.html](https://www.scala-lang.org/api/2.12.13/scala/concurrent/duration/FiniteDuration.html



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org