You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/11/09 23:37:21 UTC

[GitHub] [flink] gaoyunhaii commented on a diff in pull request #21077: [FLINK-29498] Add Scala Async Retry Strategies and ResultPredicates Helper Classes

gaoyunhaii commented on code in PR #21077:
URL: https://github.com/apache/flink/pull/21077#discussion_r1018510193


##########
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncRetryStrategies.scala:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.flink.streaming.api.scala.async
+
+import org.apache.flink.streaming.api.functions.async
+import org.apache.flink.streaming.api.functions.async.{AsyncRetryStrategy => JAsyncRetryStrategy}
+import org.apache.flink.streaming.util.retryable.{AsyncRetryStrategies => JAsyncRetryStrategies}
+
+import java.{util => ju}
+import java.util.function.Predicate
+
+object AsyncRetryStrategies {
+
+  final private class JavaToScalaRetryStrategy[T] {
+    def convert(retryStrategy: JAsyncRetryStrategy[T]): AsyncRetryStrategy[T] = {
+      new AsyncRetryStrategy[T] {
+        override def canRetry(currentAttempts: Int): Boolean =
+          retryStrategy.canRetry(currentAttempts)
+
+        override def getBackoffTimeMillis(currentAttempts: Int): Long =
+          retryStrategy.getBackoffTimeMillis(currentAttempts)
+
+        override def getRetryPredicate(): AsyncRetryPredicate[T] = new AsyncRetryPredicate[T] {
+          val retryPredicates: async.AsyncRetryPredicate[T] = retryStrategy.getRetryPredicate
+
+          override def resultPredicate: Option[Predicate[ju.Collection[T]]] = {
+            if (retryPredicates.resultPredicate.isPresent)

Review Comment:
   Perhaps it could be simplified with 
   `Option(...orElseGet(null))`



##########
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncRetryStrategies.scala:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.flink.streaming.api.scala.async
+
+import org.apache.flink.streaming.api.functions.async
+import org.apache.flink.streaming.api.functions.async.{AsyncRetryStrategy => JAsyncRetryStrategy}
+import org.apache.flink.streaming.util.retryable.{AsyncRetryStrategies => JAsyncRetryStrategies}
+
+import java.{util => ju}
+import java.util.function.Predicate
+
+object AsyncRetryStrategies {
+
+  final private class JavaToScalaRetryStrategy[T] {

Review Comment:
   It seems we do not need to wrap it as a factory ? Perhaps it would be simple if we could simplify it to 
   
   ```
   final private class JavaToScalaRetryStrategy[T](retryStrategy: JAsyncRetryStrategy[T])
       extends AsyncRetryStrategy[T] {
   
       /** @return whether the next attempt can happen */
       override def canRetry(currentAttempts: Int): Boolean = retryStrategy.canRetry(currentAttempts)
   
       /** @return the delay time of next attempt */
       override def getBackoffTimeMillis(currentAttempts: Int): Long =
         retryStrategy.getBackoffTimeMillis(currentAttempts)
   
       /** @return the defined retry predicate {@link AsyncRetryPredicate} */
       override def getRetryPredicate(): AsyncRetryPredicate[T] = new AsyncRetryPredicate[T] {
   
         /**
          * An Optional Java {@Predicate } that defines a condition on asyncFunction's future result
          * which will trigger a later reattempt operation, will be called before user's
          * ResultFuture#complete.
          *
          * @return
          *   predicate on result of {@link util.Collection}
          */
         override def resultPredicate: Option[Predicate[util.Collection[T]]] =
           Option(retryStrategy.getRetryPredicate.resultPredicate().orElseGet(null))
   
         /**
          * An Optional Java {@Predicate } that defines a condition on asyncFunction's exception which
          * will trigger a later reattempt operation, will be called before user's
          * ResultFuture#completeExceptionally.
          *
          * @return
          *   predicate on {@link Throwable} exception
          */
         override def exceptionPredicate: Option[Predicate[Throwable]] = Option(
           retryStrategy.getRetryPredicate.exceptionPredicate().orElseGet(null))
       }
     }
   ```
   
   Then we could also remove the `convert` in the builders and directly return
   
   `new JavaToScalaRetryStrategy[OUT](builder.build())`



##########
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncRetryStrategies.scala:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.flink.streaming.api.scala.async
+
+import org.apache.flink.streaming.api.functions.async
+import org.apache.flink.streaming.api.functions.async.{AsyncRetryStrategy => JAsyncRetryStrategy}
+import org.apache.flink.streaming.util.retryable.{AsyncRetryStrategies => JAsyncRetryStrategies}
+
+import java.{util => ju}
+import java.util.function.Predicate
+
+object AsyncRetryStrategies {
+
+  final private class JavaToScalaRetryStrategy[T] {
+    def convert(retryStrategy: JAsyncRetryStrategy[T]): AsyncRetryStrategy[T] = {
+      new AsyncRetryStrategy[T] {
+        override def canRetry(currentAttempts: Int): Boolean =
+          retryStrategy.canRetry(currentAttempts)
+
+        override def getBackoffTimeMillis(currentAttempts: Int): Long =
+          retryStrategy.getBackoffTimeMillis(currentAttempts)
+
+        override def getRetryPredicate(): AsyncRetryPredicate[T] = new AsyncRetryPredicate[T] {
+          val retryPredicates: async.AsyncRetryPredicate[T] = retryStrategy.getRetryPredicate
+
+          override def resultPredicate: Option[Predicate[ju.Collection[T]]] = {
+            if (retryPredicates.resultPredicate.isPresent)
+              Option(retryPredicates.resultPredicate.get)
+            else Option.empty
+          }
+
+          override def exceptionPredicate: Option[Predicate[Throwable]] = {
+            if (retryPredicates.exceptionPredicate.isPresent)
+              Option(retryPredicates.exceptionPredicate.get)
+            else Option.empty
+          }
+        }
+      }
+    }
+  }
+
+  @SerialVersionUID(1L)
+  class FixedDelayRetryStrategyBuilder[OUT](

Review Comment:
   Marked with @PublicEvolving



##########
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/RetryPredicates.scala:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.flink.streaming.api.scala.async
+
+import org.apache.flink.streaming.util.retryable.{RetryPredicates => JRetryPredicates}
+
+import java.util
+import java.util.function.Predicate
+
+object RetryPredicates {

Review Comment:
   Might add some comments.



##########
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/RetryPredicates.scala:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.flink.streaming.api.scala.async
+
+import org.apache.flink.streaming.util.retryable.{RetryPredicates => JRetryPredicates}
+
+import java.util
+import java.util.function.Predicate
+
+object RetryPredicates {

Review Comment:
   Might add some comments. Also Marked with @PublicEvolving



##########
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncRetryStrategies.scala:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.flink.streaming.api.scala.async
+
+import org.apache.flink.streaming.api.functions.async
+import org.apache.flink.streaming.api.functions.async.{AsyncRetryStrategy => JAsyncRetryStrategy}
+import org.apache.flink.streaming.util.retryable.{AsyncRetryStrategies => JAsyncRetryStrategies}
+
+import java.{util => ju}
+import java.util.function.Predicate
+
+object AsyncRetryStrategies {
+
+  final private class JavaToScalaRetryStrategy[T] {
+    def convert(retryStrategy: JAsyncRetryStrategy[T]): AsyncRetryStrategy[T] = {
+      new AsyncRetryStrategy[T] {
+        override def canRetry(currentAttempts: Int): Boolean =
+          retryStrategy.canRetry(currentAttempts)
+
+        override def getBackoffTimeMillis(currentAttempts: Int): Long =
+          retryStrategy.getBackoffTimeMillis(currentAttempts)
+
+        override def getRetryPredicate(): AsyncRetryPredicate[T] = new AsyncRetryPredicate[T] {
+          val retryPredicates: async.AsyncRetryPredicate[T] = retryStrategy.getRetryPredicate
+
+          override def resultPredicate: Option[Predicate[ju.Collection[T]]] = {
+            if (retryPredicates.resultPredicate.isPresent)
+              Option(retryPredicates.resultPredicate.get)
+            else Option.empty
+          }
+
+          override def exceptionPredicate: Option[Predicate[Throwable]] = {
+            if (retryPredicates.exceptionPredicate.isPresent)
+              Option(retryPredicates.exceptionPredicate.get)
+            else Option.empty
+          }
+        }
+      }
+    }
+  }
+
+  @SerialVersionUID(1L)
+  class FixedDelayRetryStrategyBuilder[OUT](
+      private val maxAttempts: Int,
+      private val backoffTimeMillis: Long
+  ) {
+    private val converter = new JavaToScalaRetryStrategy[OUT]
+    private var builder =
+      new JAsyncRetryStrategies.FixedDelayRetryStrategyBuilder[OUT](maxAttempts, backoffTimeMillis)
+
+    def ifResult(resultRetryPredicate: Predicate[ju.Collection[OUT]])
+        : FixedDelayRetryStrategyBuilder[OUT] = {
+      this.builder = this.builder.ifResult(resultRetryPredicate)
+      this
+    }
+
+    def ifException(
+        exceptionRetryPredicate: Predicate[Throwable]): FixedDelayRetryStrategyBuilder[OUT] = {
+      this.builder = this.builder.ifException(exceptionRetryPredicate)
+      this
+    }
+
+    def build(): AsyncRetryStrategy[OUT] = {
+      converter.convert(builder.build())
+    }
+  }
+
+  @SerialVersionUID(1L)
+  class ExponentialBackoffDelayRetryStrategyBuilder[OUT](

Review Comment:
   Marked with @PublicEvolving



##########
flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/AsyncRetryStrategies.scala:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.flink.streaming.api.scala.async
+
+import org.apache.flink.streaming.api.functions.async
+import org.apache.flink.streaming.api.functions.async.{AsyncRetryStrategy => JAsyncRetryStrategy}
+import org.apache.flink.streaming.util.retryable.{AsyncRetryStrategies => JAsyncRetryStrategies}
+
+import java.{util => ju}
+import java.util.function.Predicate
+
+object AsyncRetryStrategies {

Review Comment:
   Might add some comments 



##########
docs/content/docs/dev/datastream/operators/asyncio.md:
##########
@@ -140,8 +140,8 @@ DataStream<Tuple2<String, String>> resultStream =
 // create an async retry strategy via utility class or a user defined strategy
 AsyncRetryStrategy asyncRetryStrategy =
 	new AsyncRetryStrategies.FixedDelayRetryStrategyBuilder(3, 100L) // maxAttempts=3, fixedDelay=100ms
-		.retryIfResult(RetryPredicates.EMPTY_RESULT_PREDICATE)
-		.retryIfException(RetryPredicates.HAS_EXCEPTION_PREDICATE)
+		.ifResult(RetryPredicates.EMPTY_RESULT_PREDICATE)

Review Comment:
   If convenient might also modify the `docs/content.zh/docs/dev/datastream/operators/asyncio.md`, or it is also ok that I'll change it when merging. 



-- 
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: issues-unsubscribe@flink.apache.org

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