-
Notifications
You must be signed in to change notification settings - Fork 2.9k
NIFI-15001 - Fix PutElasticsearchRecord with recursive reads #10334
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
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 |
---|---|---|
|
@@ -61,6 +61,7 @@ | |
import org.apache.nifi.serialization.RecordSetWriter; | ||
import org.apache.nifi.serialization.RecordSetWriterFactory; | ||
import org.apache.nifi.serialization.record.DataType; | ||
import org.apache.nifi.serialization.record.MapRecord; | ||
import org.apache.nifi.serialization.record.PushBackRecordSet; | ||
import org.apache.nifi.serialization.record.Record; | ||
import org.apache.nifi.serialization.record.RecordField; | ||
|
@@ -79,6 +80,8 @@ | |
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.HashSet; | ||
import java.util.Collection; | ||
import java.util.LinkedHashMap; | ||
import java.util.List; | ||
import java.util.Locale; | ||
import java.util.Map; | ||
|
@@ -420,22 +423,28 @@ public void onTrigger(final ProcessContext context, final ProcessSession session | |
try (final InputStream inStream = session.read(input); | ||
final RecordReader reader = readerFactory.createRecordReader(input, inStream, getLogger())) { | ||
final PushBackRecordSet recordSet = new PushBackRecordSet(reader.createRecordSet()); | ||
final RecordSchema recordSchema = reader.getSchema(); | ||
final List<IndexOperationRequest> operationList = new ArrayList<>(); | ||
final List<Record> originals = new ArrayList<>(); | ||
final List<Record> processedRecords = new ArrayList<>(); | ||
final List<Record> originalRecords = new ArrayList<>(); | ||
|
||
Record record; | ||
while ((record = recordSet.next()) != null) { | ||
addOperation(operationList, record, indexOperationParameters, indices, types); | ||
originals.add(record); | ||
final Record originalRecord = cloneRecord(record); | ||
final Record processedRecord = cloneRecord(record); | ||
|
||
addOperation(operationList, processedRecord, indexOperationParameters, indices, types); | ||
processedRecords.add(processedRecord); | ||
originalRecords.add(originalRecord); | ||
|
||
if (operationList.size() == indexOperationParameters.getBatchSize() || !recordSet.isAnotherRecord()) { | ||
operate(operationList, originals, reader, session, input, indexOperationParameters, resultRecords, errorRecords, successfulRecords, batches); | ||
operate(operationList, processedRecords, originalRecords, recordSchema, session, input, indexOperationParameters, resultRecords, errorRecords, successfulRecords); | ||
batches++; | ||
} | ||
} | ||
|
||
if (!operationList.isEmpty()) { | ||
operate(operationList, originals, reader, session, input, indexOperationParameters, resultRecords, errorRecords, successfulRecords, batches); | ||
operate(operationList, processedRecords, originalRecords, recordSchema, session, input, indexOperationParameters, resultRecords, errorRecords, successfulRecords); | ||
batches++; | ||
} | ||
} catch (final ElasticsearchException ese) { | ||
|
@@ -508,20 +517,21 @@ private void addOperation(final List<IndexOperationRequest> operationList, final | |
operationList.add(new IndexOperationRequest(index, type, id, contentMap, indexOp, script, scriptedUpsert, dynamicTemplates, bulkHeaderFields)); | ||
} | ||
|
||
private void operate(final List<IndexOperationRequest> operationList, final List<Record> originals, final RecordReader reader, | ||
final ProcessSession session, final FlowFile input, final IndexOperationParameters indexOperationParameters, | ||
final List<FlowFile> resultRecords, final AtomicLong erroredRecords, final AtomicLong successfulRecords, final int batch) | ||
private void operate(final List<IndexOperationRequest> operationList, final List<Record> processedRecords, final List<Record> originalRecords, | ||
final RecordSchema recordSchema, final ProcessSession session, final FlowFile input, final IndexOperationParameters indexOperationParameters, | ||
final List<FlowFile> resultRecords, final AtomicLong erroredRecords, final AtomicLong successfulRecords) | ||
throws IOException, SchemaNotFoundException, MalformedRecordException { | ||
|
||
final BulkOperation bundle = new BulkOperation(operationList, originals, reader.getSchema()); | ||
final ResponseDetails responseDetails = indexDocuments(bundle, session, input, indexOperationParameters, batch); | ||
final BulkOperation bundle = new BulkOperation(operationList, processedRecords, recordSchema); | ||
final ResponseDetails responseDetails = indexDocuments(bundle, originalRecords, session, input, indexOperationParameters); | ||
|
||
successfulRecords.getAndAdd(responseDetails.successCount()); | ||
erroredRecords.getAndAdd(responseDetails.errorCount()); | ||
resultRecords.addAll(responseDetails.outputs().values().stream().map(Output::getFlowFile).toList()); | ||
|
||
operationList.clear(); | ||
originals.clear(); | ||
processedRecords.clear(); | ||
originalRecords.clear(); | ||
} | ||
|
||
private void removeResultRecordFlowFiles(final List<FlowFile> results, final ProcessSession session) { | ||
|
@@ -532,8 +542,8 @@ private void removeResultRecordFlowFiles(final List<FlowFile> results, final Pro | |
results.clear(); | ||
} | ||
|
||
private ResponseDetails indexDocuments(final BulkOperation bundle, final ProcessSession session, final FlowFile input, | ||
final IndexOperationParameters indexOperationParameters, final int batch) | ||
private ResponseDetails indexDocuments(final BulkOperation bundle, final List<Record> originalRecords, final ProcessSession session, final FlowFile input, | ||
final IndexOperationParameters indexOperationParameters) | ||
throws IOException, SchemaNotFoundException, MalformedRecordException { | ||
final IndexOperationResponse response = clientService.get().bulk(bundle.getOperationList(), indexOperationParameters.getElasticsearchRequestOptions()); | ||
|
||
|
@@ -546,16 +556,7 @@ private ResponseDetails indexDocuments(final BulkOperation bundle, final Process | |
final int numSuccessful = response.getItems() == null ? 0 : response.getItems().size() - numErrors; | ||
final Map<String, Output> outputs = new HashMap<>(); | ||
|
||
try (final InputStream inStream = session.read(input); | ||
final RecordReader inputReader = readerFactory.createRecordReader(input, inStream, getLogger())) { | ||
|
||
// if there are errors present, skip through the input FlowFile to the current batch of records | ||
if (numErrors > 0) { | ||
for (int r = 0; r < batch * indexOperationParameters.getBatchSize(); r++) { | ||
inputReader.nextRecord(); | ||
} | ||
} | ||
|
||
try { | ||
for (int o = 0; o < bundle.getOriginalRecords().size(); o++) { | ||
final String type; | ||
final Relationship relationship; | ||
|
@@ -574,18 +575,14 @@ private ResponseDetails indexDocuments(final BulkOperation bundle, final Process | |
} else { | ||
type = OUTPUT_TYPE_ERROR; | ||
} | ||
outputRecord = inputReader.nextRecord(); | ||
outputRecord = originalRecords.get(o); | ||
recordSchema = outputRecord.getSchema(); | ||
} else { | ||
relationship = REL_SUCCESSFUL; | ||
error = null; | ||
type = OUTPUT_TYPE_SUCCESS; | ||
outputRecord = bundle.getOriginalRecords().get(o); | ||
recordSchema = bundle.getSchema(); | ||
// skip the associated Input Record for this successful Record | ||
if (numErrors > 0) { | ||
inputReader.nextRecord(); | ||
} | ||
} | ||
final Output output = getOutputByType(outputs, type, session, relationship, input, recordSchema); | ||
output.write(outputRecord, error); | ||
|
@@ -594,7 +591,7 @@ private ResponseDetails indexDocuments(final BulkOperation bundle, final Process | |
for (final Output output : outputs.values()) { | ||
output.transfer(session); | ||
} | ||
} catch (final IOException | SchemaNotFoundException | MalformedRecordException ex) { | ||
} catch (final IOException | SchemaNotFoundException ex) { | ||
getLogger().error("Unable to write error/successful records", ex); | ||
outputs.values().forEach(o -> { | ||
try { | ||
|
@@ -773,6 +770,47 @@ private Object coerceStringToLong(final String fieldName, final String stringVal | |
: stringValue; | ||
} | ||
|
||
private Record cloneRecord(final Record record) { | ||
final Map<String, Object> valuesCopy = cloneValues(record.toMap()); | ||
return new MapRecord(record.getSchema(), valuesCopy, record.isTypeChecked(), record.isDropUnknownFields()); | ||
} | ||
|
||
private Map<String, Object> cloneValues(final Map<String, Object> values) { | ||
final Map<String, Object> cloned = new LinkedHashMap<>(values.size()); | ||
for (final Map.Entry<String, Object> entry : values.entrySet()) { | ||
cloned.put(entry.getKey(), cloneValue(entry.getValue())); | ||
} | ||
return cloned; | ||
} | ||
|
||
private Object cloneValue(final Object value) { | ||
Comment on lines
+773
to
+786
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. Instead of creating all these clone methods, it would be cleaner to use something like Apache Commons 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. In our case the records are 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 for clarifying that. Could this cloning be reused? In other words would it make sense to have a clone/deepClone method in the 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 guess it could, but I'm not sure this is really a pattern we would want to encourage. The scenario here is quite specific. Others may have a different opinion though. |
||
if (value instanceof Record recordValue) { | ||
return cloneRecord(recordValue); | ||
} | ||
if (value instanceof Map<?, ?> mapValue) { | ||
final Map<Object, Object> clonedMap = new LinkedHashMap<>(mapValue.size()); | ||
for (final Map.Entry<?, ?> entry : mapValue.entrySet()) { | ||
clonedMap.put(entry.getKey(), cloneValue(entry.getValue())); | ||
} | ||
return clonedMap; | ||
} | ||
if (value instanceof Collection<?> collectionValue) { | ||
final List<Object> clonedList = new ArrayList<>(collectionValue.size()); | ||
for (final Object element : collectionValue) { | ||
clonedList.add(cloneValue(element)); | ||
} | ||
return clonedList; | ||
} | ||
if (value instanceof Object[] arrayValue) { | ||
final Object[] clonedArray = new Object[arrayValue.length]; | ||
for (int i = 0; i < arrayValue.length; i++) { | ||
clonedArray[i] = cloneValue(arrayValue[i]); | ||
} | ||
return clonedArray; | ||
} | ||
return value; | ||
} | ||
|
||
private class Output { | ||
private FlowFile flowFile; | ||
private final RecordSetWriter writer; | ||
|
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.
Is it necessary to make two clones of the original
record
?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.
Yes it is.
recordSet.next()
reuses the same Record instance each time, so we take one deep copy before adding it toprocessedRecords
. It lets us mutate fields (for example through the variousfieldValue.updateValue(...)
calls) without those changes leaking into later iterations. We take a second deep copy fororiginalRecords
so that, whenindexDocuments()
builds FlowFiles for failures, it can write out an untouched version of the input rather than the mutated version or re-read from the original FlowFile.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.
Thanks for describing the background, that makes sense.