mirror of https://github.com/apache/nifi.git
NIFI-3087: Fixed issue with partial failure responses in PutElasticsearch(Http)
This commit is contained in:
parent
066accc274
commit
d3dbac50a8
|
@ -211,11 +211,14 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
|
|||
|
||||
final BulkResponse response = bulk.execute().actionGet();
|
||||
if (response.hasFailures()) {
|
||||
for (final BulkItemResponse item : response.getItems()) {
|
||||
final FlowFile flowFile = flowFilesToTransfer.get(item.getItemId());
|
||||
if (item.isFailed()) {
|
||||
// Responses are guaranteed to be in order, remove them in reverse order
|
||||
BulkItemResponse[] responses = response.getItems();
|
||||
if (responses != null && responses.length > 0) {
|
||||
for (int i = responses.length - 1; i >= 0; i--) {
|
||||
final FlowFile flowFile = flowFilesToTransfer.get(i);
|
||||
if (responses[i].isFailed()) {
|
||||
logger.error("Failed to insert {} into Elasticsearch due to {}, transferring to failure",
|
||||
new Object[]{flowFile, item.getFailure().getMessage()});
|
||||
new Object[]{flowFile, responses[i].getFailure().getMessage()});
|
||||
session.transfer(flowFile, REL_FAILURE);
|
||||
|
||||
} else {
|
||||
|
@ -224,6 +227,7 @@ public class PutElasticsearch extends AbstractElasticsearchTransportClientProces
|
|||
flowFilesToTransfer.remove(flowFile);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Transfer any remaining flowfiles to success
|
||||
session.transfer(flowFilesToTransfer, REL_SUCCESS);
|
||||
|
|
|
@ -328,7 +328,7 @@ public class PutElasticsearchHttp extends AbstractElasticsearchHttpProcessor {
|
|||
if (itemNodeArray.size() > 0) {
|
||||
// All items are returned whether they succeeded or failed, so iterate through the item array
|
||||
// at the same time as the flow file list, moving each to success or failure accordingly
|
||||
for (int i = 0; i < itemNodeArray.size(); i++) {
|
||||
for (int i = itemNodeArray.size() - 1; i >= 0; i--) {
|
||||
JsonNode itemNode = itemNodeArray.get(i);
|
||||
FlowFile flowFile = flowFilesToTransfer.remove(i);
|
||||
int status = itemNode.findPath("status").asInt();
|
||||
|
|
Loading…
Reference in New Issue