Skip to content

Commit 576f81d

Browse files
nsivabalanvoonhous
authored andcommitted
Revert "[HUDI-3669] Add a remote request retry mechanism for 'Remotehoodietablefiles… (apache#5884)" (apache#6501)
This reverts commit 660177b.
1 parent 238ca27 commit 576f81d

7 files changed

Lines changed: 36 additions & 195 deletions

File tree

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -117,11 +117,6 @@ public FileSystemViewStorageConfig getRemoteFileSystemViewConfig() {
117117
.withRemoteServerHost(hostAddr)
118118
.withRemoteServerPort(serverPort)
119119
.withRemoteTimelineClientTimeoutSecs(writeConfig.getClientSpecifiedViewStorageConfig().getRemoteTimelineClientTimeoutSecs())
120-
.withRemoteTimelineClientRetry(writeConfig.getClientSpecifiedViewStorageConfig().isRemoteTimelineClientRetryEnabled())
121-
.withRemoteTimelineClientMaxRetryNumbers(writeConfig.getClientSpecifiedViewStorageConfig().getRemoteTimelineClientMaxRetryNumbers())
122-
.withRemoteTimelineInitialRetryIntervalMs(writeConfig.getClientSpecifiedViewStorageConfig().getRemoteTimelineInitialRetryIntervalMs())
123-
.withRemoteTimelineClientMaxRetryIntervalMs(writeConfig.getClientSpecifiedViewStorageConfig().getRemoteTimelineClientMaxRetryIntervalMs())
124-
.withRemoteTimelineClientRetryExceptions(writeConfig.getClientSpecifiedViewStorageConfig().getRemoteTimelineClientRetryExceptions())
125120
.build();
126121
}
127122

hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -214,7 +214,8 @@ private static RemoteHoodieTableFileSystemView createRemoteFileSystemView(Serial
214214
LOG.info("Creating remote view for basePath " + metaClient.getBasePath() + ". Server="
215215
+ viewConf.getRemoteViewServerHost() + ":" + viewConf.getRemoteViewServerPort() + ", Timeout="
216216
+ viewConf.getRemoteTimelineClientTimeoutSecs());
217-
return new RemoteHoodieTableFileSystemView(metaClient, viewConf);
217+
return new RemoteHoodieTableFileSystemView(viewConf.getRemoteViewServerHost(), viewConf.getRemoteViewServerPort(),
218+
metaClient, viewConf.getRemoteTimelineClientTimeoutSecs());
218219
}
219220

220221
public static FileSystemViewManager createViewManager(final HoodieEngineContext context,

hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java

Lines changed: 0 additions & 76 deletions
Original file line numberDiff line numberDiff line change
@@ -110,37 +110,6 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
110110
.defaultValue(5 * 60) // 5 min
111111
.withDocumentation("Timeout in seconds, to wait for API requests against a remote file system view. e.g timeline server.");
112112

113-
public static final ConfigProperty<String> REMOTE_RETRY_ENABLE = ConfigProperty
114-
.key("hoodie.filesystem.view.remote.retry.enable")
115-
.defaultValue("false")
116-
.sinceVersion("0.12.0")
117-
.withDocumentation("Whether to enable API request retry for remote file system view.");
118-
119-
public static final ConfigProperty<Integer> REMOTE_MAX_RETRY_NUMBERS = ConfigProperty
120-
.key("hoodie.filesystem.view.remote.retry.max_numbers")
121-
.defaultValue(3) // 3 times
122-
.sinceVersion("0.12.0")
123-
.withDocumentation("Maximum number of retry for API requests against a remote file system view. e.g timeline server.");
124-
125-
public static final ConfigProperty<Long> REMOTE_INITIAL_RETRY_INTERVAL_MS = ConfigProperty
126-
.key("hoodie.filesystem.view.remote.retry.initial_interval_ms")
127-
.defaultValue(100L)
128-
.sinceVersion("0.12.0")
129-
.withDocumentation("Amount of time (in ms) to wait, before retry to do operations on storage.");
130-
131-
public static final ConfigProperty<Long> REMOTE_MAX_RETRY_INTERVAL_MS = ConfigProperty
132-
.key("hoodie.filesystem.view.remote.retry.max_interval_ms")
133-
.defaultValue(2000L)
134-
.sinceVersion("0.12.0")
135-
.withDocumentation("Maximum amount of time (in ms), to wait for next retry.");
136-
137-
public static final ConfigProperty<String> RETRY_EXCEPTIONS = ConfigProperty
138-
.key("hoodie.filesystem.view.remote.retry.exceptions")
139-
.defaultValue("")
140-
.sinceVersion("0.12.0")
141-
.withDocumentation("The class name of the Exception that needs to be re-tryed, separated by commas. "
142-
+ "Default is empty which means retry all the IOException and RuntimeException from Remote Request.");
143-
144113
public static final ConfigProperty<String> REMOTE_BACKUP_VIEW_ENABLE = ConfigProperty
145114
.key("hoodie.filesystem.remote.backup.view.enable")
146115
.defaultValue("true") // Need to be disabled only for tests.
@@ -175,26 +144,6 @@ public Integer getRemoteTimelineClientTimeoutSecs() {
175144
return getInt(REMOTE_TIMEOUT_SECS);
176145
}
177146

178-
public boolean isRemoteTimelineClientRetryEnabled() {
179-
return getBoolean(REMOTE_RETRY_ENABLE);
180-
}
181-
182-
public Integer getRemoteTimelineClientMaxRetryNumbers() {
183-
return getInt(REMOTE_MAX_RETRY_NUMBERS);
184-
}
185-
186-
public Long getRemoteTimelineInitialRetryIntervalMs() {
187-
return getLong(REMOTE_INITIAL_RETRY_INTERVAL_MS);
188-
}
189-
190-
public Long getRemoteTimelineClientMaxRetryIntervalMs() {
191-
return getLong(REMOTE_MAX_RETRY_INTERVAL_MS);
192-
}
193-
194-
public String getRemoteTimelineClientRetryExceptions() {
195-
return getString(RETRY_EXCEPTIONS);
196-
}
197-
198147
public long getMaxMemoryForFileGroupMap() {
199148
long totalMemory = getLong(SPILLABLE_MEMORY);
200149
return totalMemory - getMaxMemoryForPendingCompaction() - getMaxMemoryForBootstrapBaseFile();
@@ -296,31 +245,6 @@ public Builder withRemoteTimelineClientTimeoutSecs(Integer timelineClientTimeout
296245
return this;
297246
}
298247

299-
public Builder withRemoteTimelineClientRetry(boolean enableRetry) {
300-
fileSystemViewStorageConfig.setValue(REMOTE_RETRY_ENABLE, Boolean.toString(enableRetry));
301-
return this;
302-
}
303-
304-
public Builder withRemoteTimelineClientMaxRetryNumbers(Integer maxRetryNumbers) {
305-
fileSystemViewStorageConfig.setValue(REMOTE_MAX_RETRY_NUMBERS, maxRetryNumbers.toString());
306-
return this;
307-
}
308-
309-
public Builder withRemoteTimelineInitialRetryIntervalMs(Long initialRetryIntervalMs) {
310-
fileSystemViewStorageConfig.setValue(REMOTE_INITIAL_RETRY_INTERVAL_MS, initialRetryIntervalMs.toString());
311-
return this;
312-
}
313-
314-
public Builder withRemoteTimelineClientMaxRetryIntervalMs(Long maxRetryIntervalMs) {
315-
fileSystemViewStorageConfig.setValue(REMOTE_MAX_RETRY_INTERVAL_MS, maxRetryIntervalMs.toString());
316-
return this;
317-
}
318-
319-
public Builder withRemoteTimelineClientRetryExceptions(String retryExceptions) {
320-
fileSystemViewStorageConfig.setValue(RETRY_EXCEPTIONS, retryExceptions);
321-
return this;
322-
}
323-
324248
public Builder withMemFractionForPendingCompaction(Double memFractionForPendingCompaction) {
325249
fileSystemViewStorageConfig.setValue(SPILLABLE_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
326250
return this;

hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java

Lines changed: 16 additions & 51 deletions
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,6 @@
3939
import org.apache.hudi.common.table.timeline.dto.InstantDTO;
4040
import org.apache.hudi.common.table.timeline.dto.TimelineDTO;
4141
import org.apache.hudi.common.util.Option;
42-
import org.apache.hudi.common.util.RetryHelper;
4342
import org.apache.hudi.common.util.StringUtils;
4443
import org.apache.hudi.common.util.ValidationUtils;
4544
import org.apache.hudi.common.util.collection.Pair;
@@ -133,35 +132,22 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
133132

134133
private boolean closed = false;
135134

136-
private RetryHelper<Response> retryHelper;
137-
138-
private final HttpRequestCheckedFunction urlCheckedFunc;
139-
140135
private enum RequestMethod {
141136
GET, POST
142137
}
143138

144139
public RemoteHoodieTableFileSystemView(String server, int port, HoodieTableMetaClient metaClient) {
145-
this(metaClient, FileSystemViewStorageConfig.newBuilder().withRemoteServerHost(server).withRemoteServerPort(port).build());
140+
this(server, port, metaClient, 300);
146141
}
147142

148-
public RemoteHoodieTableFileSystemView(HoodieTableMetaClient metaClient, FileSystemViewStorageConfig viewConf) {
143+
public RemoteHoodieTableFileSystemView(String server, int port, HoodieTableMetaClient metaClient, int timeoutSecs) {
149144
this.basePath = metaClient.getBasePath();
145+
this.serverHost = server;
146+
this.serverPort = port;
150147
this.mapper = new ObjectMapper();
151148
this.metaClient = metaClient;
152149
this.timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
153-
this.serverHost = viewConf.getRemoteViewServerHost();
154-
this.serverPort = viewConf.getRemoteViewServerPort();
155-
this.timeoutSecs = viewConf.getRemoteTimelineClientTimeoutSecs();
156-
this.urlCheckedFunc = new HttpRequestCheckedFunction(this.timeoutSecs * 1000);
157-
if (viewConf.isRemoteTimelineClientRetryEnabled()) {
158-
retryHelper = new RetryHelper(
159-
viewConf.getRemoteTimelineClientMaxRetryIntervalMs(),
160-
viewConf.getRemoteTimelineClientMaxRetryNumbers(),
161-
viewConf.getRemoteTimelineInitialRetryIntervalMs(),
162-
viewConf.getRemoteTimelineClientRetryExceptions(),
163-
"Sending request");
164-
}
150+
this.timeoutSecs = timeoutSecs;
165151
}
166152

167153
private <T> T executeRequest(String requestPath, Map<String, String> queryParameters, TypeReference reference,
@@ -179,9 +165,17 @@ private <T> T executeRequest(String requestPath, Map<String, String> queryParame
179165

180166
String url = builder.toString();
181167
LOG.info("Sending request : (" + url + ")");
182-
// Reset url and method, to avoid repeatedly instantiating objects.
183-
urlCheckedFunc.setUrlAndMethod(url, method);
184-
Response response = retryHelper != null ? retryHelper.tryWith(urlCheckedFunc).start() : urlCheckedFunc.get();
168+
Response response;
169+
int timeout = this.timeoutSecs * 1000; // msec
170+
switch (method) {
171+
case GET:
172+
response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
173+
break;
174+
case POST:
175+
default:
176+
response = Request.Post(url).connectTimeout(timeout).socketTimeout(timeout).execute();
177+
break;
178+
}
185179
String content = response.returnContent().asString();
186180
return (T) mapper.readValue(content, reference);
187181
}
@@ -501,33 +495,4 @@ public Option<HoodieBaseFile> getLatestBaseFile(String partitionPath, String fil
501495
throw new HoodieRemoteException(e);
502496
}
503497
}
504-
505-
/**
506-
* For remote HTTP requests, to avoid repeatedly instantiating objects.
507-
*/
508-
private class HttpRequestCheckedFunction implements RetryHelper.CheckedFunction<Response> {
509-
private String url;
510-
private RequestMethod method;
511-
private final int timeoutMs;
512-
513-
public void setUrlAndMethod(String url, RequestMethod method) {
514-
this.method = method;
515-
this.url = url;
516-
}
517-
518-
public HttpRequestCheckedFunction(int timeoutMs) {
519-
this.timeoutMs = timeoutMs;
520-
}
521-
522-
@Override
523-
public Response get() throws IOException {
524-
switch (method) {
525-
case GET:
526-
return Request.Get(url).connectTimeout(timeoutMs).socketTimeout(timeoutMs).execute();
527-
case POST:
528-
default:
529-
return Request.Post(url).connectTimeout(timeoutMs).socketTimeout(timeoutMs).execute();
530-
}
531-
}
532-
}
533498
}

hudi-common/src/main/java/org/apache/hudi/common/util/RetryHelper.java

Lines changed: 18 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -18,52 +18,47 @@
1818

1919
package org.apache.hudi.common.util;
2020

21-
import org.apache.hudi.exception.HoodieException;
2221
import org.apache.log4j.LogManager;
2322
import org.apache.log4j.Logger;
2423

2524
import java.io.IOException;
26-
import java.io.Serializable;
2725
import java.util.ArrayList;
2826
import java.util.Arrays;
2927
import java.util.List;
3028
import java.util.Random;
3129
import java.util.stream.Collectors;
3230

33-
public class RetryHelper<T> implements Serializable {
31+
public class RetryHelper<T> {
3432
private static final Logger LOG = LogManager.getLogger(RetryHelper.class);
35-
private transient CheckedFunction<T> func;
36-
private final int num;
37-
private final long maxIntervalTime;
38-
private final long initialIntervalTime;
33+
private CheckedFunction<T> func;
34+
private int num;
35+
private long maxIntervalTime;
36+
private long initialIntervalTime = 100L;
3937
private String taskInfo = "N/A";
4038
private List<? extends Class<? extends Exception>> retryExceptionsClasses;
4139

40+
public RetryHelper() {
41+
}
42+
4243
public RetryHelper(long maxRetryIntervalMs, int maxRetryNumbers, long initialRetryIntervalMs, String retryExceptions) {
4344
this.num = maxRetryNumbers;
4445
this.initialIntervalTime = initialRetryIntervalMs;
4546
this.maxIntervalTime = maxRetryIntervalMs;
4647
if (StringUtils.isNullOrEmpty(retryExceptions)) {
4748
this.retryExceptionsClasses = new ArrayList<>();
4849
} else {
49-
try {
50-
this.retryExceptionsClasses = Arrays.stream(retryExceptions.split(","))
51-
.map(exception -> (Exception) ReflectionUtils.loadClass(exception, ""))
52-
.map(Exception::getClass)
53-
.collect(Collectors.toList());
54-
} catch (HoodieException e) {
55-
LOG.error("Exception while loading retry exceptions classes '" + retryExceptions + "'.", e);
56-
this.retryExceptionsClasses = new ArrayList<>();
57-
}
50+
this.retryExceptionsClasses = Arrays.stream(retryExceptions.split(","))
51+
.map(exception -> (Exception) ReflectionUtils.loadClass(exception, ""))
52+
.map(Exception::getClass)
53+
.collect(Collectors.toList());
5854
}
5955
}
6056

61-
public RetryHelper(long maxRetryIntervalMs, int maxRetryNumbers, long initialRetryIntervalMs, String retryExceptions, String taskInfo) {
62-
this(maxRetryIntervalMs, maxRetryNumbers, initialRetryIntervalMs, retryExceptions);
57+
public RetryHelper(String taskInfo) {
6358
this.taskInfo = taskInfo;
6459
}
6560

66-
public RetryHelper<T> tryWith(CheckedFunction<T> func) {
61+
public RetryHelper tryWith(CheckedFunction<T> func) {
6762
this.func = func;
6863
return this;
6964
}
@@ -82,26 +77,21 @@ public T start() throws IOException {
8277
throw e;
8378
}
8479
if (retries++ >= num) {
85-
String message = "Still failed to " + taskInfo + " after retried " + num + " times.";
86-
LOG.error(message, e);
87-
if (e instanceof IOException) {
88-
throw new IOException(message, e);
89-
}
80+
LOG.error("Still failed to " + taskInfo + " after retried " + num + " times.", e);
9081
throw e;
9182
}
92-
LOG.warn("Catch Exception for " + taskInfo + ", will retry after " + waitTime + " ms.", e);
83+
LOG.warn("Catch Exception " + taskInfo + ", will retry after " + waitTime + " ms.", e);
9384
try {
9485
Thread.sleep(waitTime);
9586
} catch (InterruptedException ex) {
96-
// ignore InterruptedException here
87+
// ignore InterruptedException here
9788
}
9889
}
9990
}
10091

10192
if (retries > 0) {
10293
LOG.info("Success to " + taskInfo + " after retried " + retries + " times.");
10394
}
104-
10595
return functionResult;
10696
}
10797

@@ -133,7 +123,7 @@ private long getWaitTimeExp(int retryCount) {
133123
}
134124

135125
@FunctionalInterface
136-
public interface CheckedFunction<T> extends Serializable {
126+
public interface CheckedFunction<T> {
137127
T get() throws IOException;
138128
}
139129
}

hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -428,11 +428,6 @@ public static HoodieFlinkWriteClient createWriteClient(Configuration conf) throw
428428
.withRemoteServerHost(viewStorageConfig.getRemoteViewServerHost())
429429
.withRemoteServerPort(viewStorageConfig.getRemoteViewServerPort())
430430
.withRemoteTimelineClientTimeoutSecs(viewStorageConfig.getRemoteTimelineClientTimeoutSecs())
431-
.withRemoteTimelineClientRetry(viewStorageConfig.isRemoteTimelineClientRetryEnabled())
432-
.withRemoteTimelineClientMaxRetryNumbers(viewStorageConfig.getRemoteTimelineClientMaxRetryNumbers())
433-
.withRemoteTimelineInitialRetryIntervalMs(viewStorageConfig.getRemoteTimelineInitialRetryIntervalMs())
434-
.withRemoteTimelineClientMaxRetryIntervalMs(viewStorageConfig.getRemoteTimelineClientMaxRetryIntervalMs())
435-
.withRemoteTimelineClientRetryExceptions(viewStorageConfig.getRemoteTimelineClientRetryExceptions())
436431
.build();
437432
ViewStorageProperties.createProperties(conf.getString(FlinkOptions.PATH), rebuilt, conf);
438433
return writeClient;

hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/functional/TestRemoteHoodieTableFileSystemView.java

Lines changed: 0 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -28,14 +28,12 @@
2828
import org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView;
2929
import org.apache.hudi.common.table.view.SyncableFileSystemView;
3030
import org.apache.hudi.common.table.view.TestHoodieTableFileSystemView;
31-
import org.apache.hudi.exception.HoodieRemoteException;
3231
import org.apache.hudi.timeline.service.TimelineService;
3332

3433
import org.apache.hadoop.conf.Configuration;
3534
import org.apache.hadoop.fs.FileSystem;
3635
import org.apache.log4j.LogManager;
3736
import org.apache.log4j.Logger;
38-
import org.junit.jupiter.api.Test;
3937

4038
/**
4139
* Bring up a remote Timeline Server and run all test-cases of TestHoodieTableFileSystemView against it.
@@ -66,31 +64,4 @@ protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) {
6664
view = new RemoteHoodieTableFileSystemView("localhost", server.getServerPort(), metaClient);
6765
return view;
6866
}
69-
70-
@Test
71-
public void testRemoteHoodieTableFileSystemViewWithRetry() {
72-
// Service is available.
73-
view.getLatestBaseFiles();
74-
// Shut down the service.
75-
server.close();
76-
try {
77-
// Immediately fails and throws a connection refused exception.
78-
view.getLatestBaseFiles();
79-
} catch (HoodieRemoteException e) {
80-
assert e.getMessage().contains("Connection refused (Connection refused)");
81-
}
82-
// Enable API request retry for remote file system view.
83-
view = new RemoteHoodieTableFileSystemView(metaClient, FileSystemViewStorageConfig
84-
.newBuilder()
85-
.withRemoteServerHost("localhost")
86-
.withRemoteServerPort(server.getServerPort())
87-
.withRemoteTimelineClientRetry(true)
88-
.withRemoteTimelineClientMaxRetryNumbers(4)
89-
.build());
90-
try {
91-
view.getLatestBaseFiles();
92-
} catch (HoodieRemoteException e) {
93-
assert e.getMessage().equalsIgnoreCase("Still failed to Sending request after retried 4 times.");
94-
}
95-
}
9667
}

0 commit comments

Comments
 (0)