[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16270589#comment-16270589 ] Etienne Chauchot commented on BEAM-3201: [~chet.aldrich] your summary is correct. I'll just precise {quote} Each will require a function that takes in a JSON object and returns a String *value (from the json element provided by the means the user choses)*, which is what will be placed in the corresponding *ES bulk metadata place while inserting* . {quote} > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Chet Aldrich > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16270565#comment-16270565 ] Etienne Chauchot commented on BEAM-3201: Hi [~jkff] My bad, you're right, thanks for pointing out ! > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Chet Aldrich > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16269318#comment-16269318 ] Chet Aldrich commented on BEAM-3201: [~echauchot] [~nerdynick] sounds like we have reached a rough agreement on the design, at least enough for me to start coding something up and show you guys the PR. To summarize: We will keep the API of PCollection. Three optional methods will be added, one for each of the following metadata fields: _id, _index, _type. Each will require a function that takes in a JSON object and returns a String, which is what will be placed in the corresponding metadata field. If any of these methods are called, parse the string into JSON so that each of the methods can use it. Reuse the deserialization for speed. Run the method for each element in the PCollection. I'm going to start coding this up based on what I said above, PR will come soon. Let me know if I'm missing something important, and I'll edit the PR accordingly. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Chet Aldrich > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16269000#comment-16269000 ] Eugene Kirpichov commented on BEAM-3201: Brief comment: "The deserialized object cannot be jackson JSONObject because it is not serializable preventing the 3 with[id|type|index]Fn user defined functions to be called by beam" This is not the case; Beam does not prevent your code from dealing with non-serializable objects. Non-serializable objects just can not be part of the pipeline specification or (without a coder) elements of PCollections. Eg. take a look at how JdbcIO works - the row mappers and statement setters are exactly this - they deal with something non-serializable (a ResultSet), and Beam doesn't prevent them from doing so. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Chet Aldrich > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16268601#comment-16268601 ] Etienne Chauchot commented on BEAM-3201: Hi [~nerdynick]. Ok for the partition transform, it does not fit your use case. Of course deserialization of the json string will be done inside the {{writeFn.ProcessElement}} only once and the deserialized object will be passed to the three {{with[id|type|index]Fn}}. The deserialized object cannot be jackson JSONObject because it is not serializable preventing the 3 {{with[id|type|index]Fn}} user defined functions to be called by beam. We can chose whatever object representation of json as long as it is serializable. The {{with[id|type|index]Fn}} functions will take this object representation as parameter and output {{String}} value (String id value, String index value, String type value) determined by the user out of the object representation of the ES document. Beam will not add or remove metadata _id, _type, _index to the message payload in Read and Write (to avoid deserialize/parse/re-serialize). But if the user wants to add these fields to his documents to get them afterwards in {{with[id|type|index]Fn}} or just determine their value out of other fields it is ok but these fields would be stored as part of the paylaod (leaving the document untouched). > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Chet Aldrich > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16264618#comment-16264618 ] Nicholas Verbeck commented on BEAM-3201: [~echauchot] When dealing with Time series data, as well as other sets of highly dynamic data, in a streaming fashion. The partition approach is just not a practical one. If we only do the function approach then I'd either suggest we change the method signature to look for a JSON object or parse the string provided and give it to each function. The extra CPU overhead for both functions to deserialize the JSON to a usable data model would be a big waste, especially when dealing with large volumes of data. Which is my use-case. If we don't want to go that route then I'd suggest doing both; the field look up and function. This would just leave it up to the engineer to make a choice that suites their use-case and resources better. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Chet Aldrich > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16264477#comment-16264477 ] Etienne Chauchot commented on BEAM-3201: [~nerdynick] regarding what Chet said, and with the knowledge about your use case that you gave, have you tried the {{Partition}} transform? It allows to "split" a PCollection (your unbounded one got after reading from kafka) into several based on a user defined function, allowing then to plug different {{ESIO.Write()}} transforms to write to different index/type. The only thing is that it requires to specify the number of partitions when apply {{Partition}} transform. If it does not fit, then I prefer having 3 user defined functions that specify id, type and index values rather than adding metadata fields to the element stored in the PCollection. Adding them will not break the backward compatibility because {{with[id|type|index]Fn}} will be optional. In the end, at writing time, of course, the payload of the ES doc will not contain the metadata fields, they will be in the regular metadata bulk place {code} { "index" : { "_index" : "test", "_type" : "type1", "_id" : "1" } } {code} Besides keeping PCollection is mainly to avoid breaking the API. You said "Doing the work to deserialize, add fields, serialize is a lot of extra work "; in the user functions you will not need to add fields because the fields would not be in the output payload. You would just need to parse the json string to determine the id, type, index names out of the json on each call to {{processElement}} in the {{ESIO.WriteFn}}. To parse you can use JSonPath of course. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Chet Aldrich > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16263519#comment-16263519 ] Nicholas Verbeck commented on BEAM-3201: [~chetaldrich] I'm not against the user functions. I just feel to really support them efficiently the ESIO.Write() signature would need to change from String to something else; Map, Object, JSONObject, etc. In fact I talked about it when I tried to start the discussions on BEAM-3222 within the dev mailing list. The use-case I'm trying to solve for is not a unique one or a new of one. In most cases, including my own currently, it involves timeseries data. Where you'd bucket the data into separate indexes by day, hour, etc. It'd be impractical to launch separate jobs or define an unlimited list of partitions for each time separation. Especially when streaming data from Kafka. Data shows up late and other issues would make it very difficult if you couldn't change the index/type dynamically as the data flows by. ES already supports this action/use-case with the Bulk API. As well as further enhances the ability to do this with index templates. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Chet Aldrich > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16263472#comment-16263472 ] Chet Aldrich commented on BEAM-3201: [~echauchot] First of all, thanks for getting that all sorted for me. [~nerdynick] {quote}That said, if you want to have dynamic index/type (meaning do not use ConnectionConfiguration.withIndex and ConnectionConfiguration.withType) and also dynamic id depending of the document itself, we should add 3 optional user defined functions so that the user can provide them. I guess it makes the withDocumentIdField(String fieldName) redundant. So we should not implement it.{quote} According to what Etienne said here, it seems like if we want to go this route we may want to rethink the design for this, especially since I agree with him about not polluting the document payload. However, I'm not necessarily sold on why this is necessary in the first place. Could you ([~nerdynick]) elaborate more on why your use case requires dynamically changing the index and type that you're writing on a per-element basis? Why not just split up the elements and write to a separate index via a separate sink with a different `ConnectionConfiguration`? IMHO one write operation should write to only one index, since, for example, it'd be odd to be writing entries to two different DB tables depending on a given element instead of just splitting them up into separate PCollections and _then_ writing them out to the different tables with separate sinks. This opinion is only based on my current understanding of what you're trying to accomplish though. Feel free to enlighten me if an assumption I made about your use case is incorrect. Would appreciate input from both of you on whether this use case is needed, and if it is, whether we should rethink how we're approaching this so we don't pollute the document payload with metadata. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Chet Aldrich > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16260893#comment-16260893 ] Nicholas Verbeck commented on BEAM-3201: [~echauchot] The reason I ask about the _index and _type is to support dynamic index and types. To implement the document id that this story aims to do, the metadata needs to change anyway. So why not change it together and get 3 features out of one. withIndex and withType wouldn't change as those are treated as fallback values when the data is not provided in the metadata, same goes for document id but that would defeat the purpose of the bulk api in this case. We'd just add the withDocumentIdField as described here, as well as two more fields withTypeField and withIndexField. Then only modify the metadata field as needed based on the available data and configuration. If the field lookups fail to find something. They fall back to using the defaults provided. Example metadata for all 3: {code:json} { "index" : { "_index" : "test", "_type" : "type1", "_id" : "1" } } {code} Here are the docs describing what I'm referring to with defaults and provided metadata via the bulk api https://www.elastic.co/guide/en/elasticsearch/reference/5.5/docs-bulk.html I had create BEAM-3222 to implement dynamic types and indexes. As I really need them right now to support auto scaling of ES indexes via index templates. But in researching the work needed for that. I found out both of these stories are fully the same. Just different fields within the same metadata. As for User Funcs for providing the fields. I think the work should just be handled up the pipeline while the document is still in an object form. Doing the work to deserialize, add fields, serialize is a lot of extra work that would slow a pipelines throughput down. This is why I linked the JsonPath project for fetching the field. It doesn't require a full deserialize of the document but token walks it for performance. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Chet Aldrich > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16260696#comment-16260696 ] Etienne Chauchot commented on BEAM-3201: [~chet.aldrich]: jackson JSONObject is not serializable, you can parse the json string. you can use the jackson library that is already included in the io dependencies. In theory, the IO is annotated with {{@Experimental(Experimental.Kind.SOURCE_SINK)}} so we could change the signature, but I'm not convinced it's worth the braking change. Another thing, I needed a PMC to add you to the list of contributors so that I can assign you the ticket. [~iemejia] just did that. [~nerdynick]: _type and _index bulk metadata are not set in {{WriteFn.processElement}} because they are set in the REST call. IMHO we should not polluate the payload of documents with metadata fields. That said, if you want to have dynamic index/type (meaning do not use {{ConnectionConfiguration.withIndex}} and {{ConnectionConfiguration.withType}}) and also dynamic id depending of the document itself, we should add 3 optional user defined functions so that the user can provide them. I guess it makes the {{withDocumentIdField(String fieldName)}} redundant. So we should not implement it. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Etienne Chauchot > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16260193#comment-16260193 ] Nicholas Verbeck commented on BEAM-3201: >From what I dug into for the bull api, they are. Just 2 more fields within the meta data json record, _type and _index to go with the _id fields you'd be adding. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Etienne Chauchot > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16260187#comment-16260187 ] Chet Aldrich commented on BEAM-3201: [~nerdynick] Thanks for the lib reference, I'll look into it. I'll also take a look at dealing with those other cases, I think that should be pretty similar in theory, but I'll confirm. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Etienne Chauchot > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16260188#comment-16260188 ] Chet Aldrich commented on BEAM-3201: Also [~echauchot] you can feel free to assign this to me when you get the chance. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Etienne Chauchot > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16259885#comment-16259885 ] Nicholas Verbeck commented on BEAM-3201: [~chetaldrich] This might be a lib to use for this. https://github.com/json-path/JsonPath Also similar to this I'd like to add support for overriding the index and type. I think in all 3 use cases we could use the same logic. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Etienne Chauchot > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16256316#comment-16256316 ] Chet Aldrich commented on BEAM-3201: Hey, so I'd be happy to take this ticket on, and the design seems reasonable. I have one question about the design above: The API for writing currently is of the form PCollection as defined [here|https://github.com/apache/beam/blob/master/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java#L731], and not PCollection. I suppose we can convert the String that is passed in to a JSONObject or some similar construct and then try to find the field specified in `withDocumentIdField`. I'm assuming that we _don't_ want to change the input type to PCollection, right? We would instead just throw an exception if a String that is passed in is not valid JSON. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Etienne Chauchot > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (BEAM-3201) ElasticsearchIO should deal with documents id
[ https://issues.apache.org/jira/browse/BEAM-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16255513#comment-16255513 ] Etienne Chauchot commented on BEAM-3201: Users have offered to contribute. So, here are my design comments: 1. to keep the API I propose to stick with PCollection 2. We provide a method {{withDocumentIdField(String fieldName)}} in the write transform that allows the user to specify which json field is to be used as document id. The document id field will stay part of the payload to ensure that write then read provides the exact same document. Also keeping the id field part of the payload makes {{read.withDocumentIdField(String fieldName)}} useless, only {{write.withDocumentIdField(String fieldName)}} is needed. 3.That parameter is optional if it is set then If we insert twice a record with same doc id then the document will be automatically updated by ES (no code needed in beam). If it is not provided then id is auto generated on the ES side (same as now) I propose that the {{withDocumentIdField(String fieldName)}} is optional for these reasons: a. backward compatible b. users who do not want to update documents and just want to insert should not have to generate a UUID per document c. ES connector made by ES dev team also chose to make the doc id optional d. forcing to provide a doc id to work around the lack of exactly-once semantic is not a good choice. Exactly-once semantics is a broader discussion in the project than just dealing with it in the datastores. > ElasticsearchIO should deal with documents id > - > > Key: BEAM-3201 > URL: https://issues.apache.org/jira/browse/BEAM-3201 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Reporter: Etienne Chauchot >Assignee: Etienne Chauchot > > Today the ESIO only inserts the payload of the ES documents. Elasticsearch > generates a document id for each record inserted. So each new insertion is > considered as a new document. Users want to be able to update documents using > the IO. So, for the write part of the IO, users should be able to provide a > document id so that they could update already stored documents. Providing an > id for the documents could also help the user on indempotency. -- This message was sent by Atlassian JIRA (v6.4.14#64029)