-
Notifications
You must be signed in to change notification settings - Fork 46
HTTP-122 Retry for source lookup table #148
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
Changes from all commits
610d4b9
16ffc35
a547bde
e0c7bcc
b67567a
680723b
9485f27
853d23d
0c5097b
75eaa2d
65de8b8
9753dea
c3ed544
0a56ba0
3cf56da
b08575d
e497587
dfb1133
d9c4257
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -13,3 +13,4 @@ bin | |
/src/test/test.iml | ||
/flink-http-connector.iml | ||
/dependency-reduced-pom.xml | ||
/.java-version |
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,15 @@ | ||
package com.getindata.connectors.http; | ||
|
||
import java.net.http.HttpResponse; | ||
|
||
import lombok.Getter; | ||
|
||
@Getter | ||
public class HttpStatusCodeValidationFailedException extends Exception { | ||
private final HttpResponse<?> response; | ||
|
||
public HttpStatusCodeValidationFailedException(String message, HttpResponse<?> response) { | ||
super(message); | ||
this.response = response; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,71 @@ | ||
package com.getindata.connectors.http.internal.retry; | ||
|
||
import java.io.IOException; | ||
import java.net.http.HttpClient; | ||
import java.net.http.HttpRequest; | ||
import java.net.http.HttpResponse; | ||
import java.util.function.Supplier; | ||
|
||
import io.github.resilience4j.retry.Retry; | ||
import io.github.resilience4j.retry.RetryConfig; | ||
import lombok.Builder; | ||
import lombok.Getter; | ||
import lombok.extern.slf4j.Slf4j; | ||
import org.apache.flink.metrics.MetricGroup; | ||
|
||
import com.getindata.connectors.http.HttpStatusCodeValidationFailedException; | ||
import com.getindata.connectors.http.internal.status.HttpResponseChecker; | ||
|
||
@Slf4j | ||
public class HttpClientWithRetry { | ||
|
||
private final HttpClient httpClient; | ||
@Getter | ||
private final HttpResponseChecker responseChecker; | ||
private final Retry retry; | ||
|
||
@Builder | ||
HttpClientWithRetry(HttpClient httpClient, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am wondering what happens with OIDC, the short lived bearer token may need to be regenerated if the retries occur after the token has expired). Is this regeneration check done for the retries? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The request is created only once, but OIDC processor (responsible for setting bearer token in request) is called on every retry. |
||
RetryConfig retryConfig, | ||
HttpResponseChecker responseChecker) { | ||
this.httpClient = httpClient; | ||
this.responseChecker = responseChecker; | ||
var adjustedRetryConfig = RetryConfig.from(retryConfig) | ||
.retryExceptions(IOException.class) | ||
.retryOnResult(this::isTemporalError) | ||
.build(); | ||
this.retry = Retry.of("http-lookup-connector", adjustedRetryConfig); | ||
} | ||
|
||
public void registerMetrics(MetricGroup metrics){ | ||
var group = metrics.addGroup("http_lookup_connector"); | ||
group.gauge("successfulCallsWithRetryAttempt", | ||
() -> retry.getMetrics().getNumberOfSuccessfulCallsWithRetryAttempt()); | ||
group.gauge("successfulCallsWithoutRetryAttempt", | ||
() -> retry.getMetrics().getNumberOfSuccessfulCallsWithoutRetryAttempt()); | ||
} | ||
|
||
public <T> HttpResponse<T> send( | ||
Supplier<HttpRequest> requestSupplier, | ||
HttpResponse.BodyHandler<T> responseBodyHandler | ||
) throws IOException, InterruptedException, HttpStatusCodeValidationFailedException { | ||
try { | ||
var response = Retry.decorateCheckedSupplier(retry, | ||
() -> httpClient.send(requestSupplier.get(), responseBodyHandler)).apply(); | ||
if (!responseChecker.isSuccessful(response)) { | ||
throw new HttpStatusCodeValidationFailedException( | ||
"Incorrect response code: " + response.statusCode(), response); | ||
} | ||
return response; | ||
} catch (IOException | InterruptedException | HttpStatusCodeValidationFailedException e) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why IOException and InterruptedException are special here? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. These are checked exceptions from |
||
throw e; //re-throw without wrapping | ||
} catch (Throwable t) { | ||
throw new RuntimeException("Unexpected exception", t); | ||
} | ||
} | ||
|
||
private boolean isTemporalError(Object response) { | ||
return responseChecker.isTemporalError((HttpResponse<?>) response); | ||
} | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,58 @@ | ||
package com.getindata.connectors.http.internal.retry; | ||
|
||
import io.github.resilience4j.core.IntervalFunction; | ||
import io.github.resilience4j.retry.RetryConfig; | ||
import lombok.AccessLevel; | ||
import lombok.RequiredArgsConstructor; | ||
import org.apache.flink.configuration.ReadableConfig; | ||
import org.apache.flink.table.connector.source.lookup.LookupOptions; | ||
import static io.github.resilience4j.core.IntervalFunction.ofExponentialBackoff; | ||
|
||
import static com.getindata.connectors.http.internal.table.lookup.HttpLookupConnectorOptions.SOURCE_LOOKUP_RETRY_EXPONENTIAL_DELAY_INITIAL_BACKOFF; | ||
import static com.getindata.connectors.http.internal.table.lookup.HttpLookupConnectorOptions.SOURCE_LOOKUP_RETRY_EXPONENTIAL_DELAY_MAX_BACKOFF; | ||
import static com.getindata.connectors.http.internal.table.lookup.HttpLookupConnectorOptions.SOURCE_LOOKUP_RETRY_EXPONENTIAL_DELAY_MULTIPLIER; | ||
import static com.getindata.connectors.http.internal.table.lookup.HttpLookupConnectorOptions.SOURCE_LOOKUP_RETRY_FIXED_DELAY_DELAY; | ||
import static com.getindata.connectors.http.internal.table.lookup.HttpLookupConnectorOptions.SOURCE_LOOKUP_RETRY_STRATEGY; | ||
|
||
@RequiredArgsConstructor(access = AccessLevel.PRIVATE) | ||
public class RetryConfigProvider { | ||
|
||
private final ReadableConfig config; | ||
|
||
public static RetryConfig create(ReadableConfig config) { | ||
return new RetryConfigProvider(config).create(); | ||
} | ||
|
||
private RetryConfig create() { | ||
return createBuilder() | ||
.maxAttempts(config.get(LookupOptions.MAX_RETRIES) + 1) | ||
.build(); | ||
} | ||
|
||
private RetryConfig.Builder<?> createBuilder() { | ||
var retryStrategy = getRetryStrategy(); | ||
if (retryStrategy == RetryStrategyType.FIXED_DELAY) { | ||
return configureFixedDelay(); | ||
} else if (retryStrategy == RetryStrategyType.EXPONENTIAL_DELAY) { | ||
return configureExponentialDelay(); | ||
} | ||
throw new IllegalArgumentException("Unsupported retry strategy: " + retryStrategy); | ||
} | ||
|
||
private RetryStrategyType getRetryStrategy() { | ||
return RetryStrategyType.fromCode(config.get(SOURCE_LOOKUP_RETRY_STRATEGY)); | ||
} | ||
|
||
private RetryConfig.Builder<?> configureFixedDelay() { | ||
return RetryConfig.custom() | ||
.intervalFunction(IntervalFunction.of(config.get(SOURCE_LOOKUP_RETRY_FIXED_DELAY_DELAY))); | ||
} | ||
|
||
private RetryConfig.Builder<?> configureExponentialDelay() { | ||
var initialDelay = config.get(SOURCE_LOOKUP_RETRY_EXPONENTIAL_DELAY_INITIAL_BACKOFF); | ||
var maxDelay = config.get(SOURCE_LOOKUP_RETRY_EXPONENTIAL_DELAY_MAX_BACKOFF); | ||
var multiplier = config.get(SOURCE_LOOKUP_RETRY_EXPONENTIAL_DELAY_MULTIPLIER); | ||
return RetryConfig.custom() | ||
.intervalFunction(ofExponentialBackoff(initialDelay, multiplier, maxDelay)); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can we have the logging change in a separate PR - it is easier to track the history then please
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why this change is needed anyways?
Its because of resilence4j?
One of the "rule of thumbs" when we were starting this connector was to try not add any external libraries to the connector, that my or may not clash with any user code -> i.e that is why we use Java's 11 http client.
You need the resilence4j for retry functionality right?
Which in essence is -> schedule a task on Java's scheduled thread executor and make sure to do a good job around error/exception handling.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Dedicated lib for retries might be an overkill now, but I think we can benefit in long term. The library provides Rate Limiter or Circuit Breaker. Both features might be worth adding. Or at least Rate Limiter.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
"Why this change is needed anyways?
Its because of resilence4j?"
Yes, I had to change to compile project with resilence4j. Notice that Flink use the same API:
https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/advanced/logging/
"You need the resilence4j for retry functionality right?"
I thought it's better to use mature library instead of reimplementing it. From the other side,as you said, it's additional dependency. Below part resilience4j dependencies based on mvn dependency:tree:
Do you think it's ok to add them? Another option is to shadow them.