From e682349e3cb45bb1cc2234025dabb1727f676bb6 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 26 Oct 2021 10:01:33 +0800 Subject: [PATCH 01/18] need to add configs --- .../fs/HoodieRetryWrapperFileSystem.java | 80 +++++++++++++++ .../common/fs/HoodieWrapperFileSystem.java | 7 ++ .../apache/hudi/common/util/RetryHelper.java | 97 +++++++++++++++++++ 3 files changed, 184 insertions(+) create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieRetryWrapperFileSystem.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/RetryHelper.java 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 new file mode 100644 index 0000000000000..305f1a4d1146e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieRetryWrapperFileSystem.java @@ -0,0 +1,80 @@ +package org.apache.hudi.common.fs; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.util.Progressable; +import org.apache.hudi.common.util.RetryHelper; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; + +public class HoodieRetryWrapperFileSystem extends FileSystem { + + private FileSystem fileSystem; + private RetryHelper retryHelper; + + public HoodieRetryWrapperFileSystem(FileSystem fileSystem, RetryHelper retryHelper) { + this.fileSystem = fileSystem; + this.retryHelper = retryHelper; + } + + @Override + public URI getUri() { + return fileSystem.getUri(); + } + + @Override + public FSDataInputStream open(Path f, int bufferSize) throws IOException { + return fileSystem.open(f, bufferSize); + } + + @Override + public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { + return fileSystem.create(f, permission, overwrite, bufferSize, replication, blockSize, progress); + } + + @Override + public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException { + return fileSystem.append(f, bufferSize, progress); + } + + @Override + public boolean rename(Path src, Path dst) throws IOException { + return fileSystem.rename(src, dst); + } + + @Override + public boolean delete(Path f, boolean recursive) throws IOException { + return fileSystem.delete(f, recursive); + } + + @Override + public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { + return fileSystem.listStatus(f); + } + + @Override + public void setWorkingDirectory(Path new_dir) { + fileSystem.setWorkingDirectory(new_dir); + } + + @Override + public Path getWorkingDirectory() { + return fileSystem.getWorkingDirectory(); + } + + @Override + public boolean mkdirs(Path f, FsPermission permission) throws IOException { + return fileSystem.mkdirs(f, permission); + } + + @Override + public FileStatus getFileStatus(Path f) throws IOException { + return fileSystem.getFileStatus(f); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java index 1faaad5337927..f19e301d186dd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.RetryHelper; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -126,6 +127,12 @@ public HoodieWrapperFileSystem(FileSystem fileSystem, ConsistencyGuard consisten this.consistencyGuard = consistencyGuard; } + public HoodieWrapperFileSystem(FileSystem fileSystem, ConsistencyGuard consistencyGuard, RetryHelper retryHelper) { + this.fileSystem = new HoodieRetryWrapperFileSystem(fileSystem, retryHelper); + this.uri = fileSystem.getUri(); + this.consistencyGuard = consistencyGuard; + } + public static Path convertToHoodiePath(Path file, Configuration conf) { try { String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); 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 new file mode 100644 index 0000000000000..675c3d6f61d02 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RetryHelper.java @@ -0,0 +1,97 @@ +/* + * 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.apache.hudi.common.fs.HoodieWrapperFileSystem; +import org.apache.hudi.exception.HoodieException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.Random; +import java.util.function.Supplier; + +public class RetryHelper { + private static final Logger LOG = LogManager.getLogger(RetryHelper.class); + private HoodieWrapperFileSystem.CheckedFunction func; + private int num; + private long maxIntervalTime; + private String taskInfo; + + public RetryHelper(String taskInfo) { + this.taskInfo = taskInfo; + } + + public RetryHelper tryWith(HoodieWrapperFileSystem.CheckedFunction func) { + this.func = func; + return this; + } + + public RetryHelper tryNum(int num) { + this.num = num; + return this; + } + + public RetryHelper tryInterval(long time) { + maxIntervalTime = time; + return this; + } + + public T start() throws Exception { + int retries = 0; + boolean success = false; + RuntimeException exception = null; + T t = null; + do { + long waitTime = Math.min(getWaitTimeExp(retries), maxIntervalTime); + try { + t = func.get(); + success = true; + break; + } catch (HoodieException hoodieException) { + // ignore hoodieException here + throw hoodieException; + } catch (RuntimeException e) { + // deal with other RuntimeExceptions such like AmazonS3Exception 503 + exception = e; + LOG.warn("Catch RuntimeException" + taskInfo + ", will retry after " + waitTime + " ms.", e); + try { + Thread.sleep(waitTime); + } catch (InterruptedException ex) { + // ignore InterruptedException here + } + retries ++; + } + } while (retries <= num); + + if (!success) { + LOG.error("Still failed to " + taskInfo + " after retried " + num + "times.", exception); + throw exception; + } + LOG.info("Success to " + taskInfo + " after retried " + retries + " times." ); + return t; + } + + private long getWaitTimeExp(int retryCount) { + Random random = new Random(); + if (0 == retryCount) { + return 100L; + } + + return (long) Math.pow(2, retryCount) * 100L + random.nextInt(100); + } +} \ No newline at end of file From d2e9e5ef5a137fe7751d142beb7b3d80ce8677c4 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Wed, 27 Oct 2021 12:10:03 +0800 Subject: [PATCH 02/18] add hoodie filesystem retry wrapper --- .../apache/hudi/cli/commands/SparkMain.java | 11 +- .../hudi/client/AbstractHoodieClient.java | 11 +- .../apache/hudi/config/HoodieWriteConfig.java | 7 + .../apache/hudi/table/HoodieSparkTable.java | 11 +- .../hudi/common/fs/FileSystemGuardConfig.java | 135 ++++++++++++++++++ .../fs/HoodieRetryWrapperFileSystem.java | 41 ++++-- .../common/fs/HoodieWrapperFileSystem.java | 6 - .../common/table/HoodieTableMetaClient.java | 37 ++++- .../apache/hudi/common/util/RetryHelper.java | 26 +++- 9 files changed, 250 insertions(+), 35 deletions(-) create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 82688fecc3663..05a78ab758774 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -450,9 +450,14 @@ private static int deleteSavepoint(JavaSparkContext jsc, String savepointTime, S protected static int upgradeOrDowngradeTable(JavaSparkContext jsc, String basePath, String toVersion) { HoodieWriteConfig config = getWriteConfig(basePath); HoodieTableMetaClient metaClient = - HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(config.getBasePath()) - .setLoadActiveTimelineOnLoad(false).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))).build(); + HoodieTableMetaClient.builder() + .setConf(jsc.hadoopConfiguration()) + .setBasePath(config.getBasePath()) + .setLoadActiveTimelineOnLoad(false) + .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) + .setFileSystemGuardConfig(config.getFileSystemGuardConfig()) + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) + .build(); try { new UpgradeDowngrade(metaClient, config, new HoodieSparkEngineContext(jsc), SparkUpgradeDowngradeHelper.getInstance()) .run(HoodieTableVersion.valueOf(toVersion), null); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java index 350fe0c9bf7e0..26701a061cce9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java @@ -132,9 +132,14 @@ protected void initWrapperFSMetrics() { } protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) { - return HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(config.getBasePath()) - .setLoadActiveTimelineOnLoad(loadActiveTimelineOnLoad).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))).build(); + return HoodieTableMetaClient.builder() + .setConf(hadoopConf) + .setBasePath(config.getBasePath()) + .setLoadActiveTimelineOnLoad(loadActiveTimelineOnLoad) + .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) + .setFileSystemGuardConfig(config.getFileSystemGuardConfig()) + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) + .build(); } public Option getTimelineServer() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index c9d8c4f117eaf..d7f5cd0d04b79 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; +import org.apache.hudi.common.fs.FileSystemGuardConfig; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; @@ -426,6 +427,7 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("File Id Prefix provider class, that implements `org.apache.hudi.fileid.FileIdPrefixProvider`"); private ConsistencyGuardConfig consistencyGuardConfig; + private FileSystemGuardConfig fileSystemGuardConfig; // Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled // We keep track of original config and rewritten config @@ -829,6 +831,7 @@ protected HoodieWriteConfig(EngineType engineType, Properties props) { newProps.putAll(props); this.engineType = engineType; this.consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().fromProperties(newProps).build(); + this.fileSystemGuardConfig = FileSystemGuardConfig.newBuilder().fromProperties(newProps).build(); this.clientSpecifiedViewStorageConfig = FileSystemViewStorageConfig.newBuilder().fromProperties(newProps).build(); this.viewStorageConfig = clientSpecifiedViewStorageConfig; this.hoodiePayloadConfig = HoodiePayloadConfig.newBuilder().fromProperties(newProps).build(); @@ -1581,6 +1584,10 @@ public ConsistencyGuardConfig getConsistencyGuardConfig() { return consistencyGuardConfig; } + public FileSystemGuardConfig getFileSystemGuardConfig() { + return fileSystemGuardConfig; + } + public void setConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) { this.consistencyGuardConfig = consistencyGuardConfig; } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index e3e732b473d4f..6b866347eb899 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -63,9 +63,14 @@ public static HoodieSparkTable create(HoodieW public static HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context, boolean refreshTimeline) { HoodieTableMetaClient metaClient = - HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) - .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))).build(); + HoodieTableMetaClient.builder() + .setConf(context.getHadoopConf().get()) + .setBasePath(config.getBasePath()) + .setLoadActiveTimelineOnLoad(true) + .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) + .setFileSystemGuardConfig(config.getFileSystemGuardConfig()) + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) + .build(); return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient, refreshTimeline); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java new file mode 100644 index 0000000000000..28bc9058a6168 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java @@ -0,0 +1,135 @@ +/* + * 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.fs; + +import org.apache.hudi.common.config.ConfigClassProperty; +import org.apache.hudi.common.config.ConfigGroups; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.common.config.HoodieConfig; + +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.Properties; + +/** + * The consistency guard relevant config options. + */ +@ConfigClassProperty(name = "FileSystem Guard Configurations", + groupName = ConfigGroups.Names.WRITE_CLIENT, + description = "The filesystem guard related config options, to help deal with runtime exception like s3 list/get/put/delete performance issues.") +public class FileSystemGuardConfig extends HoodieConfig { + + public static final ConfigProperty FILESYSTEM_RETRY_ENABLE = ConfigProperty + .key("hoodie.filesystem.action.retry.enabled") + .defaultValue("false") + .sinceVersion("0.10.0") + .withDocumentation("Enabled to handle S3 list/get/delete etc performance issue."); + + public static final ConfigProperty INITIAL_RETRY_INTERVAL_MS = ConfigProperty + .key("hoodie.filesystem.action.retry.initial_interval_ms") + .defaultValue(100L) + .sinceVersion("0.10.0") + .withDocumentation("Amount of time (in ms) to wait, before retry to do operations on storage."); + + public static final ConfigProperty MAX_RETRY_INTERVAL_MS = ConfigProperty + .key("hoodie.filesystem.action.retry.max_interval_ms") + .defaultValue(2000L) + .sinceVersion("0.10.0") + .withDocumentation("Maximum amount of time (in ms), to wait for next retry."); + + public static final ConfigProperty MAX_RETRY_NUMBERS = ConfigProperty + .key("hoodie.filesystem.action.retry.max_numbers") + .defaultValue(4) + .sinceVersion("0.10.0") + .withDocumentation("Maximum number of retry actions to perform, with exponential backoff."); + + private FileSystemGuardConfig() { + super(); + } + + + public long getInitialRetryIntervalMs() { + return getLong(INITIAL_RETRY_INTERVAL_MS); + } + + public long getMaxRetryIntervalMs() { + return getLong(MAX_RETRY_INTERVAL_MS); + } + + + public int getMaxRetryNumbers() { + return getInt(MAX_RETRY_NUMBERS); + } + + public boolean isFileSystemActionRetryEnable() { + return Boolean.parseBoolean(getStringOrDefault(FILESYSTEM_RETRY_ENABLE)); + } + + public static FileSystemGuardConfig.Builder newBuilder() { + return new Builder(); + } + + /** + * The builder used to build filesystem configurations. + */ + public static class Builder { + + private final FileSystemGuardConfig fileSystemGuardConfig = new FileSystemGuardConfig(); + + public Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + fileSystemGuardConfig.getProps().load(reader); + return this; + } + } + + public Builder fromProperties(Properties props) { + this.fileSystemGuardConfig.getProps().putAll(props); + return this; + } + + public Builder withMaxRetryNumbers(int numbers) { + fileSystemGuardConfig.setValue(MAX_RETRY_NUMBERS, String.valueOf(numbers)); + return this; + } + + + public Builder withInitialRetryIntervalMs(long intervalMs) { + fileSystemGuardConfig.setValue(INITIAL_RETRY_INTERVAL_MS, String.valueOf(intervalMs)); + return this; + } + + public Builder withMaxRetryIntervalMs(long intervalMs) { + fileSystemGuardConfig.setValue(MAX_RETRY_INTERVAL_MS, String.valueOf(intervalMs)); + return this; + } + + + public Builder withFileSystemActionRetryEnabled(boolean enabled) { + fileSystemGuardConfig.setValue(FILESYSTEM_RETRY_ENABLE, String.valueOf(enabled)); + return this; + } + + public FileSystemGuardConfig build() { + fileSystemGuardConfig.setDefaults(FileSystemGuardConfig.class.getName()); + return fileSystemGuardConfig; + } + } + +} 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 305f1a4d1146e..f68e23da4147a 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 @@ -1,3 +1,20 @@ +/* + * 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.fs; import org.apache.hadoop.fs.FSDataInputStream; @@ -30,32 +47,38 @@ public URI getUri() { @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException { - return fileSystem.open(f, bufferSize); + return (FSDataInputStream) retryHelper.tryWith(() -> fileSystem.open(f, bufferSize)).start(); } @Override - public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { - return fileSystem.create(f, permission, overwrite, bufferSize, replication, blockSize, progress); + public FSDataOutputStream create(Path f, + FsPermission permission, + boolean overwrite, + int bufferSize, + short replication, + long blockSize, + Progressable progress) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, overwrite, bufferSize, replication, blockSize, progress)).start(); } @Override public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException { - return fileSystem.append(f, bufferSize, progress); + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f, bufferSize, progress)).start(); } @Override public boolean rename(Path src, Path dst) throws IOException { - return fileSystem.rename(src, dst); + return (boolean) retryHelper.tryWith(() -> fileSystem.rename(src, dst)).start(); } @Override public boolean delete(Path f, boolean recursive) throws IOException { - return fileSystem.delete(f, recursive); + return (boolean) retryHelper.tryWith(() -> fileSystem.delete(f, recursive)).start(); } @Override public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { - return fileSystem.listStatus(f); + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(f)).start(); } @Override @@ -70,11 +93,11 @@ public Path getWorkingDirectory() { @Override public boolean mkdirs(Path f, FsPermission permission) throws IOException { - return fileSystem.mkdirs(f, permission); + return (boolean) retryHelper.tryWith(() -> fileSystem.mkdirs(f, permission)).start(); } @Override public FileStatus getFileStatus(Path f) throws IOException { - return fileSystem.getFileStatus(f); + return (FileStatus) retryHelper.tryWith(() -> fileSystem.getFileStatus(f)).start(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java index f19e301d186dd..f938f113f20f8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java @@ -127,12 +127,6 @@ public HoodieWrapperFileSystem(FileSystem fileSystem, ConsistencyGuard consisten this.consistencyGuard = consistencyGuard; } - public HoodieWrapperFileSystem(FileSystem fileSystem, ConsistencyGuard consistencyGuard, RetryHelper retryHelper) { - this.fileSystem = new HoodieRetryWrapperFileSystem(fileSystem, retryHelper); - this.uri = fileSystem.getUri(); - this.consistencyGuard = consistencyGuard; - } - public static Path convertToHoodiePath(Path file, Configuration conf) { try { String scheme = FSUtils.getFs(file.toString(), conf).getScheme(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 2b94d7ff072a5..99c6355be2e61 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -23,6 +23,8 @@ import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.FailSafeConsistencyGuard; +import org.apache.hudi.common.fs.FileSystemGuardConfig; +import org.apache.hudi.common.fs.HoodieRetryWrapperFileSystem; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.fs.NoOpConsistencyGuard; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -36,6 +38,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.RetryHelper; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; @@ -97,12 +100,14 @@ public class HoodieTableMetaClient implements Serializable { private HoodieActiveTimeline activeTimeline; private HoodieArchivedTimeline archivedTimeline; private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); + private FileSystemGuardConfig fileSystemGuardConfig = FileSystemGuardConfig.newBuilder().build(); private HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad, ConsistencyGuardConfig consistencyGuardConfig, Option layoutVersion, - String payloadClassName) { + String payloadClassName, FileSystemGuardConfig fileSystemGuardConfig) { LOG.info("Loading HoodieTableMetaClient from " + basePath); this.consistencyGuardConfig = consistencyGuardConfig; + this.fileSystemGuardConfig = fileSystemGuardConfig; this.hadoopConf = new SerializableConfiguration(conf); Path basePathDir = new Path(basePath); this.basePath = basePathDir.toString(); @@ -137,8 +142,15 @@ private HoodieTableMetaClient(Configuration conf, String basePath, boolean loadA public HoodieTableMetaClient() {} public static HoodieTableMetaClient reload(HoodieTableMetaClient oldMetaClient) { - return HoodieTableMetaClient.builder().setConf(oldMetaClient.hadoopConf.get()).setBasePath(oldMetaClient.basePath).setLoadActiveTimelineOnLoad(oldMetaClient.loadActiveTimelineOnLoad) - .setConsistencyGuardConfig(oldMetaClient.consistencyGuardConfig).setLayoutVersion(Option.of(oldMetaClient.timelineLayoutVersion)).setPayloadClassName(null).build(); + return HoodieTableMetaClient.builder() + .setConf(oldMetaClient.hadoopConf.get()) + .setBasePath(oldMetaClient.basePath) + .setLoadActiveTimelineOnLoad(oldMetaClient.loadActiveTimelineOnLoad) + .setConsistencyGuardConfig(oldMetaClient.consistencyGuardConfig) + .setFileSystemGuardConfig(oldMetaClient.fileSystemGuardConfig) + .setLayoutVersion(Option.of(oldMetaClient.timelineLayoutVersion)) + .setPayloadClassName(null) + .build(); } /** @@ -245,7 +257,16 @@ public TimelineLayoutVersion getTimelineLayoutVersion() { */ public HoodieWrapperFileSystem getFs() { if (fs == null) { - FileSystem fileSystem = FSUtils.getFs(metaPath, hadoopConf.newCopy()); + FileSystem fileSystem; + if (fileSystemGuardConfig.isFileSystemActionRetryEnable()) { + RetryHelper retryHelper = new RetryHelper<>() + .tryMaxInterval(fileSystemGuardConfig.getMaxRetryIntervalMs()) + .tryNum(fileSystemGuardConfig.getMaxRetryNumbers()) + .tryInitialInterval(fileSystemGuardConfig.getInitialRetryIntervalMs()); + fileSystem = new HoodieRetryWrapperFileSystem(FSUtils.getFs(metaPath, hadoopConf.newCopy()), retryHelper); + } else { + fileSystem = FSUtils.getFs(metaPath, hadoopConf.newCopy()); + } ValidationUtils.checkArgument(!(fileSystem instanceof HoodieWrapperFileSystem), "File System not expected to be that of HoodieWrapperFileSystem"); fs = new HoodieWrapperFileSystem(fileSystem, @@ -568,6 +589,7 @@ public static class Builder { private boolean loadActiveTimelineOnLoad = false; private String payloadClassName = null; private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); + private FileSystemGuardConfig fileSystemGuardConfig = FileSystemGuardConfig.newBuilder().build(); private Option layoutVersion = Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION); public Builder setConf(Configuration conf) { @@ -595,6 +617,11 @@ public Builder setConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuard return this; } + public Builder setFileSystemGuardConfig(FileSystemGuardConfig fileSystemGuardConfig) { + this.fileSystemGuardConfig = fileSystemGuardConfig; + return this; + } + public Builder setLayoutVersion(Option layoutVersion) { this.layoutVersion = layoutVersion; return this; @@ -604,7 +631,7 @@ public HoodieTableMetaClient build() { ValidationUtils.checkArgument(conf != null, "Configuration needs to be set to init HoodieTableMetaClient"); ValidationUtils.checkArgument(basePath != null, "basePath needs to be set to init HoodieTableMetaClient"); return new HoodieTableMetaClient(conf, basePath, - loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName); + loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, fileSystemGuardConfig); } } 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 675c3d6f61d02..8ee7f10582744 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 @@ -22,15 +22,19 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.IOException; import java.util.Random; -import java.util.function.Supplier; public class RetryHelper { private static final Logger LOG = LogManager.getLogger(RetryHelper.class); private HoodieWrapperFileSystem.CheckedFunction func; private int num; private long maxIntervalTime; - private String taskInfo; + private long initialIntervalTime = 100L; + private String taskInfo = "N/A"; + + public RetryHelper() { + } public RetryHelper(String taskInfo) { this.taskInfo = taskInfo; @@ -46,12 +50,22 @@ public RetryHelper tryNum(int num) { return this; } - public RetryHelper tryInterval(long time) { + public RetryHelper tryTaskInfo(String taskInfo) { + this.taskInfo = taskInfo; + return this; + } + + public RetryHelper tryMaxInterval(long time) { maxIntervalTime = time; return this; } - public T start() throws Exception { + public RetryHelper tryInitialInterval(long time) { + initialIntervalTime = time; + return this; + } + + public T start() throws IOException { int retries = 0; boolean success = false; RuntimeException exception = null; @@ -89,9 +103,9 @@ public T start() throws Exception { private long getWaitTimeExp(int retryCount) { Random random = new Random(); if (0 == retryCount) { - return 100L; + return initialIntervalTime; } - return (long) Math.pow(2, retryCount) * 100L + random.nextInt(100); + return (long) Math.pow(2, retryCount) * initialIntervalTime + random.nextInt(100); } } \ No newline at end of file From b3297f13ac46e1544d1a3beb80d066e5afc7ebb7 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Thu, 28 Oct 2021 13:46:25 +0800 Subject: [PATCH 03/18] ready to test --- .../hudi/common/fs/FileSystemGuardConfig.java | 2 +- .../fs/HoodieRetryWrapperFileSystem.java | 133 ++++++++++++++++++ 2 files changed, 134 insertions(+), 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java index 28bc9058a6168..df25c296be533 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java @@ -37,7 +37,7 @@ public class FileSystemGuardConfig extends HoodieConfig { public static final ConfigProperty FILESYSTEM_RETRY_ENABLE = ConfigProperty .key("hoodie.filesystem.action.retry.enabled") - .defaultValue("false") + .defaultValue("true") .sinceVersion("0.10.0") .withDocumentation("Enabled to handle S3 list/get/delete etc performance issue."); 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 f68e23da4147a..baa9afab39fee 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 @@ -17,11 +17,17 @@ */ package org.apache.hudi.common.fs; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.util.Progressable; import org.apache.hudi.common.util.RetryHelper; @@ -29,6 +35,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; +import java.util.EnumSet; public class HoodieRetryWrapperFileSystem extends FileSystem { @@ -50,6 +57,11 @@ public FSDataInputStream open(Path f, int bufferSize) throws IOException { return (FSDataInputStream) retryHelper.tryWith(() -> fileSystem.open(f, bufferSize)).start(); } + @Override + public FSDataInputStream open(Path f) throws IOException { + return (FSDataInputStream) retryHelper.tryWith(() -> fileSystem.open(f)).start(); + } + @Override public FSDataOutputStream create(Path f, FsPermission permission, @@ -61,11 +73,87 @@ public FSDataOutputStream create(Path f, return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, overwrite, bufferSize, replication, blockSize, progress)).start(); } + @Override + public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, overwrite)).start(); + } + + @Override + public FSDataOutputStream create(Path f) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f)).start(); + } + + @Override + public FSDataOutputStream create(Path f, Progressable progress) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, progress)).start(); + } + + @Override + public FSDataOutputStream create(Path f, short replication) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, replication)).start(); + } + + @Override + public FSDataOutputStream create(Path f, short replication, Progressable progress) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, replication, progress)).start(); + } + + @Override + public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize)).start(); + } + + @Override + public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress) + throws IOException { + return (FSDataOutputStream) retryHelper.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) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize, replication, blockSize, progress)).start(); + } + + @Override + public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, + short replication, long blockSize, Progressable progress) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, replication, blockSize, progress)).start(); + } + + @Override + public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, + short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, replication, + blockSize, progress, checksumOpt)).start(); + } + + @Override + public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize) + throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize, replication, blockSize)).start(); + } + + @Override + public boolean createNewFile(Path f) throws IOException { + return (boolean) retryHelper.tryWith(() -> fileSystem.createNewFile(f)).start(); + } + @Override public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException { return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f, bufferSize, progress)).start(); } + @Override + public FSDataOutputStream append(Path f) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f)).start(); + } + + @Override + public FSDataOutputStream append(Path f, int bufferSize) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f, bufferSize)).start(); + } + @Override public boolean rename(Path src, Path dst) throws IOException { return (boolean) retryHelper.tryWith(() -> fileSystem.rename(src, dst)).start(); @@ -76,11 +164,51 @@ public boolean delete(Path f, boolean recursive) throws IOException { return (boolean) retryHelper.tryWith(() -> fileSystem.delete(f, recursive)).start(); } + @Override + public boolean delete(Path f) throws IOException { + return (boolean) retryHelper.tryWith(() -> fileSystem.delete(f, true)).start(); + } + @Override public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(f)).start(); } + @Override + public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(f, filter)).start(); + } + + @Override + public FileStatus[] listStatus(Path[] files) throws IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(files)).start(); + } + + @Override + public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(files, filter)).start(); + } + + @Override + public FileStatus[] globStatus(Path pathPattern) throws IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.globStatus(pathPattern)).start(); + } + + @Override + public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.globStatus(pathPattern, filter)).start(); + } + + @Override + public RemoteIterator listLocatedStatus(Path f) throws IOException { + return (RemoteIterator) retryHelper.tryWith(() -> fileSystem.listLocatedStatus(f)).start(); + } + + @Override + public RemoteIterator listFiles(Path f, boolean recursive) throws IOException { + return (RemoteIterator) retryHelper.tryWith(() -> fileSystem.listFiles(f, recursive)).start(); + } + @Override public void setWorkingDirectory(Path new_dir) { fileSystem.setWorkingDirectory(new_dir); @@ -100,4 +228,9 @@ public boolean mkdirs(Path f, FsPermission permission) throws IOException { public FileStatus getFileStatus(Path f) throws IOException { return (FileStatus) retryHelper.tryWith(() -> fileSystem.getFileStatus(f)).start(); } + + @Override + public Configuration getConf() { + return fileSystem.getConf(); + } } From 99a04ae02f6da7bed79e0db8bcce087ff6a6c108 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 29 Oct 2021 10:05:04 +0800 Subject: [PATCH 04/18] done --- .../hudi/common/fs/FileSystemGuardConfig.java | 4 +- .../common/table/HoodieTableMetaClient.java | 4 + .../apache/hudi/common/util/RetryHelper.java | 15 +- .../apache/hudi/common/fs/TestFSUtils.java | 2 +- .../fs/TestFSUtilsWithRetryWrapperEnable.java | 204 ++++++++++++++++++ 5 files changed, 218 insertions(+), 11 deletions(-) create mode 100644 hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java index df25c296be533..ac0d70c176c61 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java @@ -37,9 +37,9 @@ public class FileSystemGuardConfig extends HoodieConfig { public static final ConfigProperty FILESYSTEM_RETRY_ENABLE = ConfigProperty .key("hoodie.filesystem.action.retry.enabled") - .defaultValue("true") + .defaultValue("false") .sinceVersion("0.10.0") - .withDocumentation("Enabled to handle S3 list/get/delete etc performance issue."); + .withDocumentation("Enabled to handle S3 list/get/delete etc file system performance issue."); public static final ConfigProperty INITIAL_RETRY_INTERVAL_MS = ConfigProperty .key("hoodie.filesystem.action.retry.initial_interval_ms") diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 99c6355be2e61..12384a658d048 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -277,6 +277,10 @@ public HoodieWrapperFileSystem getFs() { return fs; } + public void setFs(HoodieWrapperFileSystem fs) { + this.fs = fs; + } + /** * Return raw file-system. * 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 8ee7f10582744..bbed4fb90d07c 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 @@ -18,7 +18,6 @@ package org.apache.hudi.common.util; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; -import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -76,13 +75,10 @@ public T start() throws IOException { t = func.get(); success = true; break; - } catch (HoodieException hoodieException) { - // ignore hoodieException here - throw hoodieException; } catch (RuntimeException e) { - // deal with other RuntimeExceptions such like AmazonS3Exception 503 + // deal with RuntimeExceptions such like AmazonS3Exception 503 exception = e; - LOG.warn("Catch RuntimeException" + taskInfo + ", will retry after " + waitTime + " ms.", e); + LOG.warn("Catch RuntimeException " + taskInfo + ", will retry after " + waitTime + " ms.", e); try { Thread.sleep(waitTime); } catch (InterruptedException ex) { @@ -93,10 +89,13 @@ public T start() throws IOException { } while (retries <= num); if (!success) { - LOG.error("Still failed to " + taskInfo + " after retried " + num + "times.", exception); + LOG.error("Still failed to " + taskInfo + " after retried " + num + " times.", exception); throw exception; } - LOG.info("Success to " + taskInfo + " after retried " + retries + " times." ); + + if (retries > 0) { + LOG.info("Success to " + taskInfo + " after retried " + retries + " times." ); + } return t; } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java index ef8b09b51e440..427e522aa8e99 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java @@ -67,7 +67,7 @@ public class TestFSUtils extends HoodieCommonTestHarness { private final long minCleanToKeep = 10; private static String TEST_WRITE_TOKEN = "1-0-1"; - private static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension(); + public static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension(); @Rule public final EnvironmentVariables environmentVariables = new EnvironmentVariables(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java new file mode 100644 index 0000000000000..422805fa2a332 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java @@ -0,0 +1,204 @@ +/* + * 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 loop.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-loop.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.fs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.util.Progressable; +import org.apache.hudi.common.util.RetryHelper; +import org.junit.Assert; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; +import java.util.Arrays; +import java.util.List; + +/** + * Tests file system utils with retry wrapper enable. + */ +public class TestFSUtilsWithRetryWrapperEnable extends TestFSUtils { + + public RetryHelper retryHelper; + public static final String EXCEPTION_MESSAGE = "Fake runtime exception here."; + + @Override + @BeforeEach + public void setUp() throws IOException { + initMetaClient(); + FileSystemGuardConfig fileSystemGuardConfig = FileSystemGuardConfig.newBuilder().withFileSystemActionRetryEnabled(true).build(); + retryHelper = new RetryHelper<>() + .tryMaxInterval(fileSystemGuardConfig.getMaxRetryIntervalMs()) + .tryNum(fileSystemGuardConfig.getMaxRetryNumbers()) + .tryInitialInterval(fileSystemGuardConfig.getInitialRetryIntervalMs()); + FakeRemoteFileSystem fakeFs = new FakeRemoteFileSystem(FSUtils.getFs(metaClient.getMetaPath(), metaClient.getHadoopConf()), 1); + FileSystem fileSystem = new HoodieRetryWrapperFileSystem(fakeFs, retryHelper); + HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(fileSystem, new NoOpConsistencyGuard()); + metaClient.setFs(fs); + } + + @Test + public void testProcessFilesWithExceptions() throws Exception { + FakeRemoteFileSystem fakeFs = new FakeRemoteFileSystem(FSUtils.getFs(metaClient.getMetaPath(), metaClient.getHadoopConf()), 100); + FileSystem fileSystem = new HoodieRetryWrapperFileSystem(fakeFs, retryHelper); + HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(fileSystem, new NoOpConsistencyGuard()); + metaClient.setFs(fs); + + List folders = + Arrays.asList("2016/04/15", ".hoodie/.temp/2/2016/04/15"); + folders.forEach(f -> Assert.assertThrows(RuntimeException.class, () -> metaClient.getFs().mkdirs(new Path(new Path(basePath), f)))); + } + + /** + * Fake remote FileSystem which will throw RuntimeException something like AmazonS3Exception 503 + */ + class FakeRemoteFileSystem extends FileSystem { + + private FileSystem fs; + private int count = 1; + private int loop; + + public FakeRemoteFileSystem(FileSystem fs, int retryLoop) { + this.fs = fs; + this.loop = retryLoop; + } + + @Override + public URI getUri() { + return fs.getUri(); + } + + @Override + public FSDataInputStream open(Path f, int bufferSize) throws IOException { + if (count % loop == 0) { + count ++; + return fs.open(f, bufferSize); + } else { + count ++; + throw new RuntimeException(EXCEPTION_MESSAGE); + } + } + + @Override + public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { + if (count % loop == 0) { + count ++; + return fs.create(f, permission, overwrite, bufferSize, replication, blockSize, progress); + } else { + count ++; + throw new RuntimeException(EXCEPTION_MESSAGE); + } + } + + @Override + public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException { + if (count % loop == 0) { + count ++; + return fs.append(f, bufferSize, progress); + } else { + count ++; + throw new RuntimeException(EXCEPTION_MESSAGE); + } + } + + @Override + public boolean rename(Path src, Path dst) throws IOException { + if (count % loop == 0) { + count ++; + return fs.rename(src, dst); + } else { + count ++; + throw new RuntimeException(EXCEPTION_MESSAGE); + } + } + + @Override + public boolean delete(Path f, boolean recursive) throws IOException { + if (count % loop == 0) { + count ++; + return fs.delete(f, recursive); + } else { + count ++; + throw new RuntimeException(EXCEPTION_MESSAGE); + } + } + + @Override + public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { + if (count % loop == 0) { + count ++; + return fs.listStatus(f); + } else { + count ++; + throw new RuntimeException(EXCEPTION_MESSAGE); + } + } + + @Override + public void setWorkingDirectory(Path new_dir) { + fs.setWorkingDirectory(new_dir); + } + + @Override + public Path getWorkingDirectory() { + return fs.getWorkingDirectory(); + } + + @Override + public boolean mkdirs(Path f, FsPermission permission) throws IOException { + if (count % loop == 0) { + count ++; + return fs.mkdirs(f, permission); + } else { + count ++; + throw new RuntimeException(EXCEPTION_MESSAGE); + } + } + + @Override + public FileStatus getFileStatus(Path f) throws IOException { + if (count % loop == 0) { + count ++; + return fs.getFileStatus(f); + } else { + count ++; + throw new RuntimeException(EXCEPTION_MESSAGE); + } + } + + @Override + public RemoteIterator listLocatedStatus(Path f) throws IOException { + return fs.listLocatedStatus(f); + } + + @Override + public Configuration getConf() { + return fs.getConf(); + } + } +} From 82ec7c1e3c40af686b9a4dcc5af99ebd3671913d Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 29 Oct 2021 11:02:26 +0800 Subject: [PATCH 05/18] fix code style --- .../hudi/common/fs/FileSystemGuardConfig.java | 164 ++++---- .../fs/HoodieRetryWrapperFileSystem.java | 387 +++++++++--------- .../apache/hudi/common/util/RetryHelper.java | 139 +++---- .../fs/TestFSUtilsWithRetryWrapperEnable.java | 45 +- 4 files changed, 367 insertions(+), 368 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java index ac0d70c176c61..a74361e6dba53 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.hudi.common.fs; import org.apache.hudi.common.config.ConfigClassProperty; @@ -35,101 +36,96 @@ description = "The filesystem guard related config options, to help deal with runtime exception like s3 list/get/put/delete performance issues.") public class FileSystemGuardConfig extends HoodieConfig { - public static final ConfigProperty FILESYSTEM_RETRY_ENABLE = ConfigProperty - .key("hoodie.filesystem.action.retry.enabled") - .defaultValue("false") - .sinceVersion("0.10.0") - .withDocumentation("Enabled to handle S3 list/get/delete etc file system performance issue."); - - public static final ConfigProperty INITIAL_RETRY_INTERVAL_MS = ConfigProperty - .key("hoodie.filesystem.action.retry.initial_interval_ms") - .defaultValue(100L) - .sinceVersion("0.10.0") - .withDocumentation("Amount of time (in ms) to wait, before retry to do operations on storage."); - - public static final ConfigProperty MAX_RETRY_INTERVAL_MS = ConfigProperty - .key("hoodie.filesystem.action.retry.max_interval_ms") - .defaultValue(2000L) - .sinceVersion("0.10.0") - .withDocumentation("Maximum amount of time (in ms), to wait for next retry."); - - public static final ConfigProperty MAX_RETRY_NUMBERS = ConfigProperty - .key("hoodie.filesystem.action.retry.max_numbers") - .defaultValue(4) - .sinceVersion("0.10.0") - .withDocumentation("Maximum number of retry actions to perform, with exponential backoff."); - - private FileSystemGuardConfig() { - super(); + public static final ConfigProperty FILESYSTEM_RETRY_ENABLE = ConfigProperty + .key("hoodie.filesystem.action.retry.enabled") + .defaultValue("false") + .sinceVersion("0.10.0") + .withDocumentation("Enabled to handle S3 list/get/delete etc file system performance issue."); + + public static final ConfigProperty INITIAL_RETRY_INTERVAL_MS = ConfigProperty + .key("hoodie.filesystem.action.retry.initial_interval_ms") + .defaultValue(100L) + .sinceVersion("0.10.0") + .withDocumentation("Amount of time (in ms) to wait, before retry to do operations on storage."); + + public static final ConfigProperty MAX_RETRY_INTERVAL_MS = ConfigProperty + .key("hoodie.filesystem.action.retry.max_interval_ms") + .defaultValue(2000L) + .sinceVersion("0.10.0") + .withDocumentation("Maximum amount of time (in ms), to wait for next retry."); + + public static final ConfigProperty MAX_RETRY_NUMBERS = ConfigProperty + .key("hoodie.filesystem.action.retry.max_numbers") + .defaultValue(4) + .sinceVersion("0.10.0") + .withDocumentation("Maximum number of retry actions to perform, with exponential backoff."); + + private FileSystemGuardConfig() { + super(); + } + + public long getInitialRetryIntervalMs() { + return getLong(INITIAL_RETRY_INTERVAL_MS); + } + + public long getMaxRetryIntervalMs() { + return getLong(MAX_RETRY_INTERVAL_MS); + } + + public int getMaxRetryNumbers() { + return getInt(MAX_RETRY_NUMBERS); + } + + public boolean isFileSystemActionRetryEnable() { + return Boolean.parseBoolean(getStringOrDefault(FILESYSTEM_RETRY_ENABLE)); + } + + public static FileSystemGuardConfig.Builder newBuilder() { + return new Builder(); + } + + /** + * The builder used to build filesystem configurations. + */ + public static class Builder { + + private final FileSystemGuardConfig fileSystemGuardConfig = new FileSystemGuardConfig(); + + public Builder fromFile(File propertiesFile) throws IOException { + try (FileReader reader = new FileReader(propertiesFile)) { + fileSystemGuardConfig.getProps().load(reader); + return this; + } } - - public long getInitialRetryIntervalMs() { - return getLong(INITIAL_RETRY_INTERVAL_MS); + public Builder fromProperties(Properties props) { + this.fileSystemGuardConfig.getProps().putAll(props); + return this; } - public long getMaxRetryIntervalMs() { - return getLong(MAX_RETRY_INTERVAL_MS); + public Builder withMaxRetryNumbers(int numbers) { + fileSystemGuardConfig.setValue(MAX_RETRY_NUMBERS, String.valueOf(numbers)); + return this; } - - public int getMaxRetryNumbers() { - return getInt(MAX_RETRY_NUMBERS); + public Builder withInitialRetryIntervalMs(long intervalMs) { + fileSystemGuardConfig.setValue(INITIAL_RETRY_INTERVAL_MS, String.valueOf(intervalMs)); + return this; } - public boolean isFileSystemActionRetryEnable() { - return Boolean.parseBoolean(getStringOrDefault(FILESYSTEM_RETRY_ENABLE)); + public Builder withMaxRetryIntervalMs(long intervalMs) { + fileSystemGuardConfig.setValue(MAX_RETRY_INTERVAL_MS, String.valueOf(intervalMs)); + return this; } - public static FileSystemGuardConfig.Builder newBuilder() { - return new Builder(); + public Builder withFileSystemActionRetryEnabled(boolean enabled) { + fileSystemGuardConfig.setValue(FILESYSTEM_RETRY_ENABLE, String.valueOf(enabled)); + return this; } - /** - * The builder used to build filesystem configurations. - */ - public static class Builder { - - private final FileSystemGuardConfig fileSystemGuardConfig = new FileSystemGuardConfig(); - - public Builder fromFile(File propertiesFile) throws IOException { - try (FileReader reader = new FileReader(propertiesFile)) { - fileSystemGuardConfig.getProps().load(reader); - return this; - } - } - - public Builder fromProperties(Properties props) { - this.fileSystemGuardConfig.getProps().putAll(props); - return this; - } - - public Builder withMaxRetryNumbers(int numbers) { - fileSystemGuardConfig.setValue(MAX_RETRY_NUMBERS, String.valueOf(numbers)); - return this; - } - - - public Builder withInitialRetryIntervalMs(long intervalMs) { - fileSystemGuardConfig.setValue(INITIAL_RETRY_INTERVAL_MS, String.valueOf(intervalMs)); - return this; - } - - public Builder withMaxRetryIntervalMs(long intervalMs) { - fileSystemGuardConfig.setValue(MAX_RETRY_INTERVAL_MS, String.valueOf(intervalMs)); - return this; - } - - - public Builder withFileSystemActionRetryEnabled(boolean enabled) { - fileSystemGuardConfig.setValue(FILESYSTEM_RETRY_ENABLE, String.valueOf(enabled)); - return this; - } - - public FileSystemGuardConfig build() { - fileSystemGuardConfig.setDefaults(FileSystemGuardConfig.class.getName()); - return fileSystemGuardConfig; - } + public FileSystemGuardConfig build() { + fileSystemGuardConfig.setDefaults(FileSystemGuardConfig.class.getName()); + return fileSystemGuardConfig; } - + } } 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 baa9afab39fee..e628791ab3fc7 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 @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.hudi.common.fs; import org.apache.hadoop.conf.Configuration; @@ -39,198 +40,198 @@ public class HoodieRetryWrapperFileSystem extends FileSystem { - private FileSystem fileSystem; - private RetryHelper retryHelper; - - public HoodieRetryWrapperFileSystem(FileSystem fileSystem, RetryHelper retryHelper) { - this.fileSystem = fileSystem; - this.retryHelper = retryHelper; - } - - @Override - public URI getUri() { - return fileSystem.getUri(); - } - - @Override - public FSDataInputStream open(Path f, int bufferSize) throws IOException { - return (FSDataInputStream) retryHelper.tryWith(() -> fileSystem.open(f, bufferSize)).start(); - } - - @Override - public FSDataInputStream open(Path f) throws IOException { - return (FSDataInputStream) retryHelper.tryWith(() -> fileSystem.open(f)).start(); - } - - @Override - public FSDataOutputStream create(Path f, - FsPermission permission, - boolean overwrite, - int bufferSize, - short replication, - long blockSize, - Progressable progress) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, overwrite, bufferSize, replication, blockSize, progress)).start(); - } - - @Override - public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, overwrite)).start(); - } - - @Override - public FSDataOutputStream create(Path f) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f)).start(); - } - - @Override - public FSDataOutputStream create(Path f, Progressable progress) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, progress)).start(); - } - - @Override - public FSDataOutputStream create(Path f, short replication) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, replication)).start(); - } - - @Override - public FSDataOutputStream create(Path f, short replication, Progressable progress) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, replication, progress)).start(); - } - - @Override - public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize)).start(); - } - - @Override - public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress) - throws IOException { - return (FSDataOutputStream) retryHelper.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) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize, replication, blockSize, progress)).start(); - } - - @Override - public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, - short replication, long blockSize, Progressable progress) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, replication, blockSize, progress)).start(); - } - - @Override - public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, - short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, replication, + private FileSystem fileSystem; + private RetryHelper retryHelper; + + public HoodieRetryWrapperFileSystem(FileSystem fileSystem, RetryHelper retryHelper) { + this.fileSystem = fileSystem; + this.retryHelper = retryHelper; + } + + @Override + public URI getUri() { + return fileSystem.getUri(); + } + + @Override + public FSDataInputStream open(Path f, int bufferSize) throws IOException { + return (FSDataInputStream) retryHelper.tryWith(() -> fileSystem.open(f, bufferSize)).start(); + } + + @Override + public FSDataInputStream open(Path f) throws IOException { + return (FSDataInputStream) retryHelper.tryWith(() -> fileSystem.open(f)).start(); + } + + @Override + public FSDataOutputStream create(Path f, + FsPermission permission, + boolean overwrite, + int bufferSize, + short replication, + long blockSize, + Progressable progress) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, overwrite, bufferSize, replication, blockSize, progress)).start(); + } + + @Override + public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, overwrite)).start(); + } + + @Override + public FSDataOutputStream create(Path f) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f)).start(); + } + + @Override + public FSDataOutputStream create(Path f, Progressable progress) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, progress)).start(); + } + + @Override + public FSDataOutputStream create(Path f, short replication) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, replication)).start(); + } + + @Override + public FSDataOutputStream create(Path f, short replication, Progressable progress) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, replication, progress)).start(); + } + + @Override + public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize)).start(); + } + + @Override + public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress) + throws IOException { + return (FSDataOutputStream) retryHelper.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) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize, replication, blockSize, progress)).start(); + } + + @Override + public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, + short replication, long blockSize, Progressable progress) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, replication, blockSize, progress)).start(); + } + + @Override + public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, + short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, replication, blockSize, progress, checksumOpt)).start(); - } - - @Override - public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize) - throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize, replication, blockSize)).start(); - } - - @Override - public boolean createNewFile(Path f) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.createNewFile(f)).start(); - } - - @Override - public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f, bufferSize, progress)).start(); - } - - @Override - public FSDataOutputStream append(Path f) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f)).start(); - } - - @Override - public FSDataOutputStream append(Path f, int bufferSize) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f, bufferSize)).start(); - } - - @Override - public boolean rename(Path src, Path dst) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.rename(src, dst)).start(); - } - - @Override - public boolean delete(Path f, boolean recursive) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.delete(f, recursive)).start(); - } - - @Override - public boolean delete(Path f) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.delete(f, true)).start(); - } - - @Override - public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(f)).start(); - } - - @Override - public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(f, filter)).start(); - } - - @Override - public FileStatus[] listStatus(Path[] files) throws IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(files)).start(); - } - - @Override - public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(files, filter)).start(); - } - - @Override - public FileStatus[] globStatus(Path pathPattern) throws IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.globStatus(pathPattern)).start(); - } - - @Override - public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.globStatus(pathPattern, filter)).start(); - } - - @Override - public RemoteIterator listLocatedStatus(Path f) throws IOException { - return (RemoteIterator) retryHelper.tryWith(() -> fileSystem.listLocatedStatus(f)).start(); - } - - @Override - public RemoteIterator listFiles(Path f, boolean recursive) throws IOException { - return (RemoteIterator) retryHelper.tryWith(() -> fileSystem.listFiles(f, recursive)).start(); - } - - @Override - public void setWorkingDirectory(Path new_dir) { - fileSystem.setWorkingDirectory(new_dir); - } - - @Override - public Path getWorkingDirectory() { - return fileSystem.getWorkingDirectory(); - } - - @Override - public boolean mkdirs(Path f, FsPermission permission) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.mkdirs(f, permission)).start(); - } - - @Override - public FileStatus getFileStatus(Path f) throws IOException { - return (FileStatus) retryHelper.tryWith(() -> fileSystem.getFileStatus(f)).start(); - } - - @Override - public Configuration getConf() { - return fileSystem.getConf(); - } + } + + @Override + public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize) + throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize, replication, blockSize)).start(); + } + + @Override + public boolean createNewFile(Path f) throws IOException { + return (boolean) retryHelper.tryWith(() -> fileSystem.createNewFile(f)).start(); + } + + @Override + public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f, bufferSize, progress)).start(); + } + + @Override + public FSDataOutputStream append(Path f) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f)).start(); + } + + @Override + public FSDataOutputStream append(Path f, int bufferSize) throws IOException { + return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f, bufferSize)).start(); + } + + @Override + public boolean rename(Path src, Path dst) throws IOException { + return (boolean) retryHelper.tryWith(() -> fileSystem.rename(src, dst)).start(); + } + + @Override + public boolean delete(Path f, boolean recursive) throws IOException { + return (boolean) retryHelper.tryWith(() -> fileSystem.delete(f, recursive)).start(); + } + + @Override + public boolean delete(Path f) throws IOException { + return (boolean) retryHelper.tryWith(() -> fileSystem.delete(f, true)).start(); + } + + @Override + public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(f)).start(); + } + + @Override + public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(f, filter)).start(); + } + + @Override + public FileStatus[] listStatus(Path[] files) throws IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(files)).start(); + } + + @Override + public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(files, filter)).start(); + } + + @Override + public FileStatus[] globStatus(Path pathPattern) throws IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.globStatus(pathPattern)).start(); + } + + @Override + public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException { + return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.globStatus(pathPattern, filter)).start(); + } + + @Override + public RemoteIterator listLocatedStatus(Path f) throws IOException { + return (RemoteIterator) retryHelper.tryWith(() -> fileSystem.listLocatedStatus(f)).start(); + } + + @Override + public RemoteIterator listFiles(Path f, boolean recursive) throws IOException { + return (RemoteIterator) retryHelper.tryWith(() -> fileSystem.listFiles(f, recursive)).start(); + } + + @Override + public void setWorkingDirectory(Path newDir) { + fileSystem.setWorkingDirectory(newDir); + } + + @Override + public Path getWorkingDirectory() { + return fileSystem.getWorkingDirectory(); + } + + @Override + public boolean mkdirs(Path f, FsPermission permission) throws IOException { + return (boolean) retryHelper.tryWith(() -> fileSystem.mkdirs(f, permission)).start(); + } + + @Override + public FileStatus getFileStatus(Path f) throws IOException { + return (FileStatus) retryHelper.tryWith(() -> fileSystem.getFileStatus(f)).start(); + } + + @Override + public Configuration getConf() { + return fileSystem.getConf(); + } } 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 bbed4fb90d07c..50237afb50d8b 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 @@ -15,6 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.hudi.common.util; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; @@ -25,86 +26,86 @@ import java.util.Random; public class RetryHelper { - private static final Logger LOG = LogManager.getLogger(RetryHelper.class); - private HoodieWrapperFileSystem.CheckedFunction func; - private int num; - private long maxIntervalTime; - private long initialIntervalTime = 100L; - private String taskInfo = "N/A"; - - public RetryHelper() { - } + private static final Logger LOG = LogManager.getLogger(RetryHelper.class); + private HoodieWrapperFileSystem.CheckedFunction func; + private int num; + private long maxIntervalTime; + private long initialIntervalTime = 100L; + private String taskInfo = "N/A"; - public RetryHelper(String taskInfo) { - this.taskInfo = taskInfo; - } + public RetryHelper() { + } - public RetryHelper tryWith(HoodieWrapperFileSystem.CheckedFunction func) { - this.func = func; - return this; - } + public RetryHelper(String taskInfo) { + this.taskInfo = taskInfo; + } - public RetryHelper tryNum(int num) { - this.num = num; - return this; - } + public RetryHelper tryWith(HoodieWrapperFileSystem.CheckedFunction func) { + this.func = func; + return this; + } - public RetryHelper tryTaskInfo(String taskInfo) { - this.taskInfo = taskInfo; - return this; - } + public RetryHelper tryNum(int num) { + this.num = num; + return this; + } - public RetryHelper tryMaxInterval(long time) { - maxIntervalTime = time; - return this; - } + public RetryHelper tryTaskInfo(String taskInfo) { + this.taskInfo = taskInfo; + return this; + } - public RetryHelper tryInitialInterval(long time) { - initialIntervalTime = time; - return this; - } + public RetryHelper tryMaxInterval(long time) { + maxIntervalTime = time; + return this; + } - public T start() throws IOException { - int retries = 0; - boolean success = false; - RuntimeException exception = null; - T t = null; - do { - long waitTime = Math.min(getWaitTimeExp(retries), maxIntervalTime); - try { - t = func.get(); - success = true; - break; - } catch (RuntimeException e) { - // deal with RuntimeExceptions such like AmazonS3Exception 503 - exception = e; - LOG.warn("Catch RuntimeException " + taskInfo + ", will retry after " + waitTime + " ms.", e); - try { - Thread.sleep(waitTime); - } catch (InterruptedException ex) { - // ignore InterruptedException here - } - retries ++; - } - } while (retries <= num); - - if (!success) { - LOG.error("Still failed to " + taskInfo + " after retried " + num + " times.", exception); - throw exception; - } + public RetryHelper tryInitialInterval(long time) { + initialIntervalTime = time; + return this; + } - if (retries > 0) { - LOG.info("Success to " + taskInfo + " after retried " + retries + " times." ); + public T start() throws IOException { + int retries = 0; + boolean success = false; + RuntimeException exception = null; + T t = null; + do { + long waitTime = Math.min(getWaitTimeExp(retries), maxIntervalTime); + try { + t = func.get(); + success = true; + break; + } catch (RuntimeException e) { + // deal with RuntimeExceptions such like AmazonS3Exception 503 + exception = e; + LOG.warn("Catch RuntimeException " + taskInfo + ", will retry after " + waitTime + " ms.", e); + try { + Thread.sleep(waitTime); + } catch (InterruptedException ex) { + // ignore InterruptedException here } - return t; + retries++; + } + } while (retries <= num); + + if (!success) { + LOG.error("Still failed to " + taskInfo + " after retried " + num + " times.", exception); + throw exception; } - private long getWaitTimeExp(int retryCount) { - Random random = new Random(); - if (0 == retryCount) { - return initialIntervalTime; - } + if (retries > 0) { + LOG.info("Success to " + taskInfo + " after retried " + retries + " times."); + } + return t; + } - return (long) Math.pow(2, retryCount) * initialIntervalTime + random.nextInt(100); + private long getWaitTimeExp(int retryCount) { + Random random = new Random(); + if (0 == retryCount) { + return initialIntervalTime; } + + return (long) Math.pow(2, retryCount) * initialIntervalTime + random.nextInt(100); + } } \ No newline at end of file diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java index 422805fa2a332..2837e713c6589 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java @@ -29,7 +29,7 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.util.Progressable; import org.apache.hudi.common.util.RetryHelper; -import org.junit.Assert; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -39,6 +39,8 @@ import java.util.Arrays; import java.util.List; +import static org.junit.jupiter.api.Assertions.assertThrows; + /** * Tests file system utils with retry wrapper enable. */ @@ -68,14 +70,13 @@ public void testProcessFilesWithExceptions() throws Exception { FileSystem fileSystem = new HoodieRetryWrapperFileSystem(fakeFs, retryHelper); HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(fileSystem, new NoOpConsistencyGuard()); metaClient.setFs(fs); - List folders = Arrays.asList("2016/04/15", ".hoodie/.temp/2/2016/04/15"); - folders.forEach(f -> Assert.assertThrows(RuntimeException.class, () -> metaClient.getFs().mkdirs(new Path(new Path(basePath), f)))); + folders.forEach(f -> assertThrows(RuntimeException.class, () -> metaClient.getFs().mkdirs(new Path(new Path(basePath), f)))); } /** - * Fake remote FileSystem which will throw RuntimeException something like AmazonS3Exception 503 + * Fake remote FileSystem which will throw RuntimeException something like AmazonS3Exception 503. */ class FakeRemoteFileSystem extends FileSystem { @@ -96,10 +97,10 @@ public URI getUri() { @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException { if (count % loop == 0) { - count ++; + count++; return fs.open(f, bufferSize); } else { - count ++; + count++; throw new RuntimeException(EXCEPTION_MESSAGE); } } @@ -107,10 +108,10 @@ public FSDataInputStream open(Path f, int bufferSize) throws IOException { @Override public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { if (count % loop == 0) { - count ++; + count++; return fs.create(f, permission, overwrite, bufferSize, replication, blockSize, progress); } else { - count ++; + count++; throw new RuntimeException(EXCEPTION_MESSAGE); } } @@ -118,10 +119,10 @@ public FSDataOutputStream create(Path f, FsPermission permission, boolean overwr @Override public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException { if (count % loop == 0) { - count ++; + count++; return fs.append(f, bufferSize, progress); } else { - count ++; + count++; throw new RuntimeException(EXCEPTION_MESSAGE); } } @@ -129,10 +130,10 @@ public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) @Override public boolean rename(Path src, Path dst) throws IOException { if (count % loop == 0) { - count ++; + count++; return fs.rename(src, dst); } else { - count ++; + count++; throw new RuntimeException(EXCEPTION_MESSAGE); } } @@ -140,10 +141,10 @@ public boolean rename(Path src, Path dst) throws IOException { @Override public boolean delete(Path f, boolean recursive) throws IOException { if (count % loop == 0) { - count ++; + count++; return fs.delete(f, recursive); } else { - count ++; + count++; throw new RuntimeException(EXCEPTION_MESSAGE); } } @@ -151,17 +152,17 @@ public boolean delete(Path f, boolean recursive) throws IOException { @Override public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { if (count % loop == 0) { - count ++; + count++; return fs.listStatus(f); } else { - count ++; + count++; throw new RuntimeException(EXCEPTION_MESSAGE); } } @Override - public void setWorkingDirectory(Path new_dir) { - fs.setWorkingDirectory(new_dir); + public void setWorkingDirectory(Path newDir) { + fs.setWorkingDirectory(newDir); } @Override @@ -172,10 +173,10 @@ public Path getWorkingDirectory() { @Override public boolean mkdirs(Path f, FsPermission permission) throws IOException { if (count % loop == 0) { - count ++; + count++; return fs.mkdirs(f, permission); } else { - count ++; + count++; throw new RuntimeException(EXCEPTION_MESSAGE); } } @@ -183,10 +184,10 @@ public boolean mkdirs(Path f, FsPermission permission) throws IOException { @Override public FileStatus getFileStatus(Path f) throws IOException { if (count % loop == 0) { - count ++; + count++; return fs.getFileStatus(f); } else { - count ++; + count++; throw new RuntimeException(EXCEPTION_MESSAGE); } } From 7b6b7630351b60fb75c1e4b1767975636db42932 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 29 Oct 2021 11:10:17 +0800 Subject: [PATCH 06/18] fix code style --- .../java/org/apache/hudi/common/fs/FileSystemGuardConfig.java | 2 +- .../java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java index a74361e6dba53..d75387a37e3eb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java @@ -61,7 +61,7 @@ public class FileSystemGuardConfig extends HoodieConfig { .withDocumentation("Maximum number of retry actions to perform, with exponential backoff."); private FileSystemGuardConfig() { - super(); + super(); } public long getInitialRetryIntervalMs() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java index f938f113f20f8..1faaad5337927 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/HoodieWrapperFileSystem.java @@ -21,7 +21,6 @@ import org.apache.hudi.common.metrics.Registry; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.HoodieTimer; -import org.apache.hudi.common.util.RetryHelper; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; From fe0c868afdbc57efd8628c7380da7469e5108476 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Fri, 12 Nov 2021 11:22:38 +0800 Subject: [PATCH 07/18] set fs guard config for hoodie flink table --- .../java/org/apache/hudi/table/HoodieFlinkTable.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index f00781f8fa695..cf8c7156a517a 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -59,9 +59,14 @@ protected HoodieFlinkTable(HoodieWriteConfig config, HoodieEngineContext context public static HoodieFlinkTable create(HoodieWriteConfig config, HoodieFlinkEngineContext context) { HoodieTableMetaClient metaClient = - HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) - .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))).build(); + HoodieTableMetaClient.builder() + .setConf(context.getHadoopConf().get()) + .setBasePath(config.getBasePath()) + .setLoadActiveTimelineOnLoad(true) + .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) + .setFileSystemGuardConfig(config.getFileSystemGuardConfig()) + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) + .build(); return HoodieFlinkTable.create(config, context, metaClient); } From c36153a39b670fcd2812143cbbd2cbbf66cd8307 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 11 Dec 2021 22:36:59 +0800 Subject: [PATCH 08/18] code review --- .../apache/hudi/cli/commands/SparkMain.java | 2 +- .../hudi/client/AbstractHoodieClient.java | 2 +- .../apache/hudi/config/HoodieWriteConfig.java | 10 +++--- .../apache/hudi/table/HoodieFlinkTable.java | 2 +- .../apache/hudi/table/HoodieSparkTable.java | 2 +- ...Config.java => FileSystemRetryConfig.java} | 34 +++++++++---------- .../common/table/HoodieTableMetaClient.java | 26 +++++++------- .../apache/hudi/common/util/RetryHelper.java | 6 ++-- .../fs/TestFSUtilsWithRetryWrapperEnable.java | 8 ++--- 9 files changed, 46 insertions(+), 46 deletions(-) rename hudi-common/src/main/java/org/apache/hudi/common/fs/{FileSystemGuardConfig.java => FileSystemRetryConfig.java} (76%) diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 5160e4525b6f3..1082ac1351dff 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -459,7 +459,7 @@ protected static int upgradeOrDowngradeTable(JavaSparkContext jsc, String basePa .setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(false) .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setFileSystemGuardConfig(config.getFileSystemGuardConfig()) + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) .build(); try { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java index 26701a061cce9..53253b4baf46c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java @@ -137,7 +137,7 @@ protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoa .setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(loadActiveTimelineOnLoad) .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setFileSystemGuardConfig(config.getFileSystemGuardConfig()) + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) .build(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 6123e1020e78f..a827bc8da13ef 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -29,7 +29,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; -import org.apache.hudi.common.fs.FileSystemGuardConfig; +import org.apache.hudi.common.fs.FileSystemRetryConfig; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFileFormat; @@ -435,7 +435,7 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("File Id Prefix provider class, that implements `org.apache.hudi.fileid.FileIdPrefixProvider`"); private ConsistencyGuardConfig consistencyGuardConfig; - private FileSystemGuardConfig fileSystemGuardConfig; + private FileSystemRetryConfig fileSystemRetryConfig; // Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled // We keep track of original config and rewritten config @@ -829,7 +829,7 @@ protected HoodieWriteConfig(EngineType engineType, Properties props) { newProps.putAll(props); this.engineType = engineType; this.consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().fromProperties(newProps).build(); - this.fileSystemGuardConfig = FileSystemGuardConfig.newBuilder().fromProperties(newProps).build(); + this.fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().fromProperties(newProps).build(); this.clientSpecifiedViewStorageConfig = FileSystemViewStorageConfig.newBuilder().fromProperties(newProps).build(); this.viewStorageConfig = clientSpecifiedViewStorageConfig; this.hoodiePayloadConfig = HoodiePayloadConfig.newBuilder().fromProperties(newProps).build(); @@ -1650,8 +1650,8 @@ public ConsistencyGuardConfig getConsistencyGuardConfig() { return consistencyGuardConfig; } - public FileSystemGuardConfig getFileSystemGuardConfig() { - return fileSystemGuardConfig; + public FileSystemRetryConfig getFileSystemRetryConfig() { + return fileSystemRetryConfig; } public void setConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index 924bed9983636..33748174c5f62 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -56,7 +56,7 @@ public static HoodieFlinkTable create(HoodieW .setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true) .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setFileSystemGuardConfig(config.getFileSystemGuardConfig()) + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) .build(); return HoodieFlinkTable.create(config, context, metaClient); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index f8d8e6c60fbcf..c9cd74920b0d9 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -66,7 +66,7 @@ public static HoodieSparkTable create(HoodieW .setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true) .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setFileSystemGuardConfig(config.getFileSystemGuardConfig()) + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) .build(); return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient, refreshTimeline); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java similarity index 76% rename from hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java rename to hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java index d75387a37e3eb..3a2021c15a841 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemGuardConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java @@ -29,18 +29,18 @@ import java.util.Properties; /** - * The consistency guard relevant config options. + * The file system retry relevant config options. */ @ConfigClassProperty(name = "FileSystem Guard Configurations", groupName = ConfigGroups.Names.WRITE_CLIENT, - description = "The filesystem guard related config options, to help deal with runtime exception like s3 list/get/put/delete performance issues.") -public class FileSystemGuardConfig extends HoodieConfig { + description = "The filesystem retry related config options, to help deal with runtime exception like list/get/put/delete performance issues.") +public class FileSystemRetryConfig extends HoodieConfig { public static final ConfigProperty FILESYSTEM_RETRY_ENABLE = ConfigProperty - .key("hoodie.filesystem.action.retry.enabled") + .key("hoodie.filesystem.action.retry.enable") .defaultValue("false") .sinceVersion("0.10.0") - .withDocumentation("Enabled to handle S3 list/get/delete etc file system performance issue."); + .withDocumentation("Enabled to handle list/get/delete etc file system performance issue."); public static final ConfigProperty INITIAL_RETRY_INTERVAL_MS = ConfigProperty .key("hoodie.filesystem.action.retry.initial_interval_ms") @@ -60,7 +60,7 @@ public class FileSystemGuardConfig extends HoodieConfig { .sinceVersion("0.10.0") .withDocumentation("Maximum number of retry actions to perform, with exponential backoff."); - private FileSystemGuardConfig() { + private FileSystemRetryConfig() { super(); } @@ -80,7 +80,7 @@ public boolean isFileSystemActionRetryEnable() { return Boolean.parseBoolean(getStringOrDefault(FILESYSTEM_RETRY_ENABLE)); } - public static FileSystemGuardConfig.Builder newBuilder() { + public static FileSystemRetryConfig.Builder newBuilder() { return new Builder(); } @@ -89,43 +89,43 @@ public static FileSystemGuardConfig.Builder newBuilder() { */ public static class Builder { - private final FileSystemGuardConfig fileSystemGuardConfig = new FileSystemGuardConfig(); + private final FileSystemRetryConfig fileSystemRetryConfig = new FileSystemRetryConfig(); public Builder fromFile(File propertiesFile) throws IOException { try (FileReader reader = new FileReader(propertiesFile)) { - fileSystemGuardConfig.getProps().load(reader); + fileSystemRetryConfig.getProps().load(reader); return this; } } public Builder fromProperties(Properties props) { - this.fileSystemGuardConfig.getProps().putAll(props); + this.fileSystemRetryConfig.getProps().putAll(props); return this; } public Builder withMaxRetryNumbers(int numbers) { - fileSystemGuardConfig.setValue(MAX_RETRY_NUMBERS, String.valueOf(numbers)); + fileSystemRetryConfig.setValue(MAX_RETRY_NUMBERS, String.valueOf(numbers)); return this; } public Builder withInitialRetryIntervalMs(long intervalMs) { - fileSystemGuardConfig.setValue(INITIAL_RETRY_INTERVAL_MS, String.valueOf(intervalMs)); + fileSystemRetryConfig.setValue(INITIAL_RETRY_INTERVAL_MS, String.valueOf(intervalMs)); return this; } public Builder withMaxRetryIntervalMs(long intervalMs) { - fileSystemGuardConfig.setValue(MAX_RETRY_INTERVAL_MS, String.valueOf(intervalMs)); + fileSystemRetryConfig.setValue(MAX_RETRY_INTERVAL_MS, String.valueOf(intervalMs)); return this; } public Builder withFileSystemActionRetryEnabled(boolean enabled) { - fileSystemGuardConfig.setValue(FILESYSTEM_RETRY_ENABLE, String.valueOf(enabled)); + fileSystemRetryConfig.setValue(FILESYSTEM_RETRY_ENABLE, String.valueOf(enabled)); return this; } - public FileSystemGuardConfig build() { - fileSystemGuardConfig.setDefaults(FileSystemGuardConfig.class.getName()); - return fileSystemGuardConfig; + public FileSystemRetryConfig build() { + fileSystemRetryConfig.setDefaults(FileSystemRetryConfig.class.getName()); + return fileSystemRetryConfig; } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 8c294b05feaaa..ace048da24b15 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -23,7 +23,7 @@ import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.FailSafeConsistencyGuard; -import org.apache.hudi.common.fs.FileSystemGuardConfig; +import org.apache.hudi.common.fs.FileSystemRetryConfig; import org.apache.hudi.common.fs.HoodieRetryWrapperFileSystem; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.fs.NoOpConsistencyGuard; @@ -101,14 +101,14 @@ public class HoodieTableMetaClient implements Serializable { private HoodieActiveTimeline activeTimeline; private HoodieArchivedTimeline archivedTimeline; private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); - private FileSystemGuardConfig fileSystemGuardConfig = FileSystemGuardConfig.newBuilder().build(); + private FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().build(); private HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad, ConsistencyGuardConfig consistencyGuardConfig, Option layoutVersion, - String payloadClassName, FileSystemGuardConfig fileSystemGuardConfig) { + String payloadClassName, FileSystemRetryConfig fileSystemRetryConfig) { LOG.info("Loading HoodieTableMetaClient from " + basePath); this.consistencyGuardConfig = consistencyGuardConfig; - this.fileSystemGuardConfig = fileSystemGuardConfig; + this.fileSystemRetryConfig = fileSystemRetryConfig; this.hadoopConf = new SerializableConfiguration(conf); Path basePathDir = new Path(basePath); this.basePath = basePathDir.toString(); @@ -148,7 +148,7 @@ public static HoodieTableMetaClient reload(HoodieTableMetaClient oldMetaClient) .setBasePath(oldMetaClient.basePath) .setLoadActiveTimelineOnLoad(oldMetaClient.loadActiveTimelineOnLoad) .setConsistencyGuardConfig(oldMetaClient.consistencyGuardConfig) - .setFileSystemGuardConfig(oldMetaClient.fileSystemGuardConfig) + .setFileSystemRetryConfig(oldMetaClient.fileSystemRetryConfig) .setLayoutVersion(Option.of(oldMetaClient.timelineLayoutVersion)) .setPayloadClassName(null) .build(); @@ -266,11 +266,11 @@ public TimelineLayoutVersion getTimelineLayoutVersion() { public HoodieWrapperFileSystem getFs() { if (fs == null) { FileSystem fileSystem; - if (fileSystemGuardConfig.isFileSystemActionRetryEnable()) { + if (fileSystemRetryConfig.isFileSystemActionRetryEnable()) { RetryHelper retryHelper = new RetryHelper<>() - .tryMaxInterval(fileSystemGuardConfig.getMaxRetryIntervalMs()) - .tryNum(fileSystemGuardConfig.getMaxRetryNumbers()) - .tryInitialInterval(fileSystemGuardConfig.getInitialRetryIntervalMs()); + .tryMaxInterval(fileSystemRetryConfig.getMaxRetryIntervalMs()) + .tryNum(fileSystemRetryConfig.getMaxRetryNumbers()) + .tryInitialInterval(fileSystemRetryConfig.getInitialRetryIntervalMs()); fileSystem = new HoodieRetryWrapperFileSystem(FSUtils.getFs(metaPath, hadoopConf.newCopy()), retryHelper); } else { fileSystem = FSUtils.getFs(metaPath, hadoopConf.newCopy()); @@ -601,7 +601,7 @@ public static class Builder { private boolean loadActiveTimelineOnLoad = false; private String payloadClassName = null; private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); - private FileSystemGuardConfig fileSystemGuardConfig = FileSystemGuardConfig.newBuilder().build(); + private FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().build(); private Option layoutVersion = Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION); public Builder setConf(Configuration conf) { @@ -629,8 +629,8 @@ public Builder setConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuard return this; } - public Builder setFileSystemGuardConfig(FileSystemGuardConfig fileSystemGuardConfig) { - this.fileSystemGuardConfig = fileSystemGuardConfig; + public Builder setFileSystemRetryConfig(FileSystemRetryConfig fileSystemRetryConfig) { + this.fileSystemRetryConfig = fileSystemRetryConfig; return this; } @@ -643,7 +643,7 @@ public HoodieTableMetaClient build() { ValidationUtils.checkArgument(conf != null, "Configuration needs to be set to init HoodieTableMetaClient"); ValidationUtils.checkArgument(basePath != null, "basePath needs to be set to init HoodieTableMetaClient"); return new HoodieTableMetaClient(conf, basePath, - loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, fileSystemGuardConfig); + loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, fileSystemRetryConfig); } } 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 50237afb50d8b..29b15285af1c7 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 @@ -69,11 +69,11 @@ public T start() throws IOException { int retries = 0; boolean success = false; RuntimeException exception = null; - T t = null; + T functionResult = null; do { long waitTime = Math.min(getWaitTimeExp(retries), maxIntervalTime); try { - t = func.get(); + functionResult = func.get(); success = true; break; } catch (RuntimeException e) { @@ -97,7 +97,7 @@ public T start() throws IOException { if (retries > 0) { LOG.info("Success to " + taskInfo + " after retried " + retries + " times."); } - return t; + return functionResult; } private long getWaitTimeExp(int retryCount) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java index 2837e713c6589..3c3d32dcd3e3c 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java @@ -53,11 +53,11 @@ public class TestFSUtilsWithRetryWrapperEnable extends TestFSUtils { @BeforeEach public void setUp() throws IOException { initMetaClient(); - FileSystemGuardConfig fileSystemGuardConfig = FileSystemGuardConfig.newBuilder().withFileSystemActionRetryEnabled(true).build(); + FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().withFileSystemActionRetryEnabled(true).build(); retryHelper = new RetryHelper<>() - .tryMaxInterval(fileSystemGuardConfig.getMaxRetryIntervalMs()) - .tryNum(fileSystemGuardConfig.getMaxRetryNumbers()) - .tryInitialInterval(fileSystemGuardConfig.getInitialRetryIntervalMs()); + .tryMaxInterval(fileSystemRetryConfig.getMaxRetryIntervalMs()) + .tryNum(fileSystemRetryConfig.getMaxRetryNumbers()) + .tryInitialInterval(fileSystemRetryConfig.getInitialRetryIntervalMs()); FakeRemoteFileSystem fakeFs = new FakeRemoteFileSystem(FSUtils.getFs(metaClient.getMetaPath(), metaClient.getHadoopConf()), 1); FileSystem fileSystem = new HoodieRetryWrapperFileSystem(fakeFs, retryHelper); HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(fileSystem, new NoOpConsistencyGuard()); From e314a3c3cbe9a90b4d5f72d2b46a157985288ea1 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 11 Dec 2021 22:48:11 +0800 Subject: [PATCH 09/18] code review --- .../org/apache/hudi/common/table/HoodieTableMetaClient.java | 4 ++++ .../hudi/common/table/timeline/TestHoodieActiveTimeline.java | 1 + 2 files changed, 5 insertions(+) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index ace048da24b15..b8fa92ca692bf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -328,6 +328,10 @@ public ConsistencyGuardConfig getConsistencyGuardConfig() { return consistencyGuardConfig; } + public FileSystemRetryConfig getFileSystemRetryConfig() { + return fileSystemRetryConfig; + } + /** * Get the archived commits as a timeline. This is costly operation, as all data from the archived files are read. * This should not be used, unless for historical debugging purposes. diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 9d89c2a6b5feb..576cfd7cb0f3f 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -126,6 +126,7 @@ public void testLoadingInstantsFromFiles() throws IOException { HoodieActiveTimeline oldTimeline = new HoodieActiveTimeline( HoodieTableMetaClient.builder().setConf(metaClient.getHadoopConf()).setBasePath(metaClient.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(metaClient.getConsistencyGuardConfig()) + .setFileSystemRetryConfig(metaClient.getFileSystemRetryConfig()) .setLayoutVersion(Option.of(new TimelineLayoutVersion(VERSION_0))).build()); // Old Timeline writes both to aux and timeline folder oldTimeline.saveToCompactionRequested(instant6, Option.of(dummy)); From a38cbda09d7b7f5781701d6e984ddaad2079e0a0 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 21 Dec 2021 09:28:23 +0800 Subject: [PATCH 10/18] code revoew --- .../hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java index 3c3d32dcd3e3c..c0581beaffa94 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java @@ -43,6 +43,7 @@ /** * Tests file system utils with retry wrapper enable. + * P.S extends TestFSUtils and setUp a HoodieWrapperFileSystem for metaClient which can test all the TestFSUtils uts with RetryWrapperEnable */ public class TestFSUtilsWithRetryWrapperEnable extends TestFSUtils { @@ -58,12 +59,13 @@ public void setUp() throws IOException { .tryMaxInterval(fileSystemRetryConfig.getMaxRetryIntervalMs()) .tryNum(fileSystemRetryConfig.getMaxRetryNumbers()) .tryInitialInterval(fileSystemRetryConfig.getInitialRetryIntervalMs()); - FakeRemoteFileSystem fakeFs = new FakeRemoteFileSystem(FSUtils.getFs(metaClient.getMetaPath(), metaClient.getHadoopConf()), 1); + FakeRemoteFileSystem fakeFs = new FakeRemoteFileSystem(FSUtils.getFs(metaClient.getMetaPath(), metaClient.getHadoopConf()), 2); FileSystem fileSystem = new HoodieRetryWrapperFileSystem(fakeFs, retryHelper); HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(fileSystem, new NoOpConsistencyGuard()); metaClient.setFs(fs); } + // Test the scenario that fs keeps retrying until it fails. @Test public void testProcessFilesWithExceptions() throws Exception { FakeRemoteFileSystem fakeFs = new FakeRemoteFileSystem(FSUtils.getFs(metaClient.getMetaPath(), metaClient.getHadoopConf()), 100); From 7f7fc4b33d1602d891c9f4b370d9e9d3cab8363f Mon Sep 17 00:00:00 2001 From: yuezhang Date: Tue, 21 Dec 2021 09:30:14 +0800 Subject: [PATCH 11/18] code review --- .../org/apache/hudi/common/fs/FileSystemRetryConfig.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java index 3a2021c15a841..de0a9248b6280 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java @@ -39,25 +39,25 @@ public class FileSystemRetryConfig extends HoodieConfig { public static final ConfigProperty FILESYSTEM_RETRY_ENABLE = ConfigProperty .key("hoodie.filesystem.action.retry.enable") .defaultValue("false") - .sinceVersion("0.10.0") + .sinceVersion("0.11.0") .withDocumentation("Enabled to handle list/get/delete etc file system performance issue."); public static final ConfigProperty INITIAL_RETRY_INTERVAL_MS = ConfigProperty .key("hoodie.filesystem.action.retry.initial_interval_ms") .defaultValue(100L) - .sinceVersion("0.10.0") + .sinceVersion("0.11.0") .withDocumentation("Amount of time (in ms) to wait, before retry to do operations on storage."); public static final ConfigProperty MAX_RETRY_INTERVAL_MS = ConfigProperty .key("hoodie.filesystem.action.retry.max_interval_ms") .defaultValue(2000L) - .sinceVersion("0.10.0") + .sinceVersion("0.11.0") .withDocumentation("Maximum amount of time (in ms), to wait for next retry."); public static final ConfigProperty MAX_RETRY_NUMBERS = ConfigProperty .key("hoodie.filesystem.action.retry.max_numbers") .defaultValue(4) - .sinceVersion("0.10.0") + .sinceVersion("0.11.0") .withDocumentation("Maximum number of retry actions to perform, with exponential backoff."); private FileSystemRetryConfig() { From f4a52761592b8be6b6d67ae2339aad19705a0e5c Mon Sep 17 00:00:00 2001 From: yuezhang Date: Thu, 23 Dec 2021 13:29:31 +0800 Subject: [PATCH 12/18] code review --- .../hudi/client/AbstractHoodieClient.java | 11 +++-------- .../org/apache/hudi/table/HoodieFlinkTable.java | 11 +++-------- .../hudi/common/fs/FileSystemRetryConfig.java | 8 ++++---- .../common/table/HoodieTableMetaClient.java | 11 ++--------- .../apache/hudi/common/util/RetryHelper.java | 17 ++++++++++------- 5 files changed, 22 insertions(+), 36 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java index 53253b4baf46c..19df269e8b3f8 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java @@ -132,14 +132,9 @@ protected void initWrapperFSMetrics() { } protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) { - return HoodieTableMetaClient.builder() - .setConf(hadoopConf) - .setBasePath(config.getBasePath()) - .setLoadActiveTimelineOnLoad(loadActiveTimelineOnLoad) - .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) - .build(); + return HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(config.getBasePath()) + .setLoadActiveTimelineOnLoad(loadActiveTimelineOnLoad).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))).setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); } public Option getTimelineServer() { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index 33748174c5f62..b5833ad9e7808 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -51,14 +51,9 @@ protected HoodieFlinkTable(HoodieWriteConfig config, HoodieEngineContext context public static HoodieFlinkTable create(HoodieWriteConfig config, HoodieFlinkEngineContext context) { HoodieTableMetaClient metaClient = - HoodieTableMetaClient.builder() - .setConf(context.getHadoopConf().get()) - .setBasePath(config.getBasePath()) - .setLoadActiveTimelineOnLoad(true) - .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) - .build(); + HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) + .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))).setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); return HoodieFlinkTable.create(config, context, metaClient); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java index de0a9248b6280..4c1b096dbba9d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java @@ -37,25 +37,25 @@ public class FileSystemRetryConfig extends HoodieConfig { public static final ConfigProperty FILESYSTEM_RETRY_ENABLE = ConfigProperty - .key("hoodie.filesystem.action.retry.enable") + .key("hoodie.filesystem.operation.retry.enable") .defaultValue("false") .sinceVersion("0.11.0") .withDocumentation("Enabled to handle list/get/delete etc file system performance issue."); public static final ConfigProperty INITIAL_RETRY_INTERVAL_MS = ConfigProperty - .key("hoodie.filesystem.action.retry.initial_interval_ms") + .key("hoodie.filesystem.operation.retry.initial_interval_ms") .defaultValue(100L) .sinceVersion("0.11.0") .withDocumentation("Amount of time (in ms) to wait, before retry to do operations on storage."); public static final ConfigProperty MAX_RETRY_INTERVAL_MS = ConfigProperty - .key("hoodie.filesystem.action.retry.max_interval_ms") + .key("hoodie.filesystem.operation.retry.max_interval_ms") .defaultValue(2000L) .sinceVersion("0.11.0") .withDocumentation("Maximum amount of time (in ms), to wait for next retry."); public static final ConfigProperty MAX_RETRY_NUMBERS = ConfigProperty - .key("hoodie.filesystem.action.retry.max_numbers") + .key("hoodie.filesystem.operation.retry.max_numbers") .defaultValue(4) .sinceVersion("0.11.0") .withDocumentation("Maximum number of retry actions to perform, with exponential backoff."); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index b8fa92ca692bf..378e7f3fa80d7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -143,15 +143,8 @@ private HoodieTableMetaClient(Configuration conf, String basePath, boolean loadA public HoodieTableMetaClient() {} public static HoodieTableMetaClient reload(HoodieTableMetaClient oldMetaClient) { - return HoodieTableMetaClient.builder() - .setConf(oldMetaClient.hadoopConf.get()) - .setBasePath(oldMetaClient.basePath) - .setLoadActiveTimelineOnLoad(oldMetaClient.loadActiveTimelineOnLoad) - .setConsistencyGuardConfig(oldMetaClient.consistencyGuardConfig) - .setFileSystemRetryConfig(oldMetaClient.fileSystemRetryConfig) - .setLayoutVersion(Option.of(oldMetaClient.timelineLayoutVersion)) - .setPayloadClassName(null) - .build(); + return HoodieTableMetaClient.builder().setConf(oldMetaClient.hadoopConf.get()).setBasePath(oldMetaClient.basePath).setLoadActiveTimelineOnLoad(oldMetaClient.loadActiveTimelineOnLoad) + .setConsistencyGuardConfig(oldMetaClient.consistencyGuardConfig).setLayoutVersion(Option.of(oldMetaClient.timelineLayoutVersion)).setPayloadClassName(null).setFileSystemRetryConfig(oldMetaClient.fileSystemRetryConfig).build(); } /** 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 29b15285af1c7..0b7f3fefce576 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 @@ -18,16 +18,14 @@ package org.apache.hudi.common.util; -import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import java.io.IOException; import java.util.Random; public class RetryHelper { private static final Logger LOG = LogManager.getLogger(RetryHelper.class); - private HoodieWrapperFileSystem.CheckedFunction func; + private CheckedFunction func; private int num; private long maxIntervalTime; private long initialIntervalTime = 100L; @@ -40,7 +38,7 @@ public RetryHelper(String taskInfo) { this.taskInfo = taskInfo; } - public RetryHelper tryWith(HoodieWrapperFileSystem.CheckedFunction func) { + public RetryHelper tryWith(CheckedFunction func) { this.func = func; return this; } @@ -65,10 +63,10 @@ public RetryHelper tryInitialInterval(long time) { return this; } - public T start() throws IOException { + public T start() throws Exception { int retries = 0; boolean success = false; - RuntimeException exception = null; + Exception exception = null; T functionResult = null; do { long waitTime = Math.min(getWaitTimeExp(retries), maxIntervalTime); @@ -76,7 +74,7 @@ public T start() throws IOException { functionResult = func.get(); success = true; break; - } catch (RuntimeException e) { + } catch (Exception e) { // deal with RuntimeExceptions such like AmazonS3Exception 503 exception = e; LOG.warn("Catch RuntimeException " + taskInfo + ", will retry after " + waitTime + " ms.", e); @@ -108,4 +106,9 @@ private long getWaitTimeExp(int retryCount) { return (long) Math.pow(2, retryCount) * initialIntervalTime + random.nextInt(100); } + + @FunctionalInterface + public interface CheckedFunction { + T get() throws Exception; + } } \ No newline at end of file From 1f5392a205d4606d9e2a31ac9cd57f0f1c392aa6 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Thu, 23 Dec 2021 13:37:34 +0800 Subject: [PATCH 13/18] code review --- .../org/apache/hudi/common/util/RetryHelper.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 0b7f3fefce576..229ed32e079eb 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 @@ -21,6 +21,7 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.IOException; import java.util.Random; public class RetryHelper { @@ -63,10 +64,10 @@ public RetryHelper tryInitialInterval(long time) { return this; } - public T start() throws Exception { + public T start() throws IOException { int retries = 0; boolean success = false; - Exception exception = null; + IOException | RuntimeException exception = null; T functionResult = null; do { long waitTime = Math.min(getWaitTimeExp(retries), maxIntervalTime); @@ -74,10 +75,9 @@ public T start() throws Exception { functionResult = func.get(); success = true; break; - } catch (Exception e) { - // deal with RuntimeExceptions such like AmazonS3Exception 503 + } catch (IOException | RuntimeException e) { exception = e; - LOG.warn("Catch RuntimeException " + taskInfo + ", will retry after " + waitTime + " ms.", e); + LOG.warn("Catch Exception " + taskInfo + ", will retry after " + waitTime + " ms.", e); try { Thread.sleep(waitTime); } catch (InterruptedException ex) { @@ -109,6 +109,6 @@ private long getWaitTimeExp(int retryCount) { @FunctionalInterface public interface CheckedFunction { - T get() throws Exception; + T get() throws IOException; } } \ No newline at end of file From ae8b009239445f17a9f2447aa9c430c74c8d6c56 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Thu, 23 Dec 2021 15:35:04 +0800 Subject: [PATCH 14/18] code review --- .../common/fs/HoodieRetryWrapperFileSystem.java | 5 +++++ .../apache/hudi/common/util/RetryHelper.java | 17 ++++++----------- 2 files changed, 11 insertions(+), 11 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 e628791ab3fc7..580b519bdb19a 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 @@ -230,6 +230,11 @@ public FileStatus getFileStatus(Path f) throws IOException { return (FileStatus) retryHelper.tryWith(() -> fileSystem.getFileStatus(f)).start(); } + @Override + public boolean exists(Path f) throws IOException { + return (boolean) retryHelper.tryWith(() -> fileSystem.exists(f)).start(); + } + @Override public Configuration getConf() { return fileSystem.getConf(); 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 229ed32e079eb..b2740e3bf0311 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 @@ -66,30 +66,25 @@ public RetryHelper tryInitialInterval(long time) { public T start() throws IOException { int retries = 0; - boolean success = false; - IOException | RuntimeException exception = null; T functionResult = null; - do { + + while (true) { long waitTime = Math.min(getWaitTimeExp(retries), maxIntervalTime); try { functionResult = func.get(); - success = true; break; } catch (IOException | RuntimeException e) { - exception = e; + if (retries++ >= num) { + LOG.error("Still failed to " + taskInfo + " after retried " + num + " times.", e); + throw e; + } LOG.warn("Catch Exception " + taskInfo + ", will retry after " + waitTime + " ms.", e); try { Thread.sleep(waitTime); } catch (InterruptedException ex) { // ignore InterruptedException here } - retries++; } - } while (retries <= num); - - if (!success) { - LOG.error("Still failed to " + taskInfo + " after retried " + num + " times.", exception); - throw exception; } if (retries > 0) { From 04f93a2e6825a202642e061c831a06b4d03ccf36 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Thu, 23 Dec 2021 15:43:46 +0800 Subject: [PATCH 15/18] code review --- .../java/org/apache/hudi/cli/commands/SparkMain.java | 12 ++++-------- .../org/apache/hudi/client/AbstractHoodieClient.java | 3 ++- .../java/org/apache/hudi/table/HoodieFlinkTable.java | 3 ++- .../java/org/apache/hudi/table/HoodieSparkTable.java | 12 ++++-------- .../hudi/common/table/HoodieTableMetaClient.java | 3 ++- 5 files changed, 14 insertions(+), 19 deletions(-) diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 1082ac1351dff..16280ef429f6f 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -454,14 +454,10 @@ private static int deleteSavepoint(JavaSparkContext jsc, String savepointTime, S protected static int upgradeOrDowngradeTable(JavaSparkContext jsc, String basePath, String toVersion) { HoodieWriteConfig config = getWriteConfig(basePath); HoodieTableMetaClient metaClient = - HoodieTableMetaClient.builder() - .setConf(jsc.hadoopConfiguration()) - .setBasePath(config.getBasePath()) - .setLoadActiveTimelineOnLoad(false) - .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) - .build(); + HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(config.getBasePath()) + .setLoadActiveTimelineOnLoad(false).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); try { new UpgradeDowngrade(metaClient, config, new HoodieSparkEngineContext(jsc), SparkUpgradeDowngradeHelper.getInstance()) .run(HoodieTableVersion.valueOf(toVersion), null); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java index 19df269e8b3f8..5ef590e3cbd0e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java @@ -134,7 +134,8 @@ protected void initWrapperFSMetrics() { protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) { return HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(loadActiveTimelineOnLoad).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))).setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); } public Option getTimelineServer() { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index b5833ad9e7808..2f08a55c956fb 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -53,7 +53,8 @@ public static HoodieFlinkTable create(HoodieW HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))).setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); return HoodieFlinkTable.create(config, context, metaClient); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index c9cd74920b0d9..118438c602303 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -61,14 +61,10 @@ public static HoodieSparkTable create(HoodieW public static HoodieSparkTable create(HoodieWriteConfig config, HoodieEngineContext context, boolean refreshTimeline) { HoodieTableMetaClient metaClient = - HoodieTableMetaClient.builder() - .setConf(context.getHadoopConf().get()) - .setBasePath(config.getBasePath()) - .setLoadActiveTimelineOnLoad(true) - .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) - .build(); + HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(config.getBasePath()) + .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) + .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) + .setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient, refreshTimeline); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 378e7f3fa80d7..20e5a563d2e11 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -144,7 +144,8 @@ public HoodieTableMetaClient() {} public static HoodieTableMetaClient reload(HoodieTableMetaClient oldMetaClient) { return HoodieTableMetaClient.builder().setConf(oldMetaClient.hadoopConf.get()).setBasePath(oldMetaClient.basePath).setLoadActiveTimelineOnLoad(oldMetaClient.loadActiveTimelineOnLoad) - .setConsistencyGuardConfig(oldMetaClient.consistencyGuardConfig).setLayoutVersion(Option.of(oldMetaClient.timelineLayoutVersion)).setPayloadClassName(null).setFileSystemRetryConfig(oldMetaClient.fileSystemRetryConfig).build(); + .setConsistencyGuardConfig(oldMetaClient.consistencyGuardConfig).setLayoutVersion(Option.of(oldMetaClient.timelineLayoutVersion)).setPayloadClassName(null) + .setFileSystemRetryConfig(oldMetaClient.fileSystemRetryConfig).build(); } /** From 1139ead2daf93239741fa6e25b1dd66381fb5c9c Mon Sep 17 00:00:00 2001 From: yuezhang Date: Wed, 9 Feb 2022 16:21:24 +0800 Subject: [PATCH 16/18] code review --- .../hudi/common/fs/FileSystemRetryConfig.java | 11 +++ .../fs/HoodieRetryWrapperFileSystem.java | 87 +++++++++++-------- .../apache/hudi/common/fs/StorageSchemes.java | 59 ++++++++----- .../common/table/HoodieTableMetaClient.java | 19 ++-- .../apache/hudi/common/util/RetryHelper.java | 60 ++++++++----- .../fs/TestFSUtilsWithRetryWrapperEnable.java | 21 +++-- 6 files changed, 159 insertions(+), 98 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java index 4c1b096dbba9d..c7f99ece7e45d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FileSystemRetryConfig.java @@ -60,6 +60,13 @@ public class FileSystemRetryConfig extends HoodieConfig { .sinceVersion("0.11.0") .withDocumentation("Maximum number of retry actions to perform, with exponential backoff."); + public static final ConfigProperty RETRY_EXCEPTIONS = ConfigProperty + .key("hoodie.filesystem.operation.retry.exceptions") + .defaultValue("") + .sinceVersion("0.11.0") + .withDocumentation("The class name of the Exception that needs to be re-tryed, separated by commas. " + + "Default is empty which means retry all the IOException and RuntimeException from FileSystem"); + private FileSystemRetryConfig() { super(); } @@ -84,6 +91,10 @@ public static FileSystemRetryConfig.Builder newBuilder() { return new Builder(); } + public String getRetryExceptions() { + return getString(RETRY_EXCEPTIONS); + } + /** * The builder used to build filesystem configurations. */ 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 580b519bdb19a..075f811a42ea7 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 @@ -41,11 +41,18 @@ public class HoodieRetryWrapperFileSystem extends FileSystem { private FileSystem fileSystem; - private RetryHelper retryHelper; + private long maxRetryIntervalMs; + private int maxRetryNumbers; + private long initialRetryIntervalMs; + private String retryExceptionsList; + + public HoodieRetryWrapperFileSystem(FileSystem fs, long maxRetryIntervalMs, int maxRetryNumbers, long initialRetryIntervalMs, String retryExceptions) { + this.fileSystem = fs; + this.maxRetryIntervalMs = maxRetryIntervalMs; + this.maxRetryNumbers = maxRetryNumbers; + this.initialRetryIntervalMs = initialRetryIntervalMs; + this.retryExceptionsList = retryExceptions; - public HoodieRetryWrapperFileSystem(FileSystem fileSystem, RetryHelper retryHelper) { - this.fileSystem = fileSystem; - this.retryHelper = retryHelper; } @Override @@ -55,12 +62,12 @@ public URI getUri() { @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException { - return (FSDataInputStream) retryHelper.tryWith(() -> fileSystem.open(f, bufferSize)).start(); + return (FSDataInputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.open(f, bufferSize)).start(); } @Override public FSDataInputStream open(Path f) throws IOException { - return (FSDataInputStream) retryHelper.tryWith(() -> fileSystem.open(f)).start(); + return (FSDataInputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.open(f)).start(); } @Override @@ -71,143 +78,151 @@ public FSDataOutputStream create(Path f, short replication, long blockSize, Progressable progress) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, overwrite, bufferSize, replication, blockSize, progress)).start(); + return (FSDataOutputStream) new RetryHelper(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) retryHelper.tryWith(() -> fileSystem.create(f, overwrite)).start(); + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.create(f, overwrite)).start(); } @Override public FSDataOutputStream create(Path f) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f)).start(); + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.create(f)).start(); } @Override public FSDataOutputStream create(Path f, Progressable progress) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, progress)).start(); + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.create(f, progress)).start(); } @Override public FSDataOutputStream create(Path f, short replication) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, replication)).start(); + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.create(f, replication)).start(); } @Override public FSDataOutputStream create(Path f, short replication, Progressable progress) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, replication, progress)).start(); + return (FSDataOutputStream) new RetryHelper(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) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize)).start(); + return (FSDataOutputStream) new RetryHelper(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) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize, progress)).start(); + return (FSDataOutputStream) new RetryHelper(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) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize, replication, blockSize, progress)).start(); + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList) + .tryWith(() -> fileSystem.create(f, overwrite, bufferSize, replication, blockSize, progress)).start(); } @Override public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, replication, blockSize, progress)).start(); + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList) + .tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, replication, blockSize, progress)).start(); } @Override public FSDataOutputStream create(Path f, FsPermission permission, EnumSet flags, int bufferSize, short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, replication, + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList) + .tryWith(() -> fileSystem.create(f, permission, flags, bufferSize, replication, blockSize, progress, checksumOpt)).start(); } @Override public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.create(f, overwrite, bufferSize, replication, blockSize)).start(); + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList) + .tryWith(() -> fileSystem.create(f, overwrite, bufferSize, replication, blockSize)).start(); } @Override public boolean createNewFile(Path f) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.createNewFile(f)).start(); + return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.createNewFile(f)).start(); } @Override public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f, bufferSize, progress)).start(); + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.append(f, bufferSize, progress)).start(); } @Override public FSDataOutputStream append(Path f) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f)).start(); + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.append(f)).start(); } @Override public FSDataOutputStream append(Path f, int bufferSize) throws IOException { - return (FSDataOutputStream) retryHelper.tryWith(() -> fileSystem.append(f, bufferSize)).start(); + return (FSDataOutputStream) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.append(f, bufferSize)).start(); } @Override public boolean rename(Path src, Path dst) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.rename(src, dst)).start(); + return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.rename(src, dst)).start(); } @Override public boolean delete(Path f, boolean recursive) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.delete(f, recursive)).start(); + return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.delete(f, recursive)).start(); } @Override public boolean delete(Path f) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.delete(f, true)).start(); + return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.delete(f, true)).start(); } @Override public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(f)).start(); + return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.listStatus(f)).start(); } @Override public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(f, filter)).start(); + return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.listStatus(f, filter)).start(); } @Override public FileStatus[] listStatus(Path[] files) throws IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(files)).start(); + return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.listStatus(files)).start(); } @Override public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.listStatus(files, filter)).start(); + return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.listStatus(files, filter)).start(); } @Override public FileStatus[] globStatus(Path pathPattern) throws IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.globStatus(pathPattern)).start(); + return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.globStatus(pathPattern)).start(); } @Override public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException { - return (FileStatus[]) retryHelper.tryWith(() -> fileSystem.globStatus(pathPattern, filter)).start(); + return (FileStatus[]) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.globStatus(pathPattern, filter)).start(); } @Override public RemoteIterator listLocatedStatus(Path f) throws IOException { - return (RemoteIterator) retryHelper.tryWith(() -> fileSystem.listLocatedStatus(f)).start(); + return (RemoteIterator) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.listLocatedStatus(f)).start(); } @Override public RemoteIterator listFiles(Path f, boolean recursive) throws IOException { - return (RemoteIterator) retryHelper.tryWith(() -> fileSystem.listFiles(f, recursive)).start(); + return (RemoteIterator) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList) + .tryWith(() -> fileSystem.listFiles(f, recursive)).start(); } @Override @@ -222,17 +237,17 @@ public Path getWorkingDirectory() { @Override public boolean mkdirs(Path f, FsPermission permission) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.mkdirs(f, permission)).start(); + return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.mkdirs(f, permission)).start(); } @Override public FileStatus getFileStatus(Path f) throws IOException { - return (FileStatus) retryHelper.tryWith(() -> fileSystem.getFileStatus(f)).start(); + return (FileStatus) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.getFileStatus(f)).start(); } @Override public boolean exists(Path f) throws IOException { - return (boolean) retryHelper.tryWith(() -> fileSystem.exists(f)).start(); + return (boolean) new RetryHelper(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptionsList).tryWith(() -> fileSystem.exists(f)).start(); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java index 5dcd0b156aa32..8e2a2c24d2ca9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java @@ -25,56 +25,58 @@ */ public enum StorageSchemes { // Local filesystem - FILE("file", false), + FILE("file", false, true), // Hadoop File System - HDFS("hdfs", true), + HDFS("hdfs", true, false), // Baidu Advanced File System - AFS("afs", true), + AFS("afs", true, false), // Mapr File System - MAPRFS("maprfs", true), + MAPRFS("maprfs", true, false), // Apache Ignite FS - IGNITE("igfs", true), + IGNITE("igfs", true, false), // AWS S3 - S3A("s3a", false), S3("s3", false), + S3A("s3a", false, false), S3("s3", false, false), // Google Cloud Storage - GCS("gs", false), + GCS("gs", false, false), // Azure WASB - WASB("wasb", false), WASBS("wasbs", false), + WASB("wasb", false, false), WASBS("wasbs", false, false), // Azure ADLS - ADL("adl", false), + ADL("adl", false, false), // Azure ADLS Gen2 - ABFS("abfs", false), ABFSS("abfss", false), + ABFS("abfs", false, false), ABFSS("abfss", false, false), // Aliyun OSS - OSS("oss", false), + OSS("oss", false, false), // View FS for federated setups. If federating across cloud stores, then append support is false - VIEWFS("viewfs", true), + VIEWFS("viewfs", true, false), //ALLUXIO - ALLUXIO("alluxio", false), + ALLUXIO("alluxio", false, false), // Tencent Cloud Object Storage - COSN("cosn", false), + COSN("cosn", false, true), // Tencent Cloud HDFS - CHDFS("ofs", true), + CHDFS("ofs", true, false), // Tencent Cloud CacheFileSystem - GOOSEFS("gfs", false), + GOOSEFS("gfs", false, false), // Databricks file system - DBFS("dbfs", false), + DBFS("dbfs", false, false), // IBM Cloud Object Storage - COS("cos", false), + COS("cos", false, false), // Huawei Cloud Object Storage - OBS("obs", false), + OBS("obs", false, false), // Kingsoft Standard Storage ks3 - KS3("ks3", false), + KS3("ks3", false, false), // JuiceFileSystem - JFS("jfs", true), + JFS("jfs", true, false), // Baidu Object Storage - BOS("bos", false); + BOS("bos", false, false); private String scheme; private boolean supportsAppend; + private boolean supportsRetry; - StorageSchemes(String scheme, boolean supportsAppend) { + StorageSchemes(String scheme, boolean supportsAppend, boolean supportsRetry) { this.scheme = scheme; this.supportsAppend = supportsAppend; + this.supportsRetry = supportsRetry; } public String getScheme() { @@ -85,6 +87,10 @@ public boolean supportsAppend() { return supportsAppend; } + public boolean supportsRetry() { + return supportsRetry; + } + public static boolean isSchemeSupported(String scheme) { return Arrays.stream(values()).anyMatch(s -> s.getScheme().equals(scheme)); } @@ -95,4 +101,11 @@ public static boolean isAppendSupported(String scheme) { } return Arrays.stream(StorageSchemes.values()).anyMatch(s -> s.supportsAppend() && s.scheme.equals(scheme)); } + + public static boolean isRetrySupported(String scheme) { + if (!isSchemeSupported(scheme)) { + throw new IllegalArgumentException("Unsupported scheme :" + scheme); + } + return Arrays.stream(StorageSchemes.values()).anyMatch(s -> s.supportsRetry() && s.scheme.equals(scheme)); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index f7e075f1be5e3..8e2af86662e97 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.fs.HoodieRetryWrapperFileSystem; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.fs.NoOpConsistencyGuard; +import org.apache.hudi.common.fs.StorageSchemes; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieTimelineTimeZone; @@ -39,7 +40,6 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.RetryHelper; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; @@ -260,15 +260,14 @@ public TimelineLayoutVersion getTimelineLayoutVersion() { */ public HoodieWrapperFileSystem getFs() { if (fs == null) { - FileSystem fileSystem; - if (fileSystemRetryConfig.isFileSystemActionRetryEnable()) { - RetryHelper retryHelper = new RetryHelper<>() - .tryMaxInterval(fileSystemRetryConfig.getMaxRetryIntervalMs()) - .tryNum(fileSystemRetryConfig.getMaxRetryNumbers()) - .tryInitialInterval(fileSystemRetryConfig.getInitialRetryIntervalMs()); - fileSystem = new HoodieRetryWrapperFileSystem(FSUtils.getFs(metaPath, hadoopConf.newCopy()), retryHelper); - } else { - fileSystem = FSUtils.getFs(metaPath, hadoopConf.newCopy()); + FileSystem fileSystem = FSUtils.getFs(metaPath, hadoopConf.newCopy()); + + if (fileSystemRetryConfig.isFileSystemActionRetryEnable() && StorageSchemes.isRetrySupported(fileSystem.getScheme())) { + fileSystem = new HoodieRetryWrapperFileSystem(fileSystem, + fileSystemRetryConfig.getMaxRetryIntervalMs(), + fileSystemRetryConfig.getMaxRetryNumbers(), + fileSystemRetryConfig.getInitialRetryIntervalMs(), + fileSystemRetryConfig.getRetryExceptions()); } ValidationUtils.checkArgument(!(fileSystem instanceof HoodieWrapperFileSystem), "File System not expected to be that of HoodieWrapperFileSystem"); 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 b2740e3bf0311..067c5ee40dad7 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 @@ -22,7 +22,11 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import java.util.Random; +import java.util.stream.Collectors; public class RetryHelper { private static final Logger LOG = LogManager.getLogger(RetryHelper.class); @@ -31,10 +35,25 @@ public class RetryHelper { private long maxIntervalTime; private long initialIntervalTime = 100L; private String taskInfo = "N/A"; + private List> retryExceptionsClasses; public RetryHelper() { } + public RetryHelper(long maxRetryIntervalMs, int maxRetryNumbers, long initialRetryIntervalMs, String retryExceptions) { + this.num = maxRetryNumbers; + this.initialIntervalTime = initialRetryIntervalMs; + this.maxIntervalTime = maxRetryIntervalMs; + if (StringUtils.isNullOrEmpty(retryExceptions)) { + this.retryExceptionsClasses = new ArrayList<>(); + } else { + this.retryExceptionsClasses = Arrays.stream(retryExceptions.split(",")) + .map(exception -> (Exception) ReflectionUtils.loadClass(exception, "")) + .map(Exception::getClass) + .collect(Collectors.toList()); + } + } + public RetryHelper(String taskInfo) { this.taskInfo = taskInfo; } @@ -44,26 +63,6 @@ public RetryHelper tryWith(CheckedFunction func) { return this; } - public RetryHelper tryNum(int num) { - this.num = num; - return this; - } - - public RetryHelper tryTaskInfo(String taskInfo) { - this.taskInfo = taskInfo; - return this; - } - - public RetryHelper tryMaxInterval(long time) { - maxIntervalTime = time; - return this; - } - - public RetryHelper tryInitialInterval(long time) { - initialIntervalTime = time; - return this; - } - public T start() throws IOException { int retries = 0; T functionResult = null; @@ -74,6 +73,9 @@ public T start() throws IOException { functionResult = func.get(); break; } catch (IOException | RuntimeException e) { + if (!checkIfExceptionInRetryList(e)) { + throw e; + } if (retries++ >= num) { LOG.error("Still failed to " + taskInfo + " after retried " + num + " times.", e); throw e; @@ -93,6 +95,24 @@ public T start() throws IOException { return functionResult; } + 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 clazz : retryExceptionsClasses) { + if (clazz.isInstance(e)) { + inRetryList = true; + break; + } + } + return inRetryList; + } + private long getWaitTimeExp(int retryCount) { Random random = new Random(); if (0 == retryCount) { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java index c0581beaffa94..0b849ebec8185 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtilsWithRetryWrapperEnable.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.util.Progressable; -import org.apache.hudi.common.util.RetryHelper; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -47,20 +46,24 @@ */ public class TestFSUtilsWithRetryWrapperEnable extends TestFSUtils { - public RetryHelper retryHelper; - public static final String EXCEPTION_MESSAGE = "Fake runtime exception here."; + private static final String EXCEPTION_MESSAGE = "Fake runtime exception here."; + private long maxRetryIntervalMs; + private int maxRetryNumbers; + private long initialRetryIntervalMs; @Override @BeforeEach public void setUp() throws IOException { initMetaClient(); + basePath = "file:" + basePath; FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().withFileSystemActionRetryEnabled(true).build(); - retryHelper = new RetryHelper<>() - .tryMaxInterval(fileSystemRetryConfig.getMaxRetryIntervalMs()) - .tryNum(fileSystemRetryConfig.getMaxRetryNumbers()) - .tryInitialInterval(fileSystemRetryConfig.getInitialRetryIntervalMs()); + maxRetryIntervalMs = fileSystemRetryConfig.getMaxRetryIntervalMs(); + maxRetryNumbers = fileSystemRetryConfig.getMaxRetryNumbers(); + initialRetryIntervalMs = fileSystemRetryConfig.getInitialRetryIntervalMs(); + FakeRemoteFileSystem fakeFs = new FakeRemoteFileSystem(FSUtils.getFs(metaClient.getMetaPath(), metaClient.getHadoopConf()), 2); - FileSystem fileSystem = new HoodieRetryWrapperFileSystem(fakeFs, retryHelper); + FileSystem fileSystem = new HoodieRetryWrapperFileSystem(fakeFs, maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, ""); + HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(fileSystem, new NoOpConsistencyGuard()); metaClient.setFs(fs); } @@ -69,7 +72,7 @@ public void setUp() throws IOException { @Test public void testProcessFilesWithExceptions() throws Exception { FakeRemoteFileSystem fakeFs = new FakeRemoteFileSystem(FSUtils.getFs(metaClient.getMetaPath(), metaClient.getHadoopConf()), 100); - FileSystem fileSystem = new HoodieRetryWrapperFileSystem(fakeFs, retryHelper); + FileSystem fileSystem = new HoodieRetryWrapperFileSystem(fakeFs, maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, ""); HoodieWrapperFileSystem fs = new HoodieWrapperFileSystem(fileSystem, new NoOpConsistencyGuard()); metaClient.setFs(fs); List folders = From c83f659752b5aa3f078b3135ad45f14596e6ea04 Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 12 Feb 2022 10:16:37 +0800 Subject: [PATCH 17/18] code review --- .../apache/hudi/common/fs/StorageSchemes.java | 59 ++++++++----------- .../common/table/HoodieTableMetaClient.java | 2 +- 2 files changed, 24 insertions(+), 37 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java index 8e2a2c24d2ca9..5dcd0b156aa32 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/StorageSchemes.java @@ -25,58 +25,56 @@ */ public enum StorageSchemes { // Local filesystem - FILE("file", false, true), + FILE("file", false), // Hadoop File System - HDFS("hdfs", true, false), + HDFS("hdfs", true), // Baidu Advanced File System - AFS("afs", true, false), + AFS("afs", true), // Mapr File System - MAPRFS("maprfs", true, false), + MAPRFS("maprfs", true), // Apache Ignite FS - IGNITE("igfs", true, false), + IGNITE("igfs", true), // AWS S3 - S3A("s3a", false, false), S3("s3", false, false), + S3A("s3a", false), S3("s3", false), // Google Cloud Storage - GCS("gs", false, false), + GCS("gs", false), // Azure WASB - WASB("wasb", false, false), WASBS("wasbs", false, false), + WASB("wasb", false), WASBS("wasbs", false), // Azure ADLS - ADL("adl", false, false), + ADL("adl", false), // Azure ADLS Gen2 - ABFS("abfs", false, false), ABFSS("abfss", false, false), + ABFS("abfs", false), ABFSS("abfss", false), // Aliyun OSS - OSS("oss", false, false), + OSS("oss", false), // View FS for federated setups. If federating across cloud stores, then append support is false - VIEWFS("viewfs", true, false), + VIEWFS("viewfs", true), //ALLUXIO - ALLUXIO("alluxio", false, false), + ALLUXIO("alluxio", false), // Tencent Cloud Object Storage - COSN("cosn", false, true), + COSN("cosn", false), // Tencent Cloud HDFS - CHDFS("ofs", true, false), + CHDFS("ofs", true), // Tencent Cloud CacheFileSystem - GOOSEFS("gfs", false, false), + GOOSEFS("gfs", false), // Databricks file system - DBFS("dbfs", false, false), + DBFS("dbfs", false), // IBM Cloud Object Storage - COS("cos", false, false), + COS("cos", false), // Huawei Cloud Object Storage - OBS("obs", false, false), + OBS("obs", false), // Kingsoft Standard Storage ks3 - KS3("ks3", false, false), + KS3("ks3", false), // JuiceFileSystem - JFS("jfs", true, false), + JFS("jfs", true), // Baidu Object Storage - BOS("bos", false, false); + BOS("bos", false); private String scheme; private boolean supportsAppend; - private boolean supportsRetry; - StorageSchemes(String scheme, boolean supportsAppend, boolean supportsRetry) { + StorageSchemes(String scheme, boolean supportsAppend) { this.scheme = scheme; this.supportsAppend = supportsAppend; - this.supportsRetry = supportsRetry; } public String getScheme() { @@ -87,10 +85,6 @@ public boolean supportsAppend() { return supportsAppend; } - public boolean supportsRetry() { - return supportsRetry; - } - public static boolean isSchemeSupported(String scheme) { return Arrays.stream(values()).anyMatch(s -> s.getScheme().equals(scheme)); } @@ -101,11 +95,4 @@ public static boolean isAppendSupported(String scheme) { } return Arrays.stream(StorageSchemes.values()).anyMatch(s -> s.supportsAppend() && s.scheme.equals(scheme)); } - - public static boolean isRetrySupported(String scheme) { - if (!isSchemeSupported(scheme)) { - throw new IllegalArgumentException("Unsupported scheme :" + scheme); - } - return Arrays.stream(StorageSchemes.values()).anyMatch(s -> s.supportsRetry() && s.scheme.equals(scheme)); - } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 8e2af86662e97..af68184ead79f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -262,7 +262,7 @@ public HoodieWrapperFileSystem getFs() { if (fs == null) { FileSystem fileSystem = FSUtils.getFs(metaPath, hadoopConf.newCopy()); - if (fileSystemRetryConfig.isFileSystemActionRetryEnable() && StorageSchemes.isRetrySupported(fileSystem.getScheme())) { + if (fileSystemRetryConfig.isFileSystemActionRetryEnable()) { fileSystem = new HoodieRetryWrapperFileSystem(fileSystem, fileSystemRetryConfig.getMaxRetryIntervalMs(), fileSystemRetryConfig.getMaxRetryNumbers(), From 9d1366b79213a74fe3eb8b70ec49fa50e31b168c Mon Sep 17 00:00:00 2001 From: yuezhang Date: Sat, 12 Feb 2022 10:24:50 +0800 Subject: [PATCH 18/18] code review --- .../java/org/apache/hudi/common/table/HoodieTableMetaClient.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index af68184ead79f..31e9f4dd5eadb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.fs.HoodieRetryWrapperFileSystem; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.fs.NoOpConsistencyGuard; -import org.apache.hudi.common.fs.StorageSchemes; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieTimelineTimeZone;