[
https://issues.apache.org/jira/browse/BEAM-10990?focusedWorklogId=656596&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-656596
]
ASF GitHub Bot logged work on BEAM-10990:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 28/Sep/21 20:03
Start Date: 28/Sep/21 20:03
Worklog Time Spent: 10m
Work Description: echauchot commented on a change in pull request #15381:
URL: https://github.com/apache/beam/pull/15381#discussion_r716701847
##########
File path:
sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1623,11 +1651,153 @@ public void setup() throws IOException {
@ProcessElement
public void processElement(ProcessContext c) throws IOException {
- c.output(createBulkApiEntity(spec, c.element(), backendVersion));
+ String inputDoc = c.element();
+ String bulkDirective = createBulkApiEntity(spec, inputDoc,
backendVersion);
+ c.output(
+ WriteSummary.create()
+ .withInputDoc(inputDoc)
+ .withBulkDirective(bulkDirective)
+ // N.B. Saving the element timestamp for later use allows for
exactly emulating
+ // c.output(...) because c.output is equivalent to
+ // c.outputWithTimestamp(..., c.timestamp())
+ .withTimestamp(c.timestamp()));
}
}
}
+ public static class WriteSummaryCoder extends AtomicCoder<WriteSummary>
implements Serializable {
+ private static final WriteSummaryCoder INSTANCE = new WriteSummaryCoder();
+
+ private WriteSummaryCoder() {}
+
+ public static WriteSummaryCoder of() {
+ return INSTANCE;
+ }
+
+ @Override
+ public void encode(WriteSummary value, OutputStream outStream) throws
IOException {
+ NullableCoder.of(StringUtf8Coder.of()).encode(value.getInputDoc(),
outStream);
+ NullableCoder.of(StringUtf8Coder.of()).encode(value.getBulkDirective(),
outStream);
+ BooleanCoder.of().encode(value.getHasError(), outStream);
+
NullableCoder.of(StringUtf8Coder.of()).encode(value.getResponseItemJson(),
outStream);
+ NullableCoder.of(InstantCoder.of()).encode(value.getTimestamp(),
outStream);
+ }
+
+ @Override
+ public WriteSummary decode(InputStream inStream) throws IOException {
+ String inputDoc =
NullableCoder.of(StringUtf8Coder.of()).decode(inStream);
+ String bulkDirective =
NullableCoder.of(StringUtf8Coder.of()).decode(inStream);
+ boolean hasError = BooleanCoder.of().decode(inStream);
+ String responseItemJson =
NullableCoder.of(StringUtf8Coder.of()).decode(inStream);
+ Instant timestamp = NullableCoder.of(InstantCoder.of()).decode(inStream);
+
+ return WriteSummary.create()
+ .withInputDoc(inputDoc)
+ .withBulkDirective(bulkDirective)
+ .withHasError(hasError)
+ .withResponseItemJson(responseItemJson)
+ .withTimestamp(timestamp);
+ }
+ }
+
+ // Immutable POJO for maintaining various states of documents and their bulk
representation, plus
+ // response from ES for the given document and the timestamp of the data
+ @DefaultCoder(WriteSummaryCoder.class)
+ @AutoValue
+ public abstract static class WriteSummary implements Serializable {
Review comment:
As this class is instanciated before any write operation (in the
DocToBulk transform) I'd call it simply `Document` and that would echo with
`DocumentMetaData` existing class. WDYT ?
##########
File path:
sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -237,49 +250,64 @@ static JsonNode parseResponse(HttpEntity responseEntity)
throws IOException {
return mapper.readValue(responseEntity.getContent(), JsonNode.class);
}
- static void checkForErrors(HttpEntity responseEntity, @Nullable Set<String>
allowedErrorTypes)
+ static List<WriteSummary> checkForErrors(
+ HttpEntity responseEntity, @Nullable Set<String> allowedErrorTypes,
boolean throwWriteErrors)
throws IOException {
+ List<WriteSummary> responses = new ArrayList<>();
+ int numErrors = 0;
JsonNode searchResult = parseResponse(responseEntity);
- boolean errors = searchResult.path("errors").asBoolean();
- if (errors) {
- int numErrors = 0;
-
- StringBuilder errorMessages =
- new StringBuilder("Error writing to Elasticsearch, some elements
could not be inserted:");
- JsonNode items = searchResult.path("items");
- if (items.isMissingNode() || items.size() == 0) {
- errorMessages.append(searchResult.toString());
- }
- // some items present in bulk might have errors, concatenate error
messages
- for (JsonNode item : items) {
- JsonNode error = item.findValue("error");
- if (error != null) {
- // N.B. An empty-string within the allowedErrorTypes Set implies all
errors are allowed.
- String type = error.path("type").asText();
- String reason = error.path("reason").asText();
- String docId = item.findValue("_id").asText();
- JsonNode causedBy = error.path("caused_by"); // May not be present
- String cbReason = causedBy.path("reason").asText();
- String cbType = causedBy.path("type").asText();
-
- if (allowedErrorTypes == null
- || (!allowedErrorTypes.contains(type) &&
!allowedErrorTypes.contains(cbType))) {
- // 'error' and 'causedBy` fields are not null, and the error is
not being ignored.
- numErrors++;
-
- errorMessages.append(String.format("%nDocument id %s: %s (%s)",
docId, reason, type));
-
- if (!causedBy.isMissingNode()) {
- errorMessages.append(String.format("%nCaused by: %s (%s)",
cbReason, cbType));
- }
+ StringBuilder errorMessages =
+ new StringBuilder("Error writing to Elasticsearch, some elements could
not be inserted:");
+ JsonNode items = searchResult.path("items");
+
+ if (items.isMissingNode() || items.size() == 0) {
+ // This would only be expected in cases like connectivity issues or
similar
+ errorMessages.append(searchResult.toString());
+ throw new RuntimeException(
+ String.format(
+ "'items' missing from Elasticsearch response: %s",
errorMessages.toString()));
+ }
+
+ // some items present in bulk might have errors, concatenate error
messages and record
+ // which items had errors
+ for (JsonNode item : items) {
+ WriteSummary result =
WriteSummary.create().withResponseItemJson(item.toString());
+
+ JsonNode error = item.findValue("error");
+ if (error != null) {
+ // N.B. An empty-string within the allowedErrorTypes Set implies all
errors are allowed.
+ String type = error.path("type").asText();
+ String reason = error.path("reason").asText();
+ String docId = item.findValue("_id").asText();
+ JsonNode causedBy = error.path("caused_by"); // May not be present
+ String cbReason = causedBy.path("reason").asText();
+ String cbType = causedBy.path("type").asText();
+
+ if (allowedErrorTypes == null
+ || (!allowedErrorTypes.contains(type) &&
!allowedErrorTypes.contains(cbType))) {
+ // 'error' and 'causedBy` fields are not null, and the error is not
being ignored.
+ result = result.withHasError(true);
+ numErrors++;
+
+ errorMessages.append(String.format("%nDocument id %s: %s (%s)",
docId, reason, type));
+
+ if (!causedBy.isMissingNode()) {
+ errorMessages.append(String.format("%nCaused by: %s (%s)",
cbReason, cbType));
}
}
}
- if (numErrors > 0) {
+ responses.add(result);
+ }
+
+ if (numErrors > 0) {
+ LOG.error(errorMessages.toString());
+ if (throwWriteErrors) {
Review comment:
l like that this behavior is configurable !
##########
File path:
sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
##########
@@ -141,6 +141,18 @@ public void testWriteWithAllowableErrors() throws
Exception {
elasticsearchIOTestCommon.testWriteWithAllowedErrors();
}
+ @Test
Review comment:
No need to test in ITest it is more of an UTest. ITests are for high load
##########
File path:
sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -237,49 +250,64 @@ static JsonNode parseResponse(HttpEntity responseEntity)
throws IOException {
return mapper.readValue(responseEntity.getContent(), JsonNode.class);
}
- static void checkForErrors(HttpEntity responseEntity, @Nullable Set<String>
allowedErrorTypes)
+ static List<WriteSummary> checkForErrors(
+ HttpEntity responseEntity, @Nullable Set<String> allowedErrorTypes,
boolean throwWriteErrors)
throws IOException {
+ List<WriteSummary> responses = new ArrayList<>();
+ int numErrors = 0;
JsonNode searchResult = parseResponse(responseEntity);
- boolean errors = searchResult.path("errors").asBoolean();
- if (errors) {
- int numErrors = 0;
-
- StringBuilder errorMessages =
- new StringBuilder("Error writing to Elasticsearch, some elements
could not be inserted:");
- JsonNode items = searchResult.path("items");
- if (items.isMissingNode() || items.size() == 0) {
- errorMessages.append(searchResult.toString());
- }
- // some items present in bulk might have errors, concatenate error
messages
- for (JsonNode item : items) {
- JsonNode error = item.findValue("error");
- if (error != null) {
- // N.B. An empty-string within the allowedErrorTypes Set implies all
errors are allowed.
- String type = error.path("type").asText();
- String reason = error.path("reason").asText();
- String docId = item.findValue("_id").asText();
- JsonNode causedBy = error.path("caused_by"); // May not be present
- String cbReason = causedBy.path("reason").asText();
- String cbType = causedBy.path("type").asText();
-
- if (allowedErrorTypes == null
- || (!allowedErrorTypes.contains(type) &&
!allowedErrorTypes.contains(cbType))) {
- // 'error' and 'causedBy` fields are not null, and the error is
not being ignored.
- numErrors++;
-
- errorMessages.append(String.format("%nDocument id %s: %s (%s)",
docId, reason, type));
-
- if (!causedBy.isMissingNode()) {
- errorMessages.append(String.format("%nCaused by: %s (%s)",
cbReason, cbType));
- }
+ StringBuilder errorMessages =
+ new StringBuilder("Error writing to Elasticsearch, some elements could
not be inserted:");
+ JsonNode items = searchResult.path("items");
+
+ if (items.isMissingNode() || items.size() == 0) {
+ // This would only be expected in cases like connectivity issues or
similar
+ errorMessages.append(searchResult.toString());
+ throw new RuntimeException(
Review comment:
fail the pipeline in that case ?
##########
File path:
sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1789,8 +1964,14 @@ public Write withAllowableResponseErrors(@Nullable
Set<String> allowableResponse
return this;
}
+ /** Refer to {@link BulkIO#withThrowWriteErrors}. */
+ public Write withThrowWriteErrors(boolean throwWriteErrors) {
Review comment:
Please add a hint to tell users that this parameter could be usefull in
case the pipeline is in streaming mode to avoid infinite retry
##########
File path:
sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1636,11 +1806,15 @@ public void processElement(ProcessContext c) throws
IOException {
* cluster. This class is effectively a thin proxy for DocToBulk->BulkIO
all-in-one for
* convenience and backward compatibility.
*/
- public static class Write extends PTransform<PCollection<String>, PDone> {
+ public static class Write extends PTransform<PCollection<String>,
PCollectionTuple> {
Review comment:
I see the write tests do not change so there is no breaking change. I
guess the user could chose whether he wants to process the PCollectionTuple or
not.
##########
File path:
sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -237,49 +250,64 @@ static JsonNode parseResponse(HttpEntity responseEntity)
throws IOException {
return mapper.readValue(responseEntity.getContent(), JsonNode.class);
}
- static void checkForErrors(HttpEntity responseEntity, @Nullable Set<String>
allowedErrorTypes)
+ static List<WriteSummary> checkForErrors(
Review comment:
I would rename this method. Indeed, it can also report items written
with sucess so I'd call it something like `createWriteReport`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 656596)
Time Spent: 3h 10m (was: 3h)
> Elasticsearch IO Infinite loop with write Error when the pipeline job
> streaming mode
> -------------------------------------------------------------------------------------
>
> Key: BEAM-10990
> URL: https://issues.apache.org/jira/browse/BEAM-10990
> Project: Beam
> Issue Type: Improvement
> Components: io-java-elasticsearch
> Affects Versions: 2.24.0
> Reporter: Steven Gaunt
> Assignee: Evan Galpin
> Priority: P3
> Time Spent: 3h 10m
> Remaining Estimate: 0h
>
> When streaming messages from PubsubIO , the pipeline is in Streaming mode.
> If for some reason the ElasticSearchIO.Write() has an response from the
> ElasticSearch index api, the writefn will throw a IOException. Since this
> excetpion is part of the Write transform, it becomes an unhandled error.
> This will then inheritly cause behaviour from the job pipeline to infinitely
> retry that error..
> _*snippet form beam website*_
> The Dataflow service retries failed tasks up to 4 times in batch mode, and an
> unlimited number of times in streaming mode. In batch mode, your job will
> fail; in streaming, it may stall indefinitely.
>
> This is the ElasticSearchIO.write transform .
> {code:java}
> public PDone expand(PCollection<String> input) {
> ElasticsearchIO.ConnectionConfiguration connectionConfiguration =
> this.getConnectionConfiguration();
> Preconditions.checkState(connectionConfiguration != null,
> "withConnectionConfiguration() is required");
> * input.apply(ParDo.of(new ElasticsearchIO.Write.WriteFn(this)));*
> return PDone.in(input.getPipeline());
> }
> {code}
> The pardo function (WriteFn) finishBundle step will call
> ElasticsearchIO.checkForErrors helper method which will throw exception if
> the http response from elasticsearch has error in the json reponse.
> {code:java}
> // Some comments here
> public void finishBundle(DoFn<String, Void>.FinishBundleContext context)
> throws IOException, InterruptedException {
> this.flushBatch();
> }
> private void flushBatch() throws IOException,
> InterruptedException {
> if (!this.batch.isEmpty()) {
> StringBuilder bulkRequest = new StringBuilder();
> Iterator var2 = this.batch.iterator();
> while(var2.hasNext()) {
> String json = (String)var2.next();
> bulkRequest.append(json);
> }
> this.batch.clear();
> this.currentBatchSizeBytes = 0L;
> String endPoint = String.format("/%s/%s/_bulk",
> this.spec.getConnectionConfiguration().getIndex(),
> this.spec.getConnectionConfiguration().getType());
> HttpEntity requestBody = new
> NStringEntity(bulkRequest.toString(), ContentType.APPLICATION_JSON);
> Request request = new Request("POST", endPoint);
> request.addParameters(Collections.emptyMap());
> request.setEntity(requestBody);
> Response response =
> this.restClient.performRequest(request);
> HttpEntity responseEntity = new
> BufferedHttpEntity(response.getEntity());
> if (this.spec.getRetryConfiguration() != null &&
> this.spec.getRetryConfiguration().getRetryPredicate().test(responseEntity)) {
> responseEntity = this.handleRetry("POST", endPoint,
> Collections.emptyMap(), requestBody);
> }
>
> ElasticsearchIO.checkForErrors((HttpEntity)responseEntity,
> this.backendVersion, this.spec.getUsePartialUpdate());
> }
> }
> static void checkForErrors(HttpEntity responseEntity, int backendVersion,
> boolean partialUpdate) throws IOException {
> JsonNode searchResult = parseResponse(responseEntity);
> boolean errors = searchResult.path("errors").asBoolean();
> if (errors) {
> StringBuilder errorMessages = new StringBuilder("Error writing to
> Elasticsearch, some elements could not be inserted:");
> JsonNode items = searchResult.path("items");
> Iterator var7 = items.iterator();
> while(var7.hasNext()) {
> JsonNode item = (JsonNode)var7.next();
> String errorRootName = "";
> if (partialUpdate) {
> errorRootName = "update";
> } else if (backendVersion == 2) {
> errorRootName = "create";
> } else if (backendVersion >= 5) {
> errorRootName = "index";
> }
> JsonNode errorRoot = item.path(errorRootName);
> JsonNode error = errorRoot.get("error");
> if (error != null) {
> String type = error.path("type").asText();
> String reason = error.path("reason").asText();
> String docId = errorRoot.path("_id").asText();
> errorMessages.append(String.format("%nDocument id %s: %s
> (%s)", docId, reason, type));
> JsonNode causedBy = error.get("caused_by");
> if (causedBy != null) {
> String cbReason = causedBy.path("reason").asText();
> String cbType = causedBy.path("type").asText();
> errorMessages.append(String.format("%nCaused by: %s
> (%s)", cbReason, cbType));
> }
> }
> }
> throw new IOException(errorMessages.toString());
> }
> }
> {code}
>
> As a possible suggestion, rather than throw the exception, could it be
> possible to write the exception to an errorhandling tupeltag which then can
> handled to a deadletter queue ?
>
--
This message was sent by Atlassian Jira
(v8.3.4#803005)