This is an automated email from the ASF dual-hosted git repository.

xushiyan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new a0df6ecfb4 [HUDI-5483] Make RetryHelper more commonly usable (#7572)
a0df6ecfb4 is described below

commit a0df6ecfb47ac9e393b5eb977850cb8fbe7d3f72
Author: Shawy Geng <[email protected]>
AuthorDate: Wed Jan 4 19:17:57 2023 +0800

    [HUDI-5483] Make RetryHelper more commonly usable (#7572)
    
    
    
    Co-authored-by: gengxiaoyu <[email protected]>
---
 .../common/fs/HoodieRetryWrapperFileSystem.java    | 71 ++++++++++++----------
 .../view/RemoteHoodieTableFileSystemView.java      |  2 +-
 .../org/apache/hudi/common/util/RetryHelper.java   | 32 ++++------
 .../apache/hudi/common/util/TestRetryHelper.java   | 52 ++++++++++++++++
 .../TestRemoteHoodieTableFileSystemView.java       |  4 +-
 5 files changed, 105 insertions(+), 56 deletions(-)

diff --git 
a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieRetryWrapperFileSystem.java
 
b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieRetryWrapperFileSystem.java
index 82ea10c459..d10813021d 100644
--- 
a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieRetryWrapperFileSystem.java
+++ 
b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieRetryWrapperFileSystem.java
@@ -66,12 +66,12 @@ public class HoodieRetryWrapperFileSystem extends 
FileSystem {
 
   @Override
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
-    return (FSDataInputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.open(f, bufferSize)).start();
+    return new RetryHelper<FSDataInputStream, IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.open(f, bufferSize)).start();
   }
 
   @Override
   public FSDataInputStream open(Path f) throws IOException {
-    return (FSDataInputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.open(f)).start();
+    return new RetryHelper<FSDataInputStream, IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.open(f)).start();
   }
 
   @Override
@@ -82,66 +82,67 @@ public class HoodieRetryWrapperFileSystem extends 
FileSystem {
                                    short replication,
                                    long blockSize,
                                    Progressable progress) throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
         .tryWith(() -> fileSystem.create(f, permission, overwrite, bufferSize, 
replication, blockSize, progress)).start();
   }
 
   @Override
   public FSDataOutputStream create(Path f, boolean overwrite) throws 
IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.create(f, overwrite)).start();
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList).tryWith(() -> fileSystem.create(f, overwrite)).start();
   }
 
   @Override
   public FSDataOutputStream create(Path f) throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.create(f)).start();
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList).tryWith(() -> fileSystem.create(f)).start();
   }
 
   @Override
   public FSDataOutputStream create(Path f, Progressable progress) throws 
IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.create(f, progress)).start();
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList).tryWith(() -> fileSystem.create(f, progress)).start();
   }
 
   @Override
   public FSDataOutputStream create(Path f, short replication) throws 
IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.create(f, replication)).start();
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList).tryWith(() -> fileSystem.create(f, replication)).start();
   }
 
   @Override
   public FSDataOutputStream create(Path f, short replication, Progressable 
progress) throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.create(f, replication, progress)).start();
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
+        .tryWith(() -> fileSystem.create(f, replication, progress)).start();
   }
 
   @Override
   public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) 
throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
         .tryWith(() -> fileSystem.create(f, overwrite, bufferSize)).start();
   }
 
   @Override
   public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, 
Progressable progress)
       throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
         .tryWith(() -> fileSystem.create(f, overwrite, bufferSize, 
progress)).start();
   }
 
   @Override
   public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, 
short replication, long blockSize,
                                    Progressable progress) throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
         .tryWith(() -> fileSystem.create(f, overwrite, bufferSize, 
replication, blockSize, progress)).start();
   }
 
   @Override
   public FSDataOutputStream create(Path f, FsPermission permission, 
EnumSet<CreateFlag> flags, int bufferSize,
                                    short replication, long blockSize, 
Progressable progress) throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
         .tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, 
replication, blockSize, progress)).start();
   }
 
   @Override
   public FSDataOutputStream create(Path f, FsPermission permission, 
EnumSet<CreateFlag> flags, int bufferSize,
                                    short replication, long blockSize, 
Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
         .tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, 
replication,
             blockSize, progress, checksumOpt)).start();
   }
@@ -149,83 +150,88 @@ public class HoodieRetryWrapperFileSystem extends 
FileSystem {
   @Override
   public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, 
short replication, long blockSize)
       throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
         .tryWith(() -> fileSystem.create(f, overwrite, bufferSize, 
replication, blockSize)).start();
   }
 
   @Override
   public boolean createNewFile(Path f) throws IOException {
-    return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.createNewFile(f)).start();
+    return new RetryHelper<Boolean, IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.createNewFile(f)).start().booleanValue();
   }
 
   @Override
   public FSDataOutputStream append(Path f, int bufferSize, Progressable 
progress) throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.append(f, bufferSize, progress)).start();
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
+        .tryWith(() -> fileSystem.append(f, bufferSize, progress)).start();
   }
 
   @Override
   public FSDataOutputStream append(Path f) throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.append(f)).start();
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList).tryWith(() -> fileSystem.append(f)).start();
   }
 
   @Override
   public FSDataOutputStream append(Path f, int bufferSize) throws IOException {
-    return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.append(f, bufferSize)).start();
+    return new RetryHelper<FSDataOutputStream, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList).tryWith(() -> fileSystem.append(f, bufferSize)).start();
   }
 
   @Override
   public boolean rename(Path src, Path dst) throws IOException {
-    return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.rename(src, dst)).start();
+    return new RetryHelper<Boolean, IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+        .tryWith(() -> fileSystem.rename(src, dst)).start().booleanValue();
   }
 
   @Override
   public boolean delete(Path f, boolean recursive) throws IOException {
-    return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.delete(f, 
recursive)).start();
+    return new RetryHelper<Boolean, IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+        .tryWith(() -> fileSystem.delete(f, recursive)).start().booleanValue();
   }
 
   @Override
   public boolean delete(Path f) throws IOException {
-    return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.delete(f, 
true)).start();
+    return new RetryHelper<Boolean, IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.delete(f, true)).start().booleanValue();
   }
 
   @Override
   public FileStatus[] listStatus(Path f) throws FileNotFoundException, 
IOException {
-    return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.listStatus(f)).start();
+    return new RetryHelper<FileStatus[], IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.listStatus(f)).start();
   }
 
   @Override
   public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException 
{
-    return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.listStatus(f, filter)).start();
+    return new RetryHelper<FileStatus[], IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.listStatus(f, filter)).start();
   }
 
   @Override
   public FileStatus[] listStatus(Path[] files) throws IOException {
-    return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.listStatus(files)).start();
+    return new RetryHelper<FileStatus[], IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.listStatus(files)).start();
   }
 
   @Override
   public FileStatus[] listStatus(Path[] files, PathFilter filter) throws 
IOException {
-    return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.listStatus(files, filter)).start();
+    return new RetryHelper<FileStatus[], IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.listStatus(files, filter)).start();
   }
 
   @Override
   public FileStatus[] globStatus(Path pathPattern) throws IOException {
-    return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.globStatus(pathPattern)).start();
+    return new RetryHelper<FileStatus[], IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.globStatus(pathPattern)).start();
   }
 
   @Override
   public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws 
IOException {
-    return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.globStatus(pathPattern, filter)).start();
+    return new RetryHelper<FileStatus[], IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+        .tryWith(() -> fileSystem.globStatus(pathPattern, filter)).start();
   }
 
   @Override
   public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f) throws 
IOException {
-    return (RemoteIterator<LocatedFileStatus>) new 
RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList).tryWith(() -> fileSystem.listLocatedStatus(f)).start();
+    return new RetryHelper<RemoteIterator<LocatedFileStatus>, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
+        .tryWith(() -> fileSystem.listLocatedStatus(f)).start();
   }
 
   @Override
   public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean 
recursive) throws IOException {
-    return (RemoteIterator<LocatedFileStatus>) new 
RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
+    return new RetryHelper<RemoteIterator<LocatedFileStatus>, 
IOException>(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, 
retryExceptionsList)
         .tryWith(() -> fileSystem.listFiles(f, recursive)).start();
   }
 
@@ -241,17 +247,18 @@ public class HoodieRetryWrapperFileSystem extends 
FileSystem {
 
   @Override
   public boolean mkdirs(Path f, FsPermission permission) throws IOException {
-    return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.mkdirs(f, 
permission)).start();
+    return new RetryHelper<Boolean, IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList)
+        .tryWith(() -> fileSystem.mkdirs(f, 
permission)).start().booleanValue();
   }
 
   @Override
   public FileStatus getFileStatus(Path f) throws IOException {
-    return (FileStatus) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.getFileStatus(f)).start();
+    return new RetryHelper<FileStatus, IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.getFileStatus(f)).start();
   }
 
   @Override
   public boolean exists(Path f) throws IOException {
-    return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, 
initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.exists(f)).start();
+    return new RetryHelper<Boolean, IOException>(maxRetryIntervalMs, 
maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> 
fileSystem.exists(f)).start().booleanValue();
   }
 
   @Override
diff --git 
a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java
 
b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java
index 759ef70c6d..5ada3dc45b 100644
--- 
a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java
+++ 
b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java
@@ -136,7 +136,7 @@ public class RemoteHoodieTableFileSystemView implements 
SyncableFileSystemView,
 
   private boolean closed = false;
 
-  private RetryHelper<Response> retryHelper;
+  private RetryHelper<Response, IOException> retryHelper;
 
   private enum RequestMethod {
     GET, POST
diff --git 
a/hudi-common/src/main/java/org/apache/hudi/common/util/RetryHelper.java 
b/hudi-common/src/main/java/org/apache/hudi/common/util/RetryHelper.java
index d713a343b6..5cd89c9f5f 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/RetryHelper.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RetryHelper.java
@@ -25,7 +25,6 @@ import org.apache.log4j.Logger;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
@@ -36,9 +35,10 @@ import java.util.stream.Collectors;
  *
  * @param <T> Type of return value for checked function.
  */
-public class RetryHelper<T> implements Serializable {
+public class RetryHelper<T, R extends Exception> implements Serializable {
   private static final Logger LOG = LogManager.getLogger(RetryHelper.class);
-  private transient CheckedFunction<T> func;
+  private static final List<? extends Class<? extends Exception>> 
DEFAULT_RETRY_EXCEPTIONS = Arrays.asList(IOException.class, 
RuntimeException.class);
+  private transient CheckedFunction<T, R> func;
   private final int num;
   private final long maxIntervalTime;
   private final long initialIntervalTime;
@@ -50,7 +50,7 @@ public class RetryHelper<T> implements Serializable {
     this.initialIntervalTime = initialRetryIntervalMs;
     this.maxIntervalTime = maxRetryIntervalMs;
     if (StringUtils.isNullOrEmpty(retryExceptions)) {
-      this.retryExceptionsClasses = new ArrayList<>();
+      this.retryExceptionsClasses = DEFAULT_RETRY_EXCEPTIONS;
     } else {
       try {
         this.retryExceptionsClasses = Arrays.stream(retryExceptions.split(","))
@@ -59,7 +59,7 @@ public class RetryHelper<T> implements Serializable {
             .collect(Collectors.toList());
       } catch (HoodieException e) {
         LOG.error("Exception while loading retry exceptions classes '" + 
retryExceptions + "'.", e);
-        this.retryExceptionsClasses = new ArrayList<>();
+        this.retryExceptionsClasses = DEFAULT_RETRY_EXCEPTIONS;
       }
     }
   }
@@ -69,12 +69,12 @@ public class RetryHelper<T> implements Serializable {
     this.taskInfo = taskInfo;
   }
 
-  public RetryHelper<T> tryWith(CheckedFunction<T> func) {
+  public RetryHelper<T, R> tryWith(CheckedFunction<T, R> func) {
     this.func = func;
     return this;
   }
 
-  public T start(CheckedFunction<T> func) throws IOException {
+  public <R extends Exception> T start(CheckedFunction<T, R> func) throws R {
     int retries = 0;
     T functionResult = null;
 
@@ -83,16 +83,13 @@ public class RetryHelper<T> implements Serializable {
       try {
         functionResult = func.get();
         break;
-      } catch (IOException | RuntimeException e) {
+      } catch (Exception e) {
         if (!checkIfExceptionInRetryList(e)) {
           throw e;
         }
         if (retries++ >= num) {
           String message = "Still failed to " + taskInfo + " after retried " + 
num + " times.";
           LOG.error(message, e);
-          if (e instanceof IOException) {
-            throw new IOException(message, e);
-          }
           throw e;
         }
         LOG.warn("Catch Exception for " + taskInfo + ", will retry after " + 
waitTime + " ms.", e);
@@ -111,19 +108,12 @@ public class RetryHelper<T> implements Serializable {
     return functionResult;
   }
 
-  public T start() throws IOException {
+  public T start() throws R {
     return start(this.func);
   }
 
   private boolean checkIfExceptionInRetryList(Exception e) {
     boolean inRetryList = false;
-
-    // if users didn't set hoodie.filesystem.operation.retry.exceptions
-    // we will retry all the IOException and RuntimeException
-    if (retryExceptionsClasses.isEmpty()) {
-      return true;
-    }
-
     for (Class<? extends Exception> clazz : retryExceptionsClasses) {
       if (clazz.isInstance(e)) {
         inRetryList = true;
@@ -148,7 +138,7 @@ public class RetryHelper<T> implements Serializable {
    * @param <T> Type of return value.
    */
   @FunctionalInterface
-  public interface CheckedFunction<T> extends Serializable {
-    T get() throws IOException;
+  public interface CheckedFunction<T, R extends Exception> extends 
Serializable {
+    T get() throws R;
   }
 }
\ No newline at end of file
diff --git 
a/hudi-common/src/test/java/org/apache/hudi/common/util/TestRetryHelper.java 
b/hudi-common/src/test/java/org/apache/hudi/common/util/TestRetryHelper.java
new file mode 100644
index 0000000000..55f63288f6
--- /dev/null
+++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestRetryHelper.java
@@ -0,0 +1,52 @@
+/*
+ * 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.hudi.common.util;
+
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.lang.reflect.Method;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+/**
+ * Test retry helper.
+ */
+public class TestRetryHelper {
+
+  private static final int NUM = 1;
+  private static final long INTERVAL_TIME = 1L;
+
+  @Test
+  public void testCheckIfExceptionInRetryList() throws Exception {
+    // test default retry exceptions
+    RetryHelper retryHelper = new RetryHelper(INTERVAL_TIME, NUM, 
INTERVAL_TIME, null);
+    Method privateOne = 
retryHelper.getClass().getDeclaredMethod("checkIfExceptionInRetryList", 
Exception.class);
+    privateOne.setAccessible(true);
+    boolean retry = (boolean) privateOne.invoke(retryHelper, new 
IOException("test"));
+    assertTrue(retry);
+    retry = (boolean) privateOne.invoke(retryHelper, new Exception("test"));
+    assertFalse(retry);
+    // test user-defined retry exceptions
+    retryHelper =  new RetryHelper(INTERVAL_TIME, NUM, INTERVAL_TIME, 
Exception.class.getName());
+    retry = (boolean) privateOne.invoke(retryHelper, new 
UnsupportedOperationException("test"));
+    assertTrue(retry);
+  }
+}
diff --git 
a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java
 
b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java
index 127bc51e95..bc7f7ee311 100644
--- 
a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java
+++ 
b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java
@@ -93,9 +93,9 @@ public class TestRemoteHoodieTableFileSystemView extends 
TestHoodieTableFileSyst
             .build());
     try {
       view.getLatestBaseFiles();
-      fail("Should be catch Exception 'Still failed to Sending request after 
retried 4 times.'");
+      fail("Should be catch Exception 'Connection refused (Connection 
refused)'");
     } catch (HoodieRemoteException e) {
-      assert e.getMessage().equalsIgnoreCase("Still failed to Sending request 
after retried 4 times.");
+      assert e.getMessage().contains("Connection refused (Connection 
refused)");
     }
     // Retry succeed after 2 or 3 tries.
     new Thread(() -> {

Reply via email to