This is definitely a bug, classic reverse logic/iteration error, not sure what I was thinking there :(
I took the liberty of filing a JIRA ticket [1], and will take the liberty of fixing it shortly :P Regards, Matt [1] https://issues.apache.org/jira/browse/NIFI-3087 On Tue, Nov 22, 2016 at 5:33 PM, James Wing <[email protected]> wrote: > Andreas, > > I'm not familiar with PutElasticSearch, but the code you point to does > appear strange. It doesn't look like there are any unit tests where > multiple results are returned from ElasticSearch with one or more failures > that would exercise this case. > > L329... > // 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++) { > JsonNode itemNode = itemNodeArray.get(i); > FlowFile flowFile = flowFilesToTransfer.remove(i); > ... > > > Have you had problems with PutElasticSearch? Again, I'm not very familiar > with it, but a workaround might be to keep the batch size at 1 (not ideal). > > Would you be willing to create a JIRA ticket for this issue at > https://issues.apache.org/jira/browse/NIFI? > > > Thanks, > > James > > On Mon, Nov 21, 2016 at 3:28 AM, Andreas Petter (External) > <[email protected]> wrote: >> >> Hello everybody, >> >> >> >> I have a small question concerning the putElasticSearch.java code… >> >> While reading through the code I found the following line… >> https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttp.java#L333 >> >> >> >> Maybe it is a dumb question, but is the index in the remove (i.e. “i”) the >> index it was intended for (think about what happens if run on at least two >> items)? >> >> >> >> Thanks in advance, >> >> Regards, >> >> Andreas >> >> >> >> >> ________________________________ >> >> Este mensaje y sus adjuntos se dirigen exclusivamente a su destinatario, >> puede contener información privilegiada o confidencial y es para uso >> exclusivo de la persona o entidad de destino. Si no es usted. el >> destinatario indicado, queda notificado de que la lectura, utilización, >> divulgación y/o copia sin autorización puede estar prohibida en virtud de la >> legislación vigente. Si ha recibido este mensaje por error, le rogamos que >> nos lo comunique inmediatamente por esta misma vía y proceda a su >> destrucción. >> >> The information contained in this transmission is privileged and >> confidential information intended only for the use of the individual or >> entity named above. If the reader of this message is not the intended >> recipient, you are hereby notified that any dissemination, distribution or >> copying of this communication is strictly prohibited. If you have received >> this transmission in error, do not read it. Please immediately reply to the >> sender that you have received this communication in error and then delete >> it. >> >> Esta mensagem e seus anexos se dirigem exclusivamente ao seu destinatário, >> pode conter informação privilegiada ou confidencial e é para uso exclusivo >> da pessoa ou entidade de destino. Se não é vossa senhoria o destinatário >> indicado, fica notificado de que a leitura, utilização, divulgação e/ou >> cópia sem autorização pode estar proibida em virtude da legislação vigente. >> Se recebeu esta mensagem por erro, rogamos-lhe que nos o comunique >> imediatamente por esta mesma via e proceda a sua destruição > >
