-
Notifications
You must be signed in to change notification settings - Fork 4.3k
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
Handle response exception from Elasticsearch client #26424
Changes from all commits
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 |
---|---|---|
|
@@ -110,6 +110,7 @@ | |
import org.checkerframework.checker.nullness.qual.Nullable; | ||
import org.elasticsearch.client.Request; | ||
import org.elasticsearch.client.Response; | ||
import org.elasticsearch.client.ResponseException; | ||
import org.elasticsearch.client.RestClient; | ||
import org.elasticsearch.client.RestClientBuilder; | ||
import org.joda.time.Duration; | ||
|
@@ -343,6 +344,8 @@ public abstract static class ConnectionConfiguration implements Serializable { | |
|
||
public abstract boolean isTrustSelfSignedCerts(); | ||
|
||
abstract boolean isInvalidBulkEndpoint(); | ||
|
||
abstract Builder builder(); | ||
|
||
@AutoValue.Builder | ||
|
@@ -373,6 +376,8 @@ abstract static class Builder { | |
|
||
abstract Builder setTrustSelfSignedCerts(boolean trustSelfSignedCerts); | ||
|
||
abstract Builder setInvalidBulkEndpoint(boolean invalidBulkEndpoint); | ||
|
||
abstract ConnectionConfiguration build(); | ||
} | ||
|
||
|
@@ -394,6 +399,7 @@ public static ConnectionConfiguration create(String[] addresses, String index, S | |
.setIndex(index) | ||
.setType(type) | ||
.setTrustSelfSignedCerts(false) | ||
.setInvalidBulkEndpoint(false) | ||
.build(); | ||
} | ||
|
||
|
@@ -413,6 +419,7 @@ public static ConnectionConfiguration create(String[] addresses, String index) { | |
.setIndex(index) | ||
.setType("") | ||
.setTrustSelfSignedCerts(false) | ||
.setInvalidBulkEndpoint(false) | ||
.build(); | ||
} | ||
|
||
|
@@ -430,6 +437,7 @@ public static ConnectionConfiguration create(String[] addresses) { | |
.setIndex("") | ||
.setType("") | ||
.setTrustSelfSignedCerts(false) | ||
.setInvalidBulkEndpoint(false) | ||
.build(); | ||
} | ||
|
||
|
@@ -468,6 +476,9 @@ public String getPrefixedEndpoint(String endpoint) { | |
} | ||
|
||
public String getBulkEndPoint() { | ||
if (isInvalidBulkEndpoint()) { | ||
return ""; | ||
} | ||
return getPrefixedEndpoint("_bulk"); | ||
} | ||
|
||
|
@@ -642,6 +653,12 @@ public ConnectionConfiguration withConnectTimeout(Integer connectTimeout) { | |
return builder().setConnectTimeout(connectTimeout).build(); | ||
} | ||
|
||
// Exposed only to allow tests to easily simulate server errors | ||
@VisibleForTesting | ||
ConnectionConfiguration withInvalidBulkEndpoint() { | ||
return builder().setInvalidBulkEndpoint(true).build(); | ||
} | ||
|
||
private void populateDisplayData(DisplayData.Builder builder) { | ||
builder.add(DisplayData.item("address", getAddresses().toString())); | ||
builder.add(DisplayData.item("index", getIndex())); | ||
|
@@ -1204,7 +1221,7 @@ RetryConfiguration withRetryPredicate(RetryPredicate predicate) { | |
* the requests to the Elasticsearch server if the {@link RetryConfiguration} permits it. | ||
*/ | ||
@FunctionalInterface | ||
interface RetryPredicate extends Predicate<HttpEntity>, Serializable {} | ||
interface RetryPredicate extends Predicate<Integer>, Serializable {} | ||
|
||
/** | ||
* This is the default predicate used to test if a failed ES operation should be retried. A | ||
|
@@ -1224,26 +1241,9 @@ static class DefaultRetryPredicate implements RetryPredicate { | |
this(429); | ||
} | ||
|
||
/** Returns true if the response has the error code for any mutation. */ | ||
private static boolean errorCodePresent(HttpEntity responseEntity, int errorCode) { | ||
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. Thanks @mrkm4ntr for your patience, I have kept you waiting too many times. This change is definitely interesting. I can see that in some cases, such as successfully creating a document via _bulk, the http status code is {
"took": 383,
"errors": false,
"items": [
{
"update": {
"_index": "foo",
"_type": "_doc",
"_id": "2",
"_version": 1,
"result": "created",
"_shards": {
"total": 2,
"successful": 1,
"failed": 0
},
"_seq_no": 0,
"_primary_term": 1,
"status": 201
}
}
]
} It's also the case, more importantly, that when one of the operations of the bulk payload fails, but another succeeds, there can be an http status code of 200 but one of the items can have {
"took": 19,
"errors": true,
"items": [
{
"update": {
"_index": "foo",
"_type": "_doc",
"_id": "2",
"_version": 7,
"result": "updated",
"_shards": {
"total": 2,
"successful": 2,
"failed": 0
},
"_seq_no": 7,
"_primary_term": 1,
"status": 200
}
},
{
"update": {
"_index": "foo",
"_type": "_doc",
"_id": "2",
"status": 400,
"error": {
"type": "illegal_argument_exception",
"reason": "failed to execute script",
"caused_by": {
"type": "script_exception",
"reason": "compile error",
"script_stack": [
"if(ctx._source.group !=== null) { ctx._source.gro ...",
" ^---- HERE"
],
"script": "if(ctx._source.group !=== null) { ctx._source.group = params.id % 2 } else { ctx._source.group = 0 }",
"lang": "painless",
"position": {
"offset": 24,
"start": 0,
"end": 49
},
"caused_by": {
"type": "illegal_argument_exception",
"reason": "invalid sequence of tokens near ['='].",
"caused_by": {
"type": "no_viable_alt_exception",
"reason": "no_viable_alt_exception: null"
}
}
}
}
}
}
]
} So the status code alone cannot be "trusted" to inform of all failures, since there could be false negatives. However, it seems that the http status code can be used as a first check: if the http status code is >=400, we know a failure has occurred and we don't need to iterate through the Checking the http status code first might be a viable path forward for properly handling the 429 case. Thoughts? 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. Thanks. OK, I will change my PR like this.
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.
It's more so that I feel that we cannot remove the I believe the failure mode for not properly handling http 429 is that the response body does not contain the key So I feel that we need to keep all existing |
||
try { | ||
JsonNode json = parseResponse(responseEntity); | ||
if (json.path("errors").asBoolean()) { | ||
for (JsonNode item : json.path("items")) { | ||
if (item.findValue("status").asInt() == errorCode) { | ||
return true; | ||
} | ||
} | ||
} | ||
} catch (IOException e) { | ||
LOG.warn("Could not extract error codes from responseEntity {}", responseEntity); | ||
} | ||
return false; | ||
} | ||
|
||
@Override | ||
public boolean test(HttpEntity responseEntity) { | ||
return errorCodePresent(responseEntity, errorCode); | ||
public boolean test(Integer statusCode) { | ||
return this.errorCode == statusCode; | ||
} | ||
} | ||
} | ||
|
@@ -2535,8 +2535,7 @@ private List<Document> flushBatch() throws IOException, InterruptedException { | |
bulkRequest.append(doc.getBulkDirective()); | ||
} | ||
|
||
Response response = null; | ||
HttpEntity responseEntity = null; | ||
HttpEntity responseEntity; | ||
|
||
// Elasticsearch will default to the index/type provided the {@link | ||
// ConnectionConfiguration} if none are set in the document meta (i.e. | ||
|
@@ -2546,28 +2545,15 @@ private List<Document> flushBatch() throws IOException, InterruptedException { | |
|
||
HttpEntity requestBody = | ||
new NStringEntity(bulkRequest.toString(), ContentType.APPLICATION_JSON); | ||
try { | ||
if (spec.getRetryConfiguration() != null) { | ||
responseEntity = | ||
performRequestWithRetry("POST", endPoint, Collections.emptyMap(), requestBody); | ||
} else { | ||
Request request = new Request("POST", endPoint); | ||
request.addParameters(Collections.emptyMap()); | ||
request.setEntity(requestBody); | ||
response = restClient.performRequest(request); | ||
Response response = restClient.performRequest(request); | ||
responseEntity = new BufferedHttpEntity(response.getEntity()); | ||
} catch (java.io.IOException ex) { | ||
if (spec.getRetryConfiguration() == null || !isRetryableClientException(ex)) { | ||
throw ex; | ||
} | ||
LOG.error("Caught ES timeout, retrying", ex); | ||
} | ||
|
||
if (spec.getRetryConfiguration() != null | ||
&& (response == null | ||
|| responseEntity == null | ||
|| spec.getRetryConfiguration().getRetryPredicate().test(responseEntity))) { | ||
if (responseEntity != null | ||
&& spec.getRetryConfiguration().getRetryPredicate().test(responseEntity)) { | ||
LOG.warn("ES Cluster is responding with HTP 429 - TOO_MANY_REQUESTS."); | ||
} | ||
responseEntity = handleRetry("POST", endPoint, Collections.emptyMap(), requestBody); | ||
} | ||
|
||
List<Document> responses = | ||
|
@@ -2584,39 +2570,40 @@ private List<Document> flushBatch() throws IOException, InterruptedException { | |
.collect(Collectors.toList()); | ||
} | ||
|
||
/** retry request based on retry configuration policy. */ | ||
private HttpEntity handleRetry( | ||
/** performe and retry request based on retry configuration policy. */ | ||
private HttpEntity performRequestWithRetry( | ||
String method, String endpoint, Map<String, String> params, HttpEntity requestBody) | ||
throws IOException, InterruptedException { | ||
Response response; | ||
HttpEntity responseEntity = null; | ||
RetryConfiguration.RetryPredicate predicate = | ||
Objects.requireNonNull(spec.getRetryConfiguration()).getRetryPredicate(); | ||
Sleeper sleeper = Sleeper.DEFAULT; | ||
BackOff backoff = retryBackoff.backoff(); | ||
int attempt = 0; | ||
int attempt = -1; | ||
// while retry policy exists | ||
while (BackOffUtils.next(sleeper, backoff)) { | ||
LOG.warn(RETRY_ATTEMPT_LOG, ++attempt); | ||
do { | ||
if (++attempt > 0) { | ||
LOG.warn(RETRY_ATTEMPT_LOG, attempt); | ||
} | ||
try { | ||
Request request = new Request(method, endpoint); | ||
request.addParameters(params); | ||
request.setEntity(requestBody); | ||
response = restClient.performRequest(request); | ||
responseEntity = new BufferedHttpEntity(response.getEntity()); | ||
Response response = restClient.performRequest(request); | ||
return new BufferedHttpEntity(response.getEntity()); | ||
} catch (ResponseException ex) { | ||
if (predicate.test(ex.getResponse().getStatusLine().getStatusCode())) { | ||
LOG.warn("ES Cluster is responding with HTTP 429 - TOO_MANY_REQUESTS.", ex); | ||
} else { | ||
throw ex; | ||
} | ||
} catch (java.io.IOException ex) { | ||
if (isRetryableClientException(ex)) { | ||
LOG.error("Caught ES timeout, retrying", ex); | ||
continue; | ||
LOG.warn("Caught ES timeout, retrying", ex); | ||
} else { | ||
throw ex; | ||
} | ||
} | ||
// if response has no 429 errors | ||
if (!Objects.requireNonNull(spec.getRetryConfiguration()) | ||
.getRetryPredicate() | ||
.test(responseEntity)) { | ||
return responseEntity; | ||
} else { | ||
LOG.warn("ES Cluster is responding with HTP 429 - TOO_MANY_REQUESTS."); | ||
} | ||
} | ||
} while (BackOffUtils.next(sleeper, backoff)); | ||
throw new IOException(String.format(RETRY_FAILED_LOG, attempt)); | ||
} | ||
|
||
|
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.
I don't particularly like the idea of having this builder method only for the sake of testing. I feel it will be very confusing to users to see this as an option when building configurations. Could you look for an alternative approach?
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.
I think this method and this are not visible from users. Same approach as this.