[ 
https://issues.apache.org/jira/browse/NIFI-15001?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18027877#comment-18027877
 ] 

ASF subversion and git services commented on NIFI-15001:
--------------------------------------------------------

Commit 0274742a2920d399b7ec3aa486a5ca94e0f0a804 in nifi's branch 
refs/heads/main from Pierre Villard
[ https://gitbox.apache.org/repos/asf?p=nifi.git;h=0274742a29 ]

NIFI-15001 Fixed PutElasticsearchRecord with recursive reads (#10334)

Signed-off-by: David Handermann <[email protected]>

> PutElasticsearchRecord bulk requests failing
> --------------------------------------------
>
>                 Key: NIFI-15001
>                 URL: https://issues.apache.org/jira/browse/NIFI-15001
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Extensions
>    Affects Versions: 2.3.0, 2.4.0, 2.5.0, 2.6.0
>            Reporter: Adam Turley
>            Assignee: Pierre Villard
>            Priority: Major
>          Time Spent: 1.5h
>  Remaining Estimate: 0h
>
> When PutElasticsearchRecord's batch size is larger than the records in a flow 
> file everything is fine. however, if i have a 2000 record flow file and my 
> batch size is set to 100 it will fail with the following error:
> {code:java}
> ERROR PutElasticsearchRecord[id=52bf2533-0199-1000-5339-6fb2ce41a3f9] 
> Processing halted: yielding [1 sec]: 
> org.apache.nifi.processor.exception.FlowFileHandlingException: 
> StandardFlowFileRecord[uuid=bb88525f-0ee2-47cf-83ca-f2a2072f5790,claim=StandardContentClaim
>  [resourceClaim=StandardResourceClaim[id=1758032959020-2346, 
> container=default, section=298], offset=0, 
> length=316899],offset=0,name=1375079273674113,size=316899] is not known in 
> this session (StandardProcessSession[id=40715]){code}
> {code:java}
> ERROR PutElasticsearchRecord[id=52bf2533-0199-1000-5339-6fb2ce41a3f9] Could 
> not index documents.: 
> org.apache.nifi.processor.exception.FlowFileAccessException: Could not read 
> from 
> StandardFlowFileRecord[uuid=bb88525f-0ee2-47cf-83ca-f2a2072f5790,claim=StandardContentClaim
>  [resourceClaim=StandardResourceClaim[id=1758032959020-2346, 
> container=default, section=298], offset=0, 
> length=316899],offset=0,name=1375079273674113,size=316899] - Caused by: 
> java.io.IOException: Stream closed
> {code}
>  
> It appears in 2.3.0 the processor was altered. I was able to make the code 
> work again by altering this section of PutElasticsearchRecord.java:
>  
> {code:java}
>     private ResponseDetails indexDocuments(final BulkOperation bundle, final 
> ProcessSession session, final FlowFile input,
>                                            final IndexOperationParameters 
> indexOperationParameters, final int batch)
>             throws IOException, SchemaNotFoundException {
>         final IndexOperationResponse response = 
> clientService.get().bulk(bundle.getOperationList(), 
> indexOperationParameters.getElasticsearchRequestOptions());
>         final Map<Integer, Map<String, Object>> errors = 
> findElasticsearchResponseErrors(response);
>         if (!errors.isEmpty()) {
>             handleElasticsearchDocumentErrors(errors, session, input);
>         }
>         final int numErrors = errors.size();
>         final int numSuccessful = response.getItems() == null ? 0 : 
> response.getItems().size() - numErrors;
>         final Map<String, Output> outputs = new HashMap<>();
>         try {
>             for (int o = 0; o < bundle.getOriginalRecords().size(); o++) {
>                 final String type;
>                 final Relationship relationship;
>                 final Map<String, Object> error;
>                 final Record outputRecord = 
> bundle.getOriginalRecords().get(o);
>                 final RecordSchema recordSchema = outputRecord.getSchema();
>                 if (numErrors > 0 && errors.containsKey(o)) {
>                     relationship = REL_ERRORS;
>                     error = errors.get(o);
>                     if (groupBulkErrors) {
>                         if (isElasticsearchNotFound().test(error)) {
>                             type = OUTPUT_TYPE_NOT_FOUND;
>                         } else {
>                             type = getErrorType(error);
>                         }
>                     } else {
>                         type = OUTPUT_TYPE_ERROR;
>                     }
>                 } else {
>                     relationship = REL_SUCCESSFUL;
>                     error = null;
>                     type = OUTPUT_TYPE_SUCCESS;
>                 }
>                 final Output output = getOutputByType(outputs, type, session, 
> relationship, input, recordSchema);
>                 output.write(outputRecord, error);
>             }
>             for (final Output output : outputs.values()) {
>                 output.transfer(session);
>             }
>         } catch (final IOException | SchemaNotFoundException ex) {
>             getLogger().error("Unable to write error/successful records", ex);
>             outputs.values().forEach(o -> {
>                 try {
>                     o.remove(session);
>                 } catch (IOException ioe) {
>                     getLogger().warn("Error closing RecordSetWriter for 
> FlowFile", ioe);
>                 }
>             });
>             throw ex;
>         }
>         return new ResponseDetails(outputs, numSuccessful, numErrors);
>     }
>  
> {code}
> after this change the test will fail, but when used in nifi it's able to 
> batch properly.
> more examples in slack:
> https://apachenifi.slack.com/archives/C0L9VCD47/p1757948975602889



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to