Hi Chet,

+1 for interest in this from me too.

If it helps, I'd have expected a) to be the implementation (e.g. something
like "_id" being used if present) and handing multiple delivery being a
responsibility of the developer.

Thanks,
Tim




On Wed, Nov 15, 2017 at 10:08 AM, Jean-Baptiste Onofré <j...@nanthrax.net>
wrote:

> I think it's also related to the discussion Romain raised on the dev
> mailing list (gap between batch size, checkpointing & bundles).
>
> Regards
> JB
>
> On 11/15/2017 09:53 AM, Etienne Chauchot wrote:
>
>> Hi Chet,
>>
>> What you say is totally true, docs written using ElasticSearchIO will
>> always have an ES generated id. But it might change in the future, indeed
>> it might be a good thing to allow the user to pass an id. Just in 5 seconds
>> thinking, I see 3 possible designs for that.
>>
>> a.(simplest) use a json special field for the id, if it is provided by
>> the user in the input json then it is used, auto-generated id otherwise.
>>
>> b. (a bit less user friendly) PCollection<KV> with K as an id. But forces
>> the user to do a Pardo before writing to ES to output KV pairs of <id, json>
>>
>> c. (a lot more complex) Allow the IO to serialize/deserialize java beans
>> and have an String id field. Matching java types to ES types is quite
>> tricky, so, for now we just relied on the user to serialize his beans into
>> json and let ES match the types automatically.
>>
>> Related to the problems you raise bellow:
>>
>> 1. Well, the bundle is the commit entity of beam. Consider the case of
>> ESIO.batchSize being < to bundle size. While processing records, when the
>> number of elements reaches batchSize, an ES bulk insert will be issued but
>> no finishBundle. If there is a problem later on in the bundle processing
>> before the finishBundle, the checkpoint will still be at the beginning of
>> the bundle, so all the bundle will be retried leading to duplicate
>> documents. Thanks for raising that! I'm CCing the dev list so that someone
>> could correct me on the checkpointing mecanism if I'm missing something.
>> Besides I'm thinking about forcing the user to provide an id in all cases
>> to workaround this issue.
>>
>> 2. Correct.
>>
>> Best,
>> Etienne
>>
>> Le 15/11/2017 à 02:16, Chet Aldrich a écrit :
>>
>>> Hello all!
>>>
>>> So I’ve been using the ElasticSearchIO sink for a project (unfortunately
>>> it’s Elasticsearch 5.x, and so I’ve been messing around with the latest RC)
>>> and I’m finding that it doesn’t allow for changing the document ID, but
>>> only lets you pass in a record, which means that the document ID is
>>> auto-generated. See this line for what specifically is happening:
>>>
>>> https://github.com/apache/beam/blob/master/sdks/java/io/elas
>>> ticsearch/src/main/java/org/apache/beam/sdk/io/elasticsear
>>> ch/ElasticsearchIO.java#L838
>>>
>>> Essentially the data part of the document is being placed but it doesn’t
>>> allow for other properties, such as the document ID, to be set.
>>>
>>> This leads to two problems:
>>>
>>> 1. Beam doesn’t necessarily guarantee exactly-once execution for a given
>>> item in a PCollection, as I understand it. This means that you may get more
>>> than one record in Elastic for a given item in a PCollection that you pass
>>> in.
>>>
>>> 2. You can’t do partial updates to an index. If you run a batch job
>>> once, and then run the batch job again on the same index without clearing
>>> it, you just double everything in there.
>>>
>>> Is there any good way around this?
>>>
>>> I’d be happy to try writing up a PR for this in theory, but not sure how
>>> to best approach it. Also would like to figure out a way to get around this
>>> in the meantime, if anyone has any ideas.
>>>
>>> Best,
>>>
>>> Chet
>>>
>>> P.S. CCed echauc...@gmail.com <mailto:echauc...@gmail.com> because it
>>> seems like he’s been doing work related to the elastic sink.
>>>
>>>
>>>
>>
> --
> Jean-Baptiste Onofré
> jbono...@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com
>

Reply via email to