You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2021/03/26 14:24:26 UTC

[GitHub] [beam] egalpin opened a new pull request #14347: Overhaul ElasticsearchIO.Write

egalpin opened a new pull request #14347:
URL: https://github.com/apache/beam/pull/14347


   This change set represents a rather large (and backward compatible) change to the way ElastichsearchIO.Write operates. Presently, the Write transform has 2 responsibilities:
   
   1. Convert input documents into Bulk API entities, serializing based on user settings (partial update, delete, upsert, etc) -> `DocToBulk`
   2. Batch the converted Bulk API entities together and interface with the target ES cluster  -> `BulkIO`
   
   This PR aims to separate these 2 responsibilities into discrete PTransforms to allow for greater flexibility while also maintaining the convenience of the Write transform to perform both document conversion and IO serially. Examples of how the flexibility of separate transforms could be used:
   
   1. Unit testing. It becomes trivial for pipeline developers to ensure that output Bulk API entities for a given set of inputs will produce an expected result, without the need for an available Elasticsearch cluster. 
   2. Flexible options for data backup. Serialized Bulk API entities can be forked and sent to both Elasticsearch and a data lake.
   3. Mirroring data to multiple clusters. Presently, mirroring data to multiple clusters would require duplicate computation.
   4. Better batching with input streams in one job. A job may produce multiple "shapes" of Bulk API entities based on multiple input types, and then "fan-in" all serialized Bulk entities into a single BulkIO transform to improve batching semantics.
   5. Decoupled jobs. Corollary to (3) above. Job(s) could be made to produce Bulk entities and then publish them to a message bus. A distinct job could consume from that message bus and solely be responsible for IO with the target cluster(s).
   6. Easier support for multiple BulkIO semantics.
   
   Expanding on point (6), this PR also introduces a new (optional) way to batch entities for bulk requests: Stateful Processing. Presently, Bulk request size is limited by the lesser of Runner bundle size and `maxBatchSize` user setting. In my experience, bundle sizes are often very small, and can be a small as 1 or 2. When that’s the case, it means Bulk requests contain only 1 or 2 documents, and it’s effectively the same as not using the Bulk API at all. `BulkIOStatefulFn` is made to be compatible with `GroupIntoBatches` which will use entity count and (optionally) elapsed time to create batches much closer to the `maxBatchSize` setting to improve throughput.
   
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   Post-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   <table>
     <thead>
       <tr>
         <th>Lang</th>
         <th>SDK</th>
         <th>ULR</th>
         <th>Dataflow</th>
         <th>Flink</th>
         <th>Samza</th>
         <th>Spark</th>
         <th>Twister2</th>
       </tr>
     </thead>
     <tbody>
       <tr>
         <td>Go</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/">
             <img src="https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
       </tr>
       <tr>
         <td>Java</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_ULR/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_ULR/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_VR_Dataflow_V2_Streaming/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon">
           </a>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Cron/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Java11_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Java_Examples_Dataflow_Java11_Cron/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_Examples_Dataflow_V2/lastCompletedBuild/badge/icon">
           </a><br>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/badge/icon">
           </a>
         </td>
       </tr>
       <tr>
         <td>Python</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
       </tr>
       <tr>
         <td>XLang</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Dataflow/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Dataflow/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
       </tr>
     </tbody>
   </table>
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   <table>
     <thead>
       <tr>
         <th>---</th>
         <th>Java</th>
         <th>Python</th>
         <th>Go</th>
         <th>Website</th>
         <th>Whitespace</th>
         <th>Typescript</th>
       </tr>
     </thead>
     <tbody>
       <tr>
         <td>Non-portable</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon">
           </a><br>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/lastCompletedBuild/">
             <img src="https://camo.githubusercontent.com/4565d7b7e907114e6c1d12323408bd903aa252fefed5eeab93701b05c9628a84/68747470733a2f2f63692d6265616d2e6170616368652e6f72672f6a6f622f6265616d5f507265436f6d6d69745f507974686f6e446f636b65725f43726f6e2f62616467652f69636f6e" alt="Build Status" data-canonical-src="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/badge/icon">
           </a><br>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/lastCompletedBuild/">
             <img src="https://camo.githubusercontent.com/21afb097a5745440598bee5c59a027b140585eec871c3f3b883200610fabf722/68747470733a2f2f63692d6265616d2e6170616368652e6f72672f6a6f622f6265616d5f507265436f6d6d69745f507974686f6e446f63735f43726f6e2f62616467652f69636f6e" alt="Build Status" data-canonical-src="https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
       </tr>
       <tr>
         <td>Portable</td>
         <td>---</td>
         <td>
           <a href="https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/">
             <img alt="Build Status" src="https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon">
           </a>
         </td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
         <td>---</td>
       </tr>
     </tbody>
   </table>
   
   See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
   
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629433650



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -257,6 +259,14 @@ void testWrite() throws Exception {
     executeWriteTest(write);
   }
 
+  void testWriteStateful() throws Exception {

Review comment:
       module tiny fixes in code comments:
   upsert test: LGTM
   routing test: LGTM
   doc version test: LGTM
   

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -532,6 +602,34 @@ void testWriteWithFullAddressing() throws Exception {
     }
   }
 
+  /**
+   * Tests that documents are correctly routed when index, type and document ID functions are
+   * provided to overwrite the defaults of using the configuration and auto-generation of the
+   * document IDs by Elasticsearch. The scientist name is used for the index, type and document ID.
+   * As a result there should be only a single document in each index/type.
+   */

Review comment:
       there should be numdocs / nb scientists for each (per-scientist) index, no ?

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -532,6 +602,34 @@ void testWriteWithFullAddressing() throws Exception {
     }
   }
 
+  /**
+   * Tests that documents are correctly routed when index, type and document ID functions are
+   * provided to overwrite the defaults of using the configuration and auto-generation of the
+   * document IDs by Elasticsearch. The scientist name is used for the index, type and document ID.

Review comment:
       I see no other Fn than routingFn specified, copy paste leftover?
    please correct javadoc

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -568,11 +666,167 @@ void testWritePartialUpdate() throws Exception {
     assertEquals(numDocs, currentNumDocs);
     assertEquals(
         numDocs / NUM_SCIENTISTS,
-        countByScientistName(connectionConfiguration, restClient, "Einstein"));
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
 
     // Partial update assertions
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0"));
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testWriteWithDocVersion() throws Exception {
+    List<ObjectNode> jsonData =
+        ElasticsearchIOTestUtils.createJsonDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    List<String> data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      doc.put("my_version", "1");
+      data.add(doc.toString());
+    }
+
+    insertTestDocuments(connectionConfiguration, data, restClient);
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+    // Check that all docs have the same "my_version"
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withDocVersionFn(new ExtractValueFn("my_version"))
+            .withDocVersionType("external");
+
+    data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      // Set version to larger number than originally set, and larger than next logical version
+      // number set by default by ES.
+      doc.put("my_version", "3");
+      data.add(doc.toString());
+    }
+
+    // Test that documents with lower version are rejected, but rejections ignored when specified
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+
+    // my_version and doc version should have changed
+    assertEquals(

Review comment:
       please assert also on doc version




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] mattwelke commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
mattwelke commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-1019047675


   Came across this while lurking. Where would one find docs on how to use ElasticsearchIO, including more advanced features like this? I saw some examples on the Beam site, but nothing specific to each source/sink.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r620648816



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1203,88 +1796,160 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. If data is globally windowed and this
+     * configuration is set to 1, there will only ever be 1 request in flight. Having only a single
+     * request in flight can be beneficial for ensuring an Elasticsearch cluster is not overwhelmed
+     * by parallel requests, but may not work for all use cases. If this number is less than the
+     * number of maximum workers in your pipeline, the IO work may not be distributed across all
+     * workers.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =
+            GroupIntoBatches.ofSize(getMaxBatchSize());
+
+        if (getMaxBufferingDuration() != null) {
+          groupIntoBatches = groupIntoBatches.withMaxBufferingDuration(getMaxBufferingDuration());
+        }
+        input
+            .apply(ParDo.of(new AssignShardFn<>(getMaxParallelRequestsPerWindow())))

Review comment:
       Tried to make the wording of the warning a little stronger, let me know what you think.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r623766015



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {

Review comment:
       you can't because the check needs the cluster (as you said) and the DoFn needs to have been staged to the beam cluster in  case there are networking firewalling between were pipeline main() is run and were the ES cluster is. In the past the check was done in WriteFn constructor (which runs in main so in spark/flink client or jobmanager) and it showed network issues in some cases. It was put in WriteFn.setup (which is run in taskmanagers) to avoid that. So you need a DoFn and there is no more in Write.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] ludovic-boutros commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
ludovic-boutros commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-818861713


   @egalpin @echauchot I made a quick review, but, well, sadly I don't have enough time these months to go deeper on the subject.
   We are using the module I shared in production for months without any issue.
   The development of a new multi module component is currently in stand by with one module per Elasticsearch version and a top level abstraction.
   It's not finished yet and with the Covid situation I had to refocus on other projects. That means that I will not get any time to really go further soon. I'm available to share on this with you (Slack ?). 
   I think political decision should be taken first before taking the direction I proposed. But it's not the place to have this discussion.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r616672005



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1453,14 +2062,23 @@ private HttpEntity handleRetry(
         // while retry policy exists
         while (BackOffUtils.next(sleeper, backoff)) {
           LOG.warn(String.format(RETRY_ATTEMPT_LOG, ++attempt));
-          Request request = new Request(method, endpoint);
-          request.addParameters(params);
-          request.setEntity(requestBody);
-          response = restClient.performRequest(request);
-          responseEntity = new BufferedHttpEntity(response.getEntity());
+          try {
+            Request request = new Request(method, endpoint);
+            request.addParameters(params);
+            request.setEntity(requestBody);
+            response = restClient.performRequest(request);
+            responseEntity = new BufferedHttpEntity(response.getEntity());
+          } catch (java.io.IOException ex) {

Review comment:
       Before we just threw the exception and there would be retrials only on http 429 (predicate). Now retrials are also done when receiving IOException. Are you sure all IOException cases can be retried ? I'm not sure they are all timeouts: a misconfigured IO will throw IOException and will be retried. It is good to retry on Timeouts IMHO but please filter on only timeouts.

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1310,135 +1970,84 @@ public void startBundle(StartBundleContext context) {
         currentBatchSizeBytes = 0;
       }
 
-      private class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
-
-        private DocumentMetadataSerializer() {
-          super(DocumentMetadata.class);
-        }
-
-        @Override
-        public void serialize(
-            DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
-            throws IOException {
-          gen.writeStartObject();
-          if (value.index != null) {
-            gen.writeStringField("_index", value.index);
-          }
-          if (value.type != null) {
-            gen.writeStringField("_type", value.type);
-          }
-          if (value.id != null) {
-            gen.writeStringField("_id", value.id);
-          }
-          if (value.retryOnConflict != null && (backendVersion <= 6)) {
-            gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
-          }
-          if (value.retryOnConflict != null && backendVersion >= 7) {
-            gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
-          }
-          gen.writeEndObject();
-        }
-      }
-      /**
-       * Extracts the components that comprise the document address from the document using the
-       * {@link FieldValueExtractFn} configured. This allows any or all of the index, type and
-       * document id to be controlled on a per document basis. Sanitization of the index is
-       * performed, automatically lower-casing the value as required by Elasticsearch.
-       *
-       * @param parsedDocument the json from which the index, type and id may be extracted
-       * @return the document address as JSON or the default
-       * @throws IOException if the document cannot be parsed as JSON
-       */
-      private String getDocumentMetadata(JsonNode parsedDocument) throws IOException {
-        DocumentMetadata metadata =
-            new DocumentMetadata(
-                spec.getIndexFn() != null
-                    ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
-                    : null,
-                spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
-                spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
-                spec.getUsePartialUpdate() ? DEFAULT_RETRY_ON_CONFLICT : null);
-        return OBJECT_MAPPER.writeValueAsString(metadata);
-      }
-
-      private static String lowerCaseOrNull(String input) {
-        return input == null ? null : input.toLowerCase();
+      @FinishBundle
+      public void finishBundle(FinishBundleContext context)
+          throws IOException, InterruptedException {
+        flushBatch();
       }
 
       @ProcessElement
-      public void processElement(ProcessContext context) throws Exception {
-        String document = context.element(); // use configuration and auto-generated document IDs
-        String documentMetadata = "{}";
-        boolean isDelete = false;
-        if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
-          // parse once and reused for efficiency
-          JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
-          documentMetadata = getDocumentMetadata(parsedDocument);
-          if (spec.getIsDeleteFn() != null) {
-            isDelete = spec.getIsDeleteFn().apply(parsedDocument);
-          }
+      public void processElement(@Element @NonNull Iterable<String> bulkApiEntities)
+          throws Exception {
+        for (String bulkApiEntity : bulkApiEntities) {
+          addAndMaybeFlush(bulkApiEntity);
         }
+      }
 
-        if (isDelete) {
-          // delete request used for deleting a document.
-          batch.add(String.format("{ \"delete\" : %s }%n", documentMetadata));
-        } else {
-          // index is an insert/upsert and update is a partial update (or insert if not existing)
-          if (spec.getUsePartialUpdate()) {
-            batch.add(
-                String.format(
-                    "{ \"update\" : %s }%n{ \"doc\" : %s, \"doc_as_upsert\" : true }%n",
-                    documentMetadata, document));
-          } else {
-            batch.add(String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document));
-          }
-        }
+      protected void addAndMaybeFlush(String bulkApiEntity)
+          throws IOException, InterruptedException {
+        batch.add(bulkApiEntity);
+        currentBatchSizeBytes += bulkApiEntity.getBytes(StandardCharsets.UTF_8).length;
 
-        currentBatchSizeBytes += document.getBytes(StandardCharsets.UTF_8).length;
         if (batch.size() >= spec.getMaxBatchSize()
             || currentBatchSizeBytes >= spec.getMaxBatchSizeBytes()) {
           flushBatch();
         }
       }
 
-      @FinishBundle
-      public void finishBundle(FinishBundleContext context)
-          throws IOException, InterruptedException {
-        flushBatch();
-      }
-
       private void flushBatch() throws IOException, InterruptedException {
         if (batch.isEmpty()) {
           return;
         }
+
+        LOG.info(
+            "ElasticsearchIO batch size: {}, batch size bytes: {}",
+            batch.size(),
+            currentBatchSizeBytes);
+
         StringBuilder bulkRequest = new StringBuilder();
         for (String json : batch) {
           bulkRequest.append(json);
         }
+
         batch.clear();
-        currentBatchSizeBytes = 0;
-        Response response;
-        HttpEntity responseEntity;
-        // Elasticsearch will default to the index/type provided here if none are set in the
-        // document meta (i.e. using ElasticsearchIO$Write#withIndexFn and
-        // ElasticsearchIO$Write#withTypeFn options)
-        String endPoint =
-            String.format(
-                "/%s/%s/_bulk",
-                spec.getConnectionConfiguration().getIndex(),
-                spec.getConnectionConfiguration().getType());
+        currentBatchSizeBytes = 0L;
+
+        Response response = null;
+        HttpEntity responseEntity = null;
+
+        // Elasticsearch will default to the index/type provided the {@link
+        // ConnectionConfiguration} if none are set in the document meta (i.e.
+        // using ElasticsearchIO$DocToBulk#withIndexFn and
+        // ElasticsearchIO$DocToBulk#withTypeFn options)
+        String endPoint = spec.getConnectionConfiguration().getBulkEndPoint();
+
         HttpEntity requestBody =
             new NStringEntity(bulkRequest.toString(), ContentType.APPLICATION_JSON);
-        Request request = new Request("POST", endPoint);
-        request.addParameters(Collections.emptyMap());
-        request.setEntity(requestBody);
-        response = restClient.performRequest(request);
-        responseEntity = new BufferedHttpEntity(response.getEntity());
+        try {
+          Request request = new Request("POST", endPoint);
+          request.addParameters(Collections.emptyMap());
+          request.setEntity(requestBody);
+          response = restClient.performRequest(request);
+          responseEntity = new BufferedHttpEntity(response.getEntity());
+        } catch (java.io.IOException ex) {

Review comment:
       same here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r640604970



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -532,6 +602,34 @@ void testWriteWithFullAddressing() throws Exception {
     }
   }
 
+  /**
+   * Tests that documents are correctly routed when index, type and document ID functions are
+   * provided to overwrite the defaults of using the configuration and auto-generation of the
+   * document IDs by Elasticsearch. The scientist name is used for the index, type and document ID.
+   * As a result there should be only a single document in each index/type.
+   */

Review comment:
       oops, missed it, thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-839319649


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r617002718



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {

Review comment:
       Hmm that's a good question, I hadn't quite considered that. What are your thoughts on adding a note in docstring for `withBackendVersion` stating that its value will take precedence and will preclude reading the version from the cluster? Then we would need to trust users to select what fits their need beyond that point.  Thoughts?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-829994089


   @egalpin tip: please don't squash until the review is finished otherwise difficult to see the updates.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-813443632


   @echauchot I've made a jira ticket and linked it. I'm working on getting the build to pass but struggling a bit with trying to determine the cause of the new errors in the Java PreCommit build. All the warnings have to do with a single Kotlin example which seems far removed from the changes here. I'll keep poking away at it though.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-827049282


   Run Spotless PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-847905635


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-1019264557


   @mattwelke The javadoc[1] that is generated has some description and examples, though admittedly not as fully descriptive as it could be. I would welcome any additional examples or documentation added by others, and will keep in mind to add more examples when time permits!
   
   Do you have any specific questions or a use case you would like help determining how to best use this IO? I might suggest that we move the conversation to the user mailing list or slack[2] so that others could more easily benefit from our conversation πŸ™‚
   
   [1] https://beam.apache.org/releases/javadoc/2.35.0/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.html
   
   [2] https://beam.apache.org/community/join-beam/


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin edited a comment on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin edited a comment on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-813443632


   @echauchot I've made a jira ticket and linked it. ~I'm working on getting the build to pass but struggling a bit with trying to determine the cause of the new errors in the Java PreCommit build. All the warnings have to do with a single Kotlin example which seems far removed from the changes here. I'll keep poking away at it though.~ I've found the full output from the build and can see some compilation errors. Working on those.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin edited a comment on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin edited a comment on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-817842196


   Thanks @echauchot for sharing that thread. I really like a lot of what @ludovic-boutros proposed and have many shared goals; in particular, implementing the pattern where successful and failed writes can be returned via `MultiOutputReceiver`. I believe the multi-output pattern could fit within the current IO with some additional effort (since order of request and response entities is guaranteed[1]), and I had planned to do that as a follow-up so as to not introduce even more changes in one PR.
   
   At the same time, I also see the argument that in many ways using the low-level client results in "reinventing the wheel" for a number of features (with good justification, IMO, of enabling cross-version support).
   
   I'd be very willing to contribute to brainstorming (and implementation once we reach that point) if others are open to that.
   
   [1] https://discuss.elastic.co/t/order-of-actions-in-bulk-api-via-http-between-request-and-response-is-guaranteed/122499/2


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r616674237



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -265,6 +275,13 @@ void testWriteWithErrors() throws Exception {
     List<String> input =
         ElasticsearchIOTestUtils.createDocuments(
             numDocs, ElasticsearchIOTestUtils.InjectionMode.INJECT_SOME_INVALID_DOCS);
+

Review comment:
       duplicated lines is write tests




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-822295067


   > This change set represents a rather large (and backward compatible) change to the way ElastichsearchIO.Write operates. Presently, the Write transform has 2 responsibilities:
   > 
   > 1. Convert input documents into Bulk API entities, serializing based on user settings (partial update, delete, upsert, etc) -> `DocToBulk`
   > 2. Batch the converted Bulk API entities together and interface with the target ES cluster  -> `BulkIO`
   > 
   > This PR aims to separate these 2 responsibilities into discrete PTransforms to allow for greater flexibility while also maintaining the convenience of the Write transform to perform both document conversion and IO serially. Examples of how the flexibility of separate transforms could be used:
   > 
   > 1. Unit testing. It becomes trivial for pipeline developers to ensure that output Bulk API entities for a given set of inputs will produce an expected result, without the need for an available Elasticsearch cluster.
   > 2. Flexible options for data backup. Serialized Bulk API entities can be forked and sent to both Elasticsearch and a data lake.
   > 3. Mirroring data to multiple clusters. Presently, mirroring data to multiple clusters would require duplicate computation.
   > 4. Better batching with input streams in one job. A job may produce multiple "shapes" of Bulk API entities based on multiple input types, and then "fan-in" all serialized Bulk entities into a single BulkIO transform to improve batching semantics.
   > 5. Decoupled jobs. Corollary to (4) above. Job(s) could be made to produce Bulk entities and then publish them to a message bus. A distinct job could consume from that message bus and solely be responsible for IO with the target cluster(s).
   > 6. Easier support for multiple BulkIO semantics.
   > 
   
   => Reading at the overall design goals, it looks very promising and a good analysis of the missing properties of the curent architecture ! Thanks !
   
   > Expanding on point (6), this PR also introduces a new (optional) way to batch entities for bulk requests: Stateful Processing. Presently, Bulk request size is limited by the lesser of Runner bundle size and `maxBatchSize` user setting. In my experience, bundle sizes are often very small, and can be a small as 1 or 2. When that’s the case, it means Bulk requests contain only 1 or 2 documents, and it’s effectively the same as not using the Bulk API at all. `BulkIOStatefulFn` is made to be compatible with `GroupIntoBatches` which will use entity count and (optionally) elapsed time to create batches much closer to the `maxBatchSize` setting to improve throughput.
   
   => True that very small batches can exist for example Flink being a streaming oriented platform, Flink runner tends to create very small Beam bundles. So, when the bundle is finished processing (finishBundle is called), the ES bulk request is sent leading to small ES bulk. Leveraging _GroupIntoBatches_ that creates trans-bundle groups and still respect Beam semantics (windowing, bundle retries etc...) is a very good idea.
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-827480125


   @echauchot @pabloem Any chance this PR makes the 2.30.0 cut? It would be awesome to be in that release! 😁


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-825285997


   Thanks for the 2nd round @echauchot ! Working my way through your comments and adding test coverage. Seeing some of behaviour in the tests, I may have to reach out with questions if I can’t get to the bottom of it. 
   
   Somewhat related: can you describe the high-level difference between the IOTest Vs. IOIT files?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629347751



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1203,88 +1796,160 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. If data is globally windowed and this
+     * configuration is set to 1, there will only ever be 1 request in flight. Having only a single
+     * request in flight can be beneficial for ensuring an Elasticsearch cluster is not overwhelmed
+     * by parallel requests, but may not work for all use cases. If this number is less than the
+     * number of maximum workers in your pipeline, the IO work may not be distributed across all
+     * workers.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =
+            GroupIntoBatches.ofSize(getMaxBatchSize());
+
+        if (getMaxBufferingDuration() != null) {
+          groupIntoBatches = groupIntoBatches.withMaxBufferingDuration(getMaxBufferingDuration());
+        }
+        input
+            .apply(ParDo.of(new AssignShardFn<>(getMaxParallelRequestsPerWindow())))

Review comment:
       good




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r637044281



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -532,6 +602,34 @@ void testWriteWithFullAddressing() throws Exception {
     }
   }
 
+  /**
+   * Tests that documents are correctly routed when index, type and document ID functions are
+   * provided to overwrite the defaults of using the configuration and auto-generation of the
+   * document IDs by Elasticsearch. The scientist name is used for the index, type and document ID.

Review comment:
       same as the other it asserts count = numDocs / NUM_SCIENTISTS so javadoc is to be updated no ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r617016681



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {
+          backendVersion = ElasticsearchIO.getBackendVersion(connectionConfiguration);
+        } else {
+          backendVersion = spec.getBackendVersion();
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext c) throws IOException {
+        c.output(createBulkApiEntity(spec, c.element(), backendVersion));
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransform} convenience wrapper for doing both document to bulk API serialization as
+   * well as batching those Bulk API entities and writing them to an Elasticsearch cluster. This
+   * class is effectively a thin proxy for DocToBulk->BulkIO all-in-one for convenience and backward
+   * compatibility.
+   */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<String>, PDone> {
+    public interface FieldValueExtractFn extends SerializableFunction<JsonNode, String> {}
+
+    public interface BooleanFieldValueExtractFn extends SerializableFunction<JsonNode, Boolean> {}
+
+    public abstract DocToBulk getDocToBulk();
+
+    public abstract BulkIO getBulkIO();
+
+    abstract Builder writeBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setDocToBulk(DocToBulk docToBulk);
+
+      abstract Builder setBulkIO(BulkIO bulkIO);
+
+      abstract Write build();
+    }
+
+    // For building Doc2Bulk
+    /** Refer to {@link DocToBulk#withIdFn}. */
+    public Write withIdFn(FieldValueExtractFn idFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIdFn(idFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withIndexFn}. */
+    public Write withIndexFn(FieldValueExtractFn indexFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIndexFn(indexFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withRoutingFn}. */
+    public Write withRoutingFn(FieldValueExtractFn routingFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withRoutingFn(routingFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withTypeFn}. */
+    public Write withTypeFn(FieldValueExtractFn typeFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withTypeFn(typeFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withDocVersionFn}. */
+    public Write withDocVersionFn(FieldValueExtractFn docVersionFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withDocVersionFn(docVersionFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withDocVersionType}. */
+    public Write withDocVersionType(String docVersionType) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withDocVersionType(docVersionType)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withUsePartialUpdate}. */
+    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+      return writeBuilder()
+          .setDocToBulk(getDocToBulk().withUsePartialUpdate(usePartialUpdate))
+          .build();
+    }
+
+    /** Refer to {@link DocToBulk#withUpsertScript}. */
+    public Write withUpsertScript(String source) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withUpsertScript(source)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withBackendVersion}. */
+    public Write withBackendVersion(int backendVersion) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withBackendVersion(backendVersion)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withIsDeleteFn}. */
+    public Write withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIsDeleteFn(isDeleteFn)).build();
+    }
+    // End building Doc2Bulk
+
+    /** Refer to {@link BulkIO#withConnectionConfiguration}. */
+    public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) {
+      checkArgument(connectionConfiguration != null, "connectionConfiguration can not be null");
+
+      return writeBuilder()
+          .setDocToBulk(getDocToBulk().withConnectionConfiguration(connectionConfiguration))
+          .setBulkIO(getBulkIO().withConnectionConfiguration(connectionConfiguration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBatchSize}. */
+    public Write withMaxBatchSize(long batchSize) {
+      return writeBuilder().setBulkIO(getBulkIO().withMaxBatchSize(batchSize)).build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBatchSizeBytes}. */
+    public Write withMaxBatchSizeBytes(long batchSizeBytes) {
+      return writeBuilder().setBulkIO(getBulkIO().withMaxBatchSizeBytes(batchSizeBytes)).build();
+    }
+
+    /** Refer to {@link BulkIO#withRetryConfiguration}. */
+    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withRetryConfiguration(retryConfiguration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withIgnoreVersionConflicts}. */
+    public Write withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withIgnoreVersionConflicts(ignoreVersionConflicts))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withUseStatefulBatches}. */
+    public Write withUseStatefulBatches(boolean useStatefulBatches) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withUseStatefulBatches(useStatefulBatches))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBufferingDuration}. */
+    public Write withMaxBufferingDuration(Duration maxBufferingDuration) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withMaxBufferingDuration(maxBufferingDuration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxParallelRequestsPerWindow}. */
+    public Write withMaxParallelRquestsPerWindow(int maxParallelRquestsPerWindow) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withMaxParallelRequestsPerWindow(maxParallelRquestsPerWindow))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withAllowableResponseErrors}. */
+    public Write withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrors) {

Review comment:
       It seemed reasonable that other error cases might be acceptable for different use cases, so the result here is a generalized version




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r637042073



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -532,6 +602,34 @@ void testWriteWithFullAddressing() throws Exception {
     }
   }
 
+  /**
+   * Tests that documents are correctly routed when index, type and document ID functions are
+   * provided to overwrite the defaults of using the configuration and auto-generation of the
+   * document IDs by Elasticsearch. The scientist name is used for the index, type and document ID.
+   * As a result there should be only a single document in each index/type.
+   */

Review comment:
       I think you missed to fix this javadoc




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-831261188


   @echauchot @pabloem ready for another round of reviews πŸ™‚ 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r640604970



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -532,6 +602,34 @@ void testWriteWithFullAddressing() throws Exception {
     }
   }
 
+  /**
+   * Tests that documents are correctly routed when index, type and document ID functions are
+   * provided to overwrite the defaults of using the configuration and auto-generation of the
+   * document IDs by Elasticsearch. The scientist name is used for the index, type and document ID.
+   * As a result there should be only a single document in each index/type.
+   */

Review comment:
       oups, missed it, thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629328260



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -158,10 +167,16 @@
 })
 public class ElasticsearchIO {

Review comment:
       Right now, the main ESIO javadoc lists only part of the options. I think that having all the with* options in main javadoc is source of out of date javadocs. Please update both read and write to specify only mandatory option such as withConfiguration and mention that the IO can be configured in different ways see the with* methods.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-808366820


   Sure, I'll be happy to review. Thanks for your contribution !


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629433650



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -257,6 +259,14 @@ void testWrite() throws Exception {
     executeWriteTest(write);
   }
 
+  void testWriteStateful() throws Exception {

Review comment:
       modulo minor fixes in code comments:
   upsert test: LGTM
   routing test: LGTM
   doc version test: LGTM
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-827273065


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-817842196


   Thanks @echauchot for sharing that thread. I really like a lot of what @ludovic-boutros proposed and have many shared goals; in particular, implementing the pattern where successful and failed writes can be returned via `MultiOutputReceiver`. I believe the multi-output pattern could fit within the current IO with some additional effort, and I had planned to do that as a follow-up so as to not introduce even more changes in one PR.
   
   At the same time, I also see the argument that in many ways using the low-level client results in "reinventing the wheel" for a number of features (with good justification, IMO, of enabling cross-version support).
   
   I'd be very willing to contribute to brainstorming (and implementation once we reach that point) if others are open to that.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629978211



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -568,11 +664,167 @@ void testWritePartialUpdate() throws Exception {
     assertEquals(numDocs, currentNumDocs);
     assertEquals(
         numDocs / NUM_SCIENTISTS,
-        countByScientistName(connectionConfiguration, restClient, "Einstein"));
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
 
     // Partial update assertions
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0"));
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testWriteWithDocVersion() throws Exception {
+    List<ObjectNode> jsonData =
+        ElasticsearchIOTestUtils.createJsonDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    List<String> data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      doc.put("my_version", "1");
+      data.add(doc.toString());
+    }
+
+    insertTestDocuments(connectionConfiguration, data, restClient);
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+    // Check that all docs have the same "my_version"
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withDocVersionFn(new ExtractValueFn("my_version"))
+            .withDocVersionType("external");
+
+    data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      // Set version to larger number than originally set, and larger than next logical version
+      // number set by default by ES.
+      doc.put("my_version", "3");
+      data.add(doc.toString());
+    }
+
+    // Test that documents with lower version are rejected, but rejections ignored when specified
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+
+    // my_version and doc version should have changed
+    assertEquals(
+        0,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "3", null, KV.of(3, numDocs)));
+  }
+
+  /**
+   * Tests upsert script by adding a group field to each document in the standard test set. The
+   * group field is populated as the modulo 2 of the document id allowing for a test to ensure the
+   * documents are split into 2 groups.
+   */
+  void testWriteScriptedUpsert() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withUpsertScript(SCRIPT_SOURCE);
+
+    // Test that documents can be inserted/created by using withUpsertScript
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    // defensive coding to ensure our initial state is as expected
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // All docs should have have group = 0 added by the script upon creation
+    assertEquals(
+        numDocs, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+
+    // Run the same data again. This time, because all docs exist in the index already, scripted
+    // updates should happen rather than scripted inserts.
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // The script will set either 0 or 1 for the group value on update operations
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testMaxParallelRequestsPerWindow() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withMaxParallelRequestsPerWindow(1);
+
+    PCollection<KV<Integer, Long>> batches =
+        pipeline
+            .apply(Create.of(data))
+            .apply(Window.into(new GlobalWindows()))

Review comment:
       should not be needed: it is the default window when none is specified.

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -568,11 +664,167 @@ void testWritePartialUpdate() throws Exception {
     assertEquals(numDocs, currentNumDocs);
     assertEquals(
         numDocs / NUM_SCIENTISTS,
-        countByScientistName(connectionConfiguration, restClient, "Einstein"));
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
 
     // Partial update assertions
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0"));
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testWriteWithDocVersion() throws Exception {
+    List<ObjectNode> jsonData =
+        ElasticsearchIOTestUtils.createJsonDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    List<String> data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      doc.put("my_version", "1");
+      data.add(doc.toString());
+    }
+
+    insertTestDocuments(connectionConfiguration, data, restClient);
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+    // Check that all docs have the same "my_version"
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withDocVersionFn(new ExtractValueFn("my_version"))
+            .withDocVersionType("external");
+
+    data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      // Set version to larger number than originally set, and larger than next logical version
+      // number set by default by ES.
+      doc.put("my_version", "3");
+      data.add(doc.toString());
+    }
+
+    // Test that documents with lower version are rejected, but rejections ignored when specified
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+
+    // my_version and doc version should have changed
+    assertEquals(
+        0,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "3", null, KV.of(3, numDocs)));
+  }
+
+  /**
+   * Tests upsert script by adding a group field to each document in the standard test set. The
+   * group field is populated as the modulo 2 of the document id allowing for a test to ensure the
+   * documents are split into 2 groups.
+   */
+  void testWriteScriptedUpsert() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withUpsertScript(SCRIPT_SOURCE);
+
+    // Test that documents can be inserted/created by using withUpsertScript
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    // defensive coding to ensure our initial state is as expected
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // All docs should have have group = 0 added by the script upon creation
+    assertEquals(
+        numDocs, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+
+    // Run the same data again. This time, because all docs exist in the index already, scripted
+    // updates should happen rather than scripted inserts.
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // The script will set either 0 or 1 for the group value on update operations
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testMaxParallelRequestsPerWindow() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withMaxParallelRequestsPerWindow(1);
+
+    PCollection<KV<Integer, Long>> batches =
+        pipeline
+            .apply(Create.of(data))
+            .apply(Window.into(new GlobalWindows()))
+            .apply(StatefulBatching.fromSpec(write.getBulkIO()))
+            .apply(Count.perKey());
+
+    // Number of unique keys produced should be number of maxParallelRequestsPerWindow * numWindows
+    // There is only 1 request (key) per window, and 1 (global) window ie. one key total
+    PAssert.that(batches).containsInAnyOrder(Collections.singletonList(KV.of(0, 1L)));
+
+    pipeline.run();
+  }
+
+  void testMaxBufferingDurationAndMaxParallelRequestsPerWindow() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withMaxBufferingDuration(Duration.standardSeconds(1))
+            .withMaxParallelRequestsPerWindow(1);
+
+    PCollection<KV<Integer, Long>> batches =
+        pipeline
+            .apply(Create.of(data))
+            .apply(Window.into(new GlobalWindows()))

Review comment:
       ditto

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1246,88 +1877,208 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. BE AWARE that low values for @param
+     * maxParallelRequestsPerWindow, in particular if the input data has a finite number of windows,
+     * can reduce parallelism greatly. If data is globally windowed and @param
+     * maxParallelRequestsPerWindow is set to 1,there will only ever be 1 request in flight. Having
+     * only a single request in flight can be beneficial for ensuring an Elasticsearch cluster is
+     * not overwhelmed by parallel requests,but may not work for all use cases. If this number is
+     * less than the number of maximum workers in your pipeline, the IO work will result in a
+     * sub-distribution of the last write step with most of the runners.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
+    }
+
+    /**
+     * Creates batches of documents using Stateful Processing based on user configurable settings of
+     * withMaxBufferingDuration and withMaxParallelRequestsPerWindow.
+     *
+     * <p>Mostly exists for testability of withMaxParallelRequestsPerWindow.
+     */
+    @VisibleForTesting
+    static class StatefulBatching
+        extends PTransform<PCollection<String>, PCollection<KV<Integer, Iterable<String>>>> {
+      final BulkIO spec;
+
+      private StatefulBatching(BulkIO bulkSpec) {
+        spec = bulkSpec;
+      }
+
+      public static StatefulBatching fromSpec(BulkIO spec) {
+        return new StatefulBatching(spec);
+      }
+
+      @Override
+      public PCollection<KV<Integer, Iterable<String>>> expand(PCollection<String> input) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =
+            GroupIntoBatches.ofSize(spec.getMaxBatchSize());
+
+        if (spec.getMaxBufferingDuration() != null) {
+          groupIntoBatches =
+              groupIntoBatches.withMaxBufferingDuration(spec.getMaxBufferingDuration());
+        }
+
+        return input
+            .apply(ParDo.of(new Reshuffle.AssignShardFn<>(spec.getMaxParallelRequestsPerWindow())))
+            .apply(groupIntoBatches);
+      }
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        input.apply(StatefulBatching.fromSpec(this)).apply(ParDo.of(new BulkIOStatefulFn(this)));
+      } else {
+        input.apply(ParDo.of(new BulkIOBundleFn(this)));
+      }
       return PDone.in(input.getPipeline());
     }
 
-    /** {@link DoFn} to for the {@link Write} transform. */
-    @VisibleForTesting
-    static class WriteFn extends DoFn<String, Void> {
-      private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
-      private static final int DEFAULT_RETRY_ON_CONFLICT = 5; // race conditions on updates
+    static class BulkIOBundleFn extends BulkIOBaseFn<String> {
+      @VisibleForTesting
+      BulkIOBundleFn(BulkIO bulkSpec) {
+        super(bulkSpec);
+      }
 
-      private static final Duration RETRY_INITIAL_BACKOFF = Duration.standardSeconds(5);
+      @ProcessElement
+      public void processElement(ProcessContext context) throws Exception {
+        String bulkApiEntity = context.element();
+        addAndMaybeFlush(bulkApiEntity);
+      }
+    }
 
+    /*
+    Intended for use in conjunction with {@link GroupIntoBatches}
+     */
+    static class BulkIOStatefulFn extends BulkIOBaseFn<KV<Integer, Iterable<String>>> {
       @VisibleForTesting
-      static final String RETRY_ATTEMPT_LOG = "Error writing to Elasticsearch. Retry attempt[%d]";
+      BulkIOStatefulFn(BulkIO bulkSpec) {
+        super(bulkSpec);
+      }
 
-      @VisibleForTesting
-      static final String RETRY_FAILED_LOG =
-          "Error writing to ES after %d attempt(s). No more attempts allowed";
+      @ProcessElement
+      public void processElement(ProcessContext context) throws Exception {
+        Iterable<String> bulkApiEntities = context.element().getValue();
+        for (String bulkApiEntity : bulkApiEntities) {
+          addAndMaybeFlush(bulkApiEntity);
+        }
+      }
+    }
+
+    /** {@link DoFn} to for the {@link BulkIO} transform. */
+    @VisibleForTesting
+    private abstract static class BulkIOBaseFn<T> extends DoFn<T, Void> {
+      private static final Duration RETRY_INITIAL_BACKOFF = Duration.standardSeconds(5);
 
       private transient FluentBackoff retryBackoff;
 
-      private int backendVersion;
-      private final Write spec;
+      private BulkIO spec;
       private transient RestClient restClient;
-      private ArrayList<String> batch;
-      private long currentBatchSizeBytes;
-
-      // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
-      private static class DocumentMetadata implements Serializable {
-        final String index;
-        final String type;
-        final String id;
-        final Integer retryOnConflict;
-
-        DocumentMetadata(String index, String type, String id, Integer retryOnConflict) {
-          this.index = index;
-          this.type = type;
-          this.id = id;
-          this.retryOnConflict = retryOnConflict;
-        }
-      }
+      protected ArrayList<String> batch;

Review comment:
       can be private

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -568,11 +664,167 @@ void testWritePartialUpdate() throws Exception {
     assertEquals(numDocs, currentNumDocs);
     assertEquals(
         numDocs / NUM_SCIENTISTS,
-        countByScientistName(connectionConfiguration, restClient, "Einstein"));
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
 
     // Partial update assertions
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0"));
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testWriteWithDocVersion() throws Exception {
+    List<ObjectNode> jsonData =
+        ElasticsearchIOTestUtils.createJsonDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    List<String> data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      doc.put("my_version", "1");
+      data.add(doc.toString());
+    }
+
+    insertTestDocuments(connectionConfiguration, data, restClient);
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+    // Check that all docs have the same "my_version"
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withDocVersionFn(new ExtractValueFn("my_version"))
+            .withDocVersionType("external");
+
+    data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      // Set version to larger number than originally set, and larger than next logical version
+      // number set by default by ES.
+      doc.put("my_version", "3");
+      data.add(doc.toString());
+    }
+
+    // Test that documents with lower version are rejected, but rejections ignored when specified
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+
+    // my_version and doc version should have changed
+    assertEquals(
+        0,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "3", null, KV.of(3, numDocs)));
+  }
+
+  /**
+   * Tests upsert script by adding a group field to each document in the standard test set. The
+   * group field is populated as the modulo 2 of the document id allowing for a test to ensure the
+   * documents are split into 2 groups.
+   */
+  void testWriteScriptedUpsert() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withUpsertScript(SCRIPT_SOURCE);
+
+    // Test that documents can be inserted/created by using withUpsertScript
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    // defensive coding to ensure our initial state is as expected
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // All docs should have have group = 0 added by the script upon creation
+    assertEquals(
+        numDocs, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+
+    // Run the same data again. This time, because all docs exist in the index already, scripted
+    // updates should happen rather than scripted inserts.
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // The script will set either 0 or 1 for the group value on update operations
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testMaxParallelRequestsPerWindow() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withMaxParallelRequestsPerWindow(1);
+
+    PCollection<KV<Integer, Long>> batches =
+        pipeline
+            .apply(Create.of(data))
+            .apply(Window.into(new GlobalWindows()))
+            .apply(StatefulBatching.fromSpec(write.getBulkIO()))
+            .apply(Count.perKey());
+
+    // Number of unique keys produced should be number of maxParallelRequestsPerWindow * numWindows
+    // There is only 1 request (key) per window, and 1 (global) window ie. one key total
+    PAssert.that(batches).containsInAnyOrder(Collections.singletonList(KV.of(0, 1L)));

Review comment:
       It is not testing what you think it tests: `.apply(StatefulBatching.fromSpec(write.getBulkIO()))` outputs a `PColection<KV<K, Iterable<V>>>` with a single key (as the sharded key was specified to 1 key). So the collection contains only one key (0) with an iterable associated to it. So count says that there is only a single element (the iterable) for key 0. You are not testing the content of the iterable but the number of iterables associated to the 0 key. So please add also a test of the content of the iterable. For that you need to do: `PAssert.that("error message", collection)
           .satisfies(function)` and implement the function with asserts.

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -568,11 +664,167 @@ void testWritePartialUpdate() throws Exception {
     assertEquals(numDocs, currentNumDocs);
     assertEquals(
         numDocs / NUM_SCIENTISTS,
-        countByScientistName(connectionConfiguration, restClient, "Einstein"));
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
 
     // Partial update assertions
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0"));
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testWriteWithDocVersion() throws Exception {
+    List<ObjectNode> jsonData =
+        ElasticsearchIOTestUtils.createJsonDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    List<String> data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      doc.put("my_version", "1");
+      data.add(doc.toString());
+    }
+
+    insertTestDocuments(connectionConfiguration, data, restClient);
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+    // Check that all docs have the same "my_version"
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withDocVersionFn(new ExtractValueFn("my_version"))
+            .withDocVersionType("external");
+
+    data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      // Set version to larger number than originally set, and larger than next logical version
+      // number set by default by ES.
+      doc.put("my_version", "3");
+      data.add(doc.toString());
+    }
+
+    // Test that documents with lower version are rejected, but rejections ignored when specified
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+
+    // my_version and doc version should have changed
+    assertEquals(
+        0,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "3", null, KV.of(3, numDocs)));
+  }
+
+  /**
+   * Tests upsert script by adding a group field to each document in the standard test set. The
+   * group field is populated as the modulo 2 of the document id allowing for a test to ensure the
+   * documents are split into 2 groups.
+   */
+  void testWriteScriptedUpsert() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withUpsertScript(SCRIPT_SOURCE);
+
+    // Test that documents can be inserted/created by using withUpsertScript
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    // defensive coding to ensure our initial state is as expected
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // All docs should have have group = 0 added by the script upon creation
+    assertEquals(
+        numDocs, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+
+    // Run the same data again. This time, because all docs exist in the index already, scripted
+    // updates should happen rather than scripted inserts.
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // The script will set either 0 or 1 for the group value on update operations
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testMaxParallelRequestsPerWindow() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withMaxParallelRequestsPerWindow(1);
+
+    PCollection<KV<Integer, Long>> batches =
+        pipeline
+            .apply(Create.of(data))
+            .apply(Window.into(new GlobalWindows()))
+            .apply(StatefulBatching.fromSpec(write.getBulkIO()))
+            .apply(Count.perKey());
+
+    // Number of unique keys produced should be number of maxParallelRequestsPerWindow * numWindows
+    // There is only 1 request (key) per window, and 1 (global) window ie. one key total
+    PAssert.that(batches).containsInAnyOrder(Collections.singletonList(KV.of(0, 1L)));
+
+    pipeline.run();
+  }
+
+  void testMaxBufferingDurationAndMaxParallelRequestsPerWindow() throws Exception {

Review comment:
       unless I missed something, I think it is not needed as comparing to the other parallel test, all that it tests is the fact that maxDuration works (which was already tested when the GroupIntoBatches PR was merged)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r638766762



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -532,6 +602,34 @@ void testWriteWithFullAddressing() throws Exception {
     }
   }
 
+  /**
+   * Tests that documents are correctly routed when index, type and document ID functions are
+   * provided to overwrite the defaults of using the configuration and auto-generation of the
+   * document IDs by Elasticsearch. The scientist name is used for the index, type and document ID.
+   * As a result there should be only a single document in each index/type.
+   */

Review comment:
       @echauchot should be fixed as of https://github.com/apache/beam/pull/14347/commits/7c1ed248f20ef42955e562aed7cd9fa5428c3b27




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r637774056



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -568,11 +670,202 @@ void testWritePartialUpdate() throws Exception {
     assertEquals(numDocs, currentNumDocs);
     assertEquals(
         numDocs / NUM_SCIENTISTS,
-        countByScientistName(connectionConfiguration, restClient, "Einstein"));
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
 
     // Partial update assertions
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0"));
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testWriteWithDocVersion() throws Exception {
+    List<ObjectNode> jsonData =
+        ElasticsearchIOTestUtils.createJsonDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    List<String> data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      doc.put("my_version", "1");
+      data.add(doc.toString());
+    }
+
+    insertTestDocuments(connectionConfiguration, data, restClient);
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+    // Check that all docs have the same "my_version"
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withDocVersionFn(new ExtractValueFn("my_version"))
+            .withDocVersionType("external");
+
+    data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      // Set version to larger number than originally set, and larger than next logical version
+      // number set by default by ES.
+      doc.put("my_version", "3");
+      data.add(doc.toString());
+    }
+
+    // Test that documents with lower version are rejected, but rejections ignored when specified
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+
+    // my_version and doc version should have changed
+    assertEquals(
+        0,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "3", null, KV.of(3, numDocs)));
+  }
+
+  /**
+   * Tests upsert script by adding a group field to each document in the standard test set. The
+   * group field is populated as the modulo 2 of the document id allowing for a test to ensure the
+   * documents are split into 2 groups.
+   */
+  void testWriteScriptedUpsert() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withUpsertScript(SCRIPT_SOURCE);
+
+    // Test that documents can be inserted/created by using withUpsertScript
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    // defensive coding to ensure our initial state is as expected
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // All docs should have have group = 0 added by the script upon creation
+    assertEquals(
+        numDocs, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+
+    // Run the same data again. This time, because all docs exist in the index already, scripted
+    // updates should happen rather than scripted inserts.
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // The script will set either 0 or 1 for the group value on update operations
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testMaxParallelRequestsPerWindow() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withMaxParallelRequestsPerWindow(1);
+
+    PCollection<KV<Integer, Iterable<String>>> batches =
+        pipeline.apply(Create.of(data)).apply(StatefulBatching.fromSpec(write.getBulkIO()));
+
+    PCollection<Integer> keyValues =
+        batches
+            .apply(GroupByKey.create())

Review comment:
       I think it is not needed, has _GroupintoBatches_ already outputs Iterable<String> per key as Beam state is per key. This will simplify _AssertThatHasExpectedContents_ code




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r625119960



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -257,6 +259,14 @@ void testWrite() throws Exception {
     executeWriteTest(write);
   }
 
+  void testWriteStateful() throws Exception {

Review comment:
       missing withMaxParallelRquestsPerWindow coverage




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629337655



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {

Review comment:
       plus,
    if we offer the ability to have _withBackendVersion()_ to write, it needs to be uniform and offer it to read as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629337655



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {

Review comment:
       + if we offer the ability to have _withBackendVersion()_ to write, it needs to be uniform and offer it to read as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-812412884


   @egalpin thanks for your contribution. I'm sorry I lack time a lot lately. In the meantime can you:
   - make the build pass (precommit fail)
   - as it is rather large contribution : please open a ticket with details and rename PR name.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r637042073



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -532,6 +602,34 @@ void testWriteWithFullAddressing() throws Exception {
     }
   }
 
+  /**
+   * Tests that documents are correctly routed when index, type and document ID functions are
+   * provided to overwrite the defaults of using the configuration and auto-generation of the
+   * document IDs by Elasticsearch. The scientist name is used for the index, type and document ID.
+   * As a result there should be only a single document in each index/type.
+   */

Review comment:
       you missed to fix this javadoc




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-816032965


   oh I also didn't want to pressure anyone : ) I'm just checking. thanks for the update!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-839378644


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r625195607



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -257,6 +259,14 @@ void testWrite() throws Exception {
     executeWriteTest(write);
   }
 
+  void testWriteStateful() throws Exception {

Review comment:
       ah good catch, thanks πŸ‘ I'll have to think about how to test it in a way that can confirm the setting is working as intended πŸ€” 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-816021352


   @echauchot let me know if you can take a look at this, and if not I can help you find more reviewers : )


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629351489



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1453,14 +2062,23 @@ private HttpEntity handleRetry(
         // while retry policy exists
         while (BackOffUtils.next(sleeper, backoff)) {
           LOG.warn(String.format(RETRY_ATTEMPT_LOG, ++attempt));
-          Request request = new Request(method, endpoint);
-          request.addParameters(params);
-          request.setEntity(requestBody);
-          response = restClient.performRequest(request);
-          responseEntity = new BufferedHttpEntity(response.getEntity());
+          try {
+            Request request = new Request(method, endpoint);
+            request.addParameters(params);
+            request.setEntity(requestBody);
+            response = restClient.performRequest(request);
+            responseEntity = new BufferedHttpEntity(response.getEntity());
+          } catch (java.io.IOException ex) {

Review comment:
       seems good, but please check that neither _ConnectTimeoutException_ nor _ConnectException_ are thrown in case of bad configuration.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r624553384



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1246,88 +1891,182 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. BE AWARE that low values for @param
+     * maxParallelRequestsPerWindow, in particular if the input data has a finite number of windows,
+     * can reduce parallelism greatly. If data is globally windowed and @param
+     * maxParallelRequestsPerWindow is set to 1,there will only ever be 1 request in flight. Having
+     * only a single request in flight can be beneficial for ensuring an Elasticsearch cluster is
+     * not overwhelmed by parallel requests,but may not work for all use cases. If this number is
+     * less than the number of maximum workers in your pipeline, the IO work may not be distributed

Review comment:
       added that wording πŸ‘ 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] mattwelke commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
mattwelke commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-1019668349


   Those are a good start, thanks. For any more discussion, I'll use the mailing list or Slack.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-825523777


   > Thanks for the 2nd round @echauchot ! Working my way through your comments and adding test coverage. Seeing some of behaviour in the tests, I may have to reach out with questions if I can’t get to the bottom of it.
   > 
   > Somewhat related: can you describe the high-level difference between the IOTest Vs. IOIT files?
   
   UTests for IO are not in the proper sense UTests because they depend on external software (embedded in-jvm ES instance provided by elastic test framework). We do so because to be relevant, the tests need to be run against a real backend and not a mock. And we want to fail the build if any regression occurs.
   
   ITests are run against elasticsearch instances installed in a cluster in IT environment. The tests there are more aimed for volume (which as proved to be a problem in some cases).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-808376951


   I have the local dev env setup by using `start-build-env.sh` from this repo, but I'm still working towards running `elasticsearch-tests`. Any pointers would be appreciated if anyone has time πŸ™‚ 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-823528046


   @echauchot Thanks for the review, I'll work my way through your comments and suggestions. 
   
   > Besides, Evan, as you know ES very well, and you seem to be interested in contributing. Would you be interested in putting yourself in ES Owners file and jira ES label ?
   
   I'd be very happy to πŸ‘  I've added myself to the ES owners file now, happy to lend a hand reviewing! Thanks πŸ™‚  
   
   With respect to Jira, could you please add appropriate permissions for me to either assign myself to the ES label, or assign me to the label yourself if that is the preferred workflow. I have an account on issues.apache.org/jira but only with permission to create tickets I believe.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-830000494


   Sorry about that @echauchot , I had seen a dev@ thread about not having sloppy commit history and squashing to avoid fixup commits but I may have been overzealous haha. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629336790



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {

Review comment:
       I think I was unclear, when I said "+ a warning on version mismatch", I meant adding a javadoc warning not a check in code otherwise we lose the ability to avoid cluster check and keep a dep with the cluster availability indeed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-808263557


   As this is my first time contributing, I'm not sure exactly who to select as a reviewer. I see @echauchot in the git blame a lot for ElasticsearchIO.java, so I'll start there? πŸ˜‚ 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-816028703


   @pabloem fyi I messaged echauchot on Apache beam slack previously and we had arranged to chat about tests together tomorrow. I wanted to let them know that I’d gotten past my blockers with respect to tests, but I definitely did not intend to apply pressure or name+shame or anything like that. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629336790



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {

Review comment:
       I think I was unclear, when I said "+ a warning on version mismatch", I meant adding a javadoc warning not a check in code otherwise we lose the ability to avoid cluster check and keep a dep with the cluster availability indeed.
   
   please to the check with cluster only if the version is not specified and state in the javadoc of withBackendVersion that: "allows to specify the targeted ES version. If it is not specified, a request will be sent to the cluster to determine its version. Specifying the version allows to avoid this request but the specified version bust be identical to the actual version of the cluster."




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot edited a comment on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot edited a comment on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-824632383


   @egalpin as a general review tip, could you please add a comment or an emoji on each of my comments so that I can know that you have seen/addressed it ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-846010011


   @egalpin seems good ! thanks ! I just triggered the build. it's like fort knox now on resources consumption :smile: 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-827479094


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r630620694



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -568,11 +664,167 @@ void testWritePartialUpdate() throws Exception {
     assertEquals(numDocs, currentNumDocs);
     assertEquals(
         numDocs / NUM_SCIENTISTS,
-        countByScientistName(connectionConfiguration, restClient, "Einstein"));
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
 
     // Partial update assertions
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0"));
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testWriteWithDocVersion() throws Exception {
+    List<ObjectNode> jsonData =
+        ElasticsearchIOTestUtils.createJsonDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    List<String> data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      doc.put("my_version", "1");
+      data.add(doc.toString());
+    }
+
+    insertTestDocuments(connectionConfiguration, data, restClient);
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+    // Check that all docs have the same "my_version"
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withDocVersionFn(new ExtractValueFn("my_version"))
+            .withDocVersionType("external");
+
+    data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      // Set version to larger number than originally set, and larger than next logical version
+      // number set by default by ES.
+      doc.put("my_version", "3");
+      data.add(doc.toString());
+    }
+
+    // Test that documents with lower version are rejected, but rejections ignored when specified
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+
+    // my_version and doc version should have changed
+    assertEquals(
+        0,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "3", null, KV.of(3, numDocs)));
+  }
+
+  /**
+   * Tests upsert script by adding a group field to each document in the standard test set. The
+   * group field is populated as the modulo 2 of the document id allowing for a test to ensure the
+   * documents are split into 2 groups.
+   */
+  void testWriteScriptedUpsert() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withUpsertScript(SCRIPT_SOURCE);
+
+    // Test that documents can be inserted/created by using withUpsertScript
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    // defensive coding to ensure our initial state is as expected
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // All docs should have have group = 0 added by the script upon creation
+    assertEquals(
+        numDocs, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+
+    // Run the same data again. This time, because all docs exist in the index already, scripted
+    // updates should happen rather than scripted inserts.
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // The script will set either 0 or 1 for the group value on update operations
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testMaxParallelRequestsPerWindow() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withMaxParallelRequestsPerWindow(1);
+
+    PCollection<KV<Integer, Long>> batches =
+        pipeline
+            .apply(Create.of(data))
+            .apply(Window.into(new GlobalWindows()))
+            .apply(StatefulBatching.fromSpec(write.getBulkIO()))
+            .apply(Count.perKey());
+
+    // Number of unique keys produced should be number of maxParallelRequestsPerWindow * numWindows
+    // There is only 1 request (key) per window, and 1 (global) window ie. one key total
+    PAssert.that(batches).containsInAnyOrder(Collections.singletonList(KV.of(0, 1L)));
+
+    pipeline.run();
+  }
+
+  void testMaxBufferingDurationAndMaxParallelRequestsPerWindow() throws Exception {

Review comment:
       Removed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-817684751


   @timrobertson100 you might be interested. @ludovic-boutros you were thinking about changing the overall arhictecture of the IO and introduce a testContainer based test framework here: https://lists.apache.org/thread.html/reb68f37c435995a64ded19100e09dfc31c5cf6227feae16494226100%40%3Cdev.beam.apache.org%3E
   any comment ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-816034886


   Haha thanks! I appreciate it. I wanted to be sure I was respecting others' busy schedules ❀️ 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-833428233


   @echauchot added coverage for the methods you mentioned. Anything else outstanding? πŸ™‚


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r616682156



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -257,6 +259,14 @@ void testWrite() throws Exception {
     executeWriteTest(write);
   }
 
+  void testWriteStateful() throws Exception {

Review comment:
       you need to add this test in all ElasticSearchTests.java per version otherwise this test is not run




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-849639426


   Thanks Etienne for all of your reviewing efforts, and your warm welcome to Beam! πŸ˜„ 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-817722444


   > @echauchot let me know if you can take a look at this, and if not I can help you find more reviewers : )
   
   @pabloem Β @egalpin I'll do an overall review but I'd need another reviewer to do the in-depth review because:
   - I'm busy on many other things lately
   - I coded support for ES 2, ES 5, reviewed 6 and coded ES7. I'd like to pass the torch :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-823727536


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629360627



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -265,6 +275,13 @@ void testWriteWithErrors() throws Exception {
     List<String> input =
         ElasticsearchIOTestUtils.createDocuments(
             numDocs, ElasticsearchIOTestUtils.InjectionMode.INJECT_SOME_INVALID_DOCS);
+

Review comment:
       I think you missed that one: the code below is duplicated among tests and should be extracted to a method in ESTestUtils.
   `List<String> serializedInput = new ArrayList<>();
       for (String doc : input) {
         serializedInput.add(
             DocToBulk.createBulkApiEntity(
                 write.getDocToBulk(), doc, getBackendVersion(connectionConfiguration)));
       }`
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629682525



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -158,10 +167,16 @@
 })
 public class ElasticsearchIO {

Review comment:
       updated




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r638772885



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -568,11 +670,202 @@ void testWritePartialUpdate() throws Exception {
     assertEquals(numDocs, currentNumDocs);
     assertEquals(
         numDocs / NUM_SCIENTISTS,
-        countByScientistName(connectionConfiguration, restClient, "Einstein"));
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
 
     // Partial update assertions
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0"));
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testWriteWithDocVersion() throws Exception {
+    List<ObjectNode> jsonData =
+        ElasticsearchIOTestUtils.createJsonDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    List<String> data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      doc.put("my_version", "1");
+      data.add(doc.toString());
+    }
+
+    insertTestDocuments(connectionConfiguration, data, restClient);
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+    // Check that all docs have the same "my_version"
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withDocVersionFn(new ExtractValueFn("my_version"))
+            .withDocVersionType("external");
+
+    data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      // Set version to larger number than originally set, and larger than next logical version
+      // number set by default by ES.
+      doc.put("my_version", "3");
+      data.add(doc.toString());
+    }
+
+    // Test that documents with lower version are rejected, but rejections ignored when specified
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+
+    // my_version and doc version should have changed
+    assertEquals(
+        0,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "3", null, KV.of(3, numDocs)));
+  }
+
+  /**
+   * Tests upsert script by adding a group field to each document in the standard test set. The
+   * group field is populated as the modulo 2 of the document id allowing for a test to ensure the
+   * documents are split into 2 groups.
+   */
+  void testWriteScriptedUpsert() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withUpsertScript(SCRIPT_SOURCE);
+
+    // Test that documents can be inserted/created by using withUpsertScript
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    // defensive coding to ensure our initial state is as expected
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // All docs should have have group = 0 added by the script upon creation
+    assertEquals(
+        numDocs, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+
+    // Run the same data again. This time, because all docs exist in the index already, scripted
+    // updates should happen rather than scripted inserts.
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // The script will set either 0 or 1 for the group value on update operations
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testMaxParallelRequestsPerWindow() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withMaxParallelRequestsPerWindow(1);
+
+    PCollection<KV<Integer, Iterable<String>>> batches =
+        pipeline.apply(Create.of(data)).apply(StatefulBatching.fromSpec(write.getBulkIO()));
+
+    PCollection<Integer> keyValues =
+        batches
+            .apply(GroupByKey.create())

Review comment:
       GBK removed! :-) 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r615664611



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -158,10 +167,16 @@
 })
 public class ElasticsearchIO {

Review comment:
       This change is backward compatible so the public API stays the same, but I think you could write a summing up javadoc paragraph about the separation of serialization/write transforms and the use cases that this open.

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -190,44 +219,53 @@ static JsonNode parseResponse(HttpEntity responseEntity) throws IOException {
     return mapper.readValue(responseEntity.getContent(), JsonNode.class);
   }
 
-  static void checkForErrors(HttpEntity responseEntity, int backendVersion, boolean partialUpdate)
+  static void checkForErrors(HttpEntity responseEntity, Set<String> allowedErrorTypes)
       throws IOException {
+
     JsonNode searchResult = parseResponse(responseEntity);
     boolean errors = searchResult.path("errors").asBoolean();
     if (errors) {
+      int numErrors = 0;
+
       StringBuilder errorMessages =
           new StringBuilder("Error writing to Elasticsearch, some elements could not be inserted:");
       JsonNode items = searchResult.path("items");
+      if (items.isMissingNode() || items.size() == 0) {
+        errorMessages.append(searchResult.toString());
+      }
       // some items present in bulk might have errors, concatenate error messages
       for (JsonNode item : items) {
+        JsonNode error = item.findValue("error");

Review comment:
       more robust than the previous errorRoot code, thanks !

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -171,13 +186,27 @@ public static Read read() {
         .build();
   }
 
-  public static Write write() {
-    return new AutoValue_ElasticsearchIO_Write.Builder()
+  public static DocToBulk docToBulk() {
+    return new AutoValue_ElasticsearchIO_DocToBulk.Builder()
+        .setUsePartialUpdate(false) // default is document upsert
+        .build();
+  }
+
+  public static BulkIO bulkIO() {
+    return new AutoValue_ElasticsearchIO_BulkIO.Builder()
         // advised default starting batch size in ES docs
         .setMaxBatchSize(1000L)
         // advised default starting batch size in ES docs
         .setMaxBatchSizeBytes(5L * 1024L * 1024L)
-        .setUsePartialUpdate(false) // default is document upsert
+        .setUseStatefulBatches(false)
+        .setMaxParallelRequestsPerWindow(1)
+        .build();
+  }
+
+  public static Write write() {

Review comment:
       backward compatible indeed. Need to check the tests to be sure, but it should

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -190,44 +219,53 @@ static JsonNode parseResponse(HttpEntity responseEntity) throws IOException {
     return mapper.readValue(responseEntity.getContent(), JsonNode.class);
   }
 
-  static void checkForErrors(HttpEntity responseEntity, int backendVersion, boolean partialUpdate)
+  static void checkForErrors(HttpEntity responseEntity, Set<String> allowedErrorTypes)
       throws IOException {
+
     JsonNode searchResult = parseResponse(responseEntity);
     boolean errors = searchResult.path("errors").asBoolean();
     if (errors) {
+      int numErrors = 0;
+
       StringBuilder errorMessages =
           new StringBuilder("Error writing to Elasticsearch, some elements could not be inserted:");
       JsonNode items = searchResult.path("items");
+      if (items.isMissingNode() || items.size() == 0) {
+        errorMessages.append(searchResult.toString());
+      }
       // some items present in bulk might have errors, concatenate error messages
       for (JsonNode item : items) {
+        JsonNode error = item.findValue("error");
+        if (error == null) {
+          continue;

Review comment:
       coding style: I would prefer having everything under `if (error != null)` rather than `continue`

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -190,44 +219,53 @@ static JsonNode parseResponse(HttpEntity responseEntity) throws IOException {
     return mapper.readValue(responseEntity.getContent(), JsonNode.class);
   }
 
-  static void checkForErrors(HttpEntity responseEntity, int backendVersion, boolean partialUpdate)
+  static void checkForErrors(HttpEntity responseEntity, Set<String> allowedErrorTypes)
       throws IOException {
+
     JsonNode searchResult = parseResponse(responseEntity);
     boolean errors = searchResult.path("errors").asBoolean();
     if (errors) {
+      int numErrors = 0;
+
       StringBuilder errorMessages =
           new StringBuilder("Error writing to Elasticsearch, some elements could not be inserted:");
       JsonNode items = searchResult.path("items");
+      if (items.isMissingNode() || items.size() == 0) {
+        errorMessages.append(searchResult.toString());
+      }
       // some items present in bulk might have errors, concatenate error messages
       for (JsonNode item : items) {
+        JsonNode error = item.findValue("error");
+        if (error == null) {
+          continue;
+        }
 
-        String errorRootName = "";
-        // when use partial update, the response items includes all the update.
-        if (partialUpdate) {
-          errorRootName = "update";
-        } else {
-          if (backendVersion == 2) {
-            errorRootName = "create";
-          } else if (backendVersion >= 5) {
-            errorRootName = "index";
-          }
+        // N.B. An empty-string within the allowedErrorTypes Set implies all errors are allowed.
+        String type = error.path("type").asText();
+        String reason = error.path("reason").asText();
+        String docId = item.findValue("_id").asText();
+        JsonNode causedBy = error.path("caused_by"); // May not be present

Review comment:
       better than get + if null, thanks

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -303,6 +341,73 @@ public static ConnectionConfiguration create(String[] addresses, String index, S
           .build();
     }
 
+    /**
+     * Creates a new Elasticsearch connection configuration with no default type.
+     *
+     * @param addresses list of addresses of Elasticsearch nodes
+     * @param index the index toward which the requests will be issued
+     * @return the connection configuration object
+     */
+    public static ConnectionConfiguration create(String[] addresses, String index) {
+      checkArgument(addresses != null, "addresses can not be null");
+      checkArgument(addresses.length > 0, "addresses can not be empty");
+      checkArgument(index != null, "index can not be null");
+      return new AutoValue_ElasticsearchIO_ConnectionConfiguration.Builder()
+          .setAddresses(Arrays.asList(addresses))
+          .setIndex(index)
+          .setType("")
+          .setTrustSelfSignedCerts(false)
+          .build();
+    }
+
+    /**
+     * Creates a new Elasticsearch connection configuration with no default index nor type.
+     *
+     * @param addresses list of addresses of Elasticsearch nodes
+     * @return the connection configuration object
+     */
+    public static ConnectionConfiguration create(String[] addresses) {
+      checkArgument(addresses != null, "addresses can not be null");
+      checkArgument(addresses.length > 0, "addresses can not be empty");
+      return new AutoValue_ElasticsearchIO_ConnectionConfiguration.Builder()
+          .setAddresses(Arrays.asList(addresses))
+          .setIndex("")
+          .setType("")
+          .setTrustSelfSignedCerts(false)
+          .build();
+    }
+
+    /**
+     * Generates the bulk API endpoint based on the set values.
+     *
+     * <p>Based on ConnectionConfiguration constructors, we know that one of the following is true:
+     *
+     * <ul>
+     *   <li>index and type are non-empty strings
+     *   <li>index is non-empty string, type is empty string
+     *   <li>index and type are empty string
+     * </ul>
+     *
+     * <p>Valid endpoints therefore include:
+     *
+     * <ul>
+     *   <li>/_bulk
+     *   <li>/index_name/_bulk
+     *   <li>/index_name/type_name/_bulk
+     * </ul>
+     */
+    public String getBulkEndPoint() {

Review comment:
       I did not know it was allowed to specify no index: something new ?

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -190,44 +219,53 @@ static JsonNode parseResponse(HttpEntity responseEntity) throws IOException {
     return mapper.readValue(responseEntity.getContent(), JsonNode.class);
   }
 
-  static void checkForErrors(HttpEntity responseEntity, int backendVersion, boolean partialUpdate)
+  static void checkForErrors(HttpEntity responseEntity, Set<String> allowedErrorTypes)
       throws IOException {
+
     JsonNode searchResult = parseResponse(responseEntity);
     boolean errors = searchResult.path("errors").asBoolean();
     if (errors) {
+      int numErrors = 0;
+
       StringBuilder errorMessages =
           new StringBuilder("Error writing to Elasticsearch, some elements could not be inserted:");
       JsonNode items = searchResult.path("items");
+      if (items.isMissingNode() || items.size() == 0) {
+        errorMessages.append(searchResult.toString());
+      }
       // some items present in bulk might have errors, concatenate error messages
       for (JsonNode item : items) {
+        JsonNode error = item.findValue("error");
+        if (error == null) {
+          continue;
+        }
 
-        String errorRootName = "";
-        // when use partial update, the response items includes all the update.
-        if (partialUpdate) {
-          errorRootName = "update";
-        } else {
-          if (backendVersion == 2) {
-            errorRootName = "create";
-          } else if (backendVersion >= 5) {
-            errorRootName = "index";
-          }
+        // N.B. An empty-string within the allowedErrorTypes Set implies all errors are allowed.
+        String type = error.path("type").asText();
+        String reason = error.path("reason").asText();
+        String docId = item.findValue("_id").asText();
+        JsonNode causedBy = error.path("caused_by"); // May not be present
+        String cbReason = causedBy.path("reason").asText();
+        String cbType = causedBy.path("type").asText();
+
+        if (allowedErrorTypes != null
+            && (allowedErrorTypes.contains(type) || allowedErrorTypes.contains(cbType))) {
+          continue;
         }
-        JsonNode errorRoot = item.path(errorRootName);
-        JsonNode error = errorRoot.get("error");
-        if (error != null) {
-          String type = error.path("type").asText();
-          String reason = error.path("reason").asText();
-          String docId = errorRoot.path("_id").asText();
-          errorMessages.append(String.format("%nDocument id %s: %s (%s)", docId, reason, type));
-          JsonNode causedBy = error.get("caused_by");
-          if (causedBy != null) {
-            String cbReason = causedBy.path("reason").asText();
-            String cbType = causedBy.path("type").asText();
-            errorMessages.append(String.format("%nCaused by: %s (%s)", cbReason, cbType));
-          }
+
+        // 'error' field is not null, and the error is not being ignored.
+        numErrors++;
+
+        errorMessages.append(String.format("%nDocument id %s: %s (%s)", docId, reason, type));
+
+        if (causedBy.isMissingNode()) {

Review comment:
       ditto

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {
+          backendVersion = ElasticsearchIO.getBackendVersion(connectionConfiguration);
+        } else {
+          backendVersion = spec.getBackendVersion();
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext c) throws IOException {
+        c.output(createBulkApiEntity(spec, c.element(), backendVersion));
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransform} convenience wrapper for doing both document to bulk API serialization as
+   * well as batching those Bulk API entities and writing them to an Elasticsearch cluster. This
+   * class is effectively a thin proxy for DocToBulk->BulkIO all-in-one for convenience and backward
+   * compatibility.
+   */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<String>, PDone> {
+    public interface FieldValueExtractFn extends SerializableFunction<JsonNode, String> {}
+
+    public interface BooleanFieldValueExtractFn extends SerializableFunction<JsonNode, Boolean> {}
+
+    public abstract DocToBulk getDocToBulk();
+
+    public abstract BulkIO getBulkIO();
+
+    abstract Builder writeBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setDocToBulk(DocToBulk docToBulk);
+
+      abstract Builder setBulkIO(BulkIO bulkIO);
+
+      abstract Write build();
+    }
+
+    // For building Doc2Bulk
+    /** Refer to {@link DocToBulk#withIdFn}. */
+    public Write withIdFn(FieldValueExtractFn idFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIdFn(idFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withIndexFn}. */
+    public Write withIndexFn(FieldValueExtractFn indexFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIndexFn(indexFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withRoutingFn}. */
+    public Write withRoutingFn(FieldValueExtractFn routingFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withRoutingFn(routingFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withTypeFn}. */
+    public Write withTypeFn(FieldValueExtractFn typeFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withTypeFn(typeFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withDocVersionFn}. */
+    public Write withDocVersionFn(FieldValueExtractFn docVersionFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withDocVersionFn(docVersionFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withDocVersionType}. */
+    public Write withDocVersionType(String docVersionType) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withDocVersionType(docVersionType)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withUsePartialUpdate}. */
+    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+      return writeBuilder()
+          .setDocToBulk(getDocToBulk().withUsePartialUpdate(usePartialUpdate))
+          .build();
+    }
+
+    /** Refer to {@link DocToBulk#withUpsertScript}. */
+    public Write withUpsertScript(String source) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withUpsertScript(source)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withBackendVersion}. */
+    public Write withBackendVersion(int backendVersion) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withBackendVersion(backendVersion)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withIsDeleteFn}. */
+    public Write withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIsDeleteFn(isDeleteFn)).build();
+    }
+    // End building Doc2Bulk
+
+    /** Refer to {@link BulkIO#withConnectionConfiguration}. */
+    public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) {
+      checkArgument(connectionConfiguration != null, "connectionConfiguration can not be null");
+
+      return writeBuilder()
+          .setDocToBulk(getDocToBulk().withConnectionConfiguration(connectionConfiguration))
+          .setBulkIO(getBulkIO().withConnectionConfiguration(connectionConfiguration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBatchSize}. */
+    public Write withMaxBatchSize(long batchSize) {
+      return writeBuilder().setBulkIO(getBulkIO().withMaxBatchSize(batchSize)).build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBatchSizeBytes}. */
+    public Write withMaxBatchSizeBytes(long batchSizeBytes) {
+      return writeBuilder().setBulkIO(getBulkIO().withMaxBatchSizeBytes(batchSizeBytes)).build();
+    }
+
+    /** Refer to {@link BulkIO#withRetryConfiguration}. */
+    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withRetryConfiguration(retryConfiguration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withIgnoreVersionConflicts}. */
+    public Write withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withIgnoreVersionConflicts(ignoreVersionConflicts))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withUseStatefulBatches}. */
+    public Write withUseStatefulBatches(boolean useStatefulBatches) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withUseStatefulBatches(useStatefulBatches))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBufferingDuration}. */
+    public Write withMaxBufferingDuration(Duration maxBufferingDuration) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withMaxBufferingDuration(maxBufferingDuration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxParallelRequestsPerWindow}. */
+    public Write withMaxParallelRquestsPerWindow(int maxParallelRquestsPerWindow) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withMaxParallelRequestsPerWindow(maxParallelRquestsPerWindow))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withAllowableResponseErrors}. */
+    public Write withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrors) {

Review comment:
       to avoid stacktrace messages flooding ?

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1030,115 +1135,92 @@ public boolean test(HttpEntity responseEntity) {
     }
   }
 
-  /** A {@link PTransform} writing data to Elasticsearch. */
+  /** A {@link PTransform} converting docs to their Bulk API counterparts. */
   @AutoValue
-  public abstract static class Write extends PTransform<PCollection<String>, PDone> {
+  public abstract static class DocToBulk
+      extends PTransform<PCollection<String>, PCollection<String>> {
 
-    /**
-     * Interface allowing a specific field value to be returned from a parsed JSON document. This is
-     * used for using explicit document ids, and for dynamic routing (index/Type) on a document
-     * basis. A null response will result in default behaviour and an exception will be propagated
-     * as a failure.
-     */
-    public interface FieldValueExtractFn extends SerializableFunction<JsonNode, String> {}
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final int DEFAULT_RETRY_ON_CONFLICT = 5; // race conditions on updates
 
-    public interface BooleanFieldValueExtractFn extends SerializableFunction<JsonNode, Boolean> {}
+    static {
+      SimpleModule module = new SimpleModule();
+      module.addSerializer(DocumentMetadata.class, new DocumentMetadataSerializer());
+      OBJECT_MAPPER.registerModule(module);
+    }
 
     abstract @Nullable ConnectionConfiguration getConnectionConfiguration();
 
-    abstract long getMaxBatchSize();
+    abstract Write.@Nullable FieldValueExtractFn getIdFn();
 
-    abstract long getMaxBatchSizeBytes();
+    abstract Write.@Nullable FieldValueExtractFn getIndexFn();
 
-    abstract @Nullable FieldValueExtractFn getIdFn();
+    abstract Write.@Nullable FieldValueExtractFn getRoutingFn();

Review comment:
       good addition !

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -303,6 +341,73 @@ public static ConnectionConfiguration create(String[] addresses, String index, S
           .build();
     }
 
+    /**
+     * Creates a new Elasticsearch connection configuration with no default type.
+     *
+     * @param addresses list of addresses of Elasticsearch nodes
+     * @param index the index toward which the requests will be issued
+     * @return the connection configuration object
+     */
+    public static ConnectionConfiguration create(String[] addresses, String index) {
+      checkArgument(addresses != null, "addresses can not be null");
+      checkArgument(addresses.length > 0, "addresses can not be empty");
+      checkArgument(index != null, "index can not be null");
+      return new AutoValue_ElasticsearchIO_ConnectionConfiguration.Builder()
+          .setAddresses(Arrays.asList(addresses))
+          .setIndex(index)
+          .setType("")
+          .setTrustSelfSignedCerts(false)
+          .build();
+    }
+
+    /**
+     * Creates a new Elasticsearch connection configuration with no default index nor type.
+     *
+     * @param addresses list of addresses of Elasticsearch nodes
+     * @return the connection configuration object
+     */
+    public static ConnectionConfiguration create(String[] addresses) {
+      checkArgument(addresses != null, "addresses can not be null");
+      checkArgument(addresses.length > 0, "addresses can not be empty");
+      return new AutoValue_ElasticsearchIO_ConnectionConfiguration.Builder()
+          .setAddresses(Arrays.asList(addresses))
+          .setIndex("")
+          .setType("")
+          .setTrustSelfSignedCerts(false)
+          .build();
+    }
+
+    /**
+     * Generates the bulk API endpoint based on the set values.
+     *
+     * <p>Based on ConnectionConfiguration constructors, we know that one of the following is true:
+     *
+     * <ul>
+     *   <li>index and type are non-empty strings
+     *   <li>index is non-empty string, type is empty string
+     *   <li>index and type are empty string
+     * </ul>
+     *
+     * <p>Valid endpoints therefore include:
+     *
+     * <ul>
+     *   <li>/_bulk
+     *   <li>/index_name/_bulk
+     *   <li>/index_name/type_name/_bulk
+     * </ul>
+     */
+    public String getBulkEndPoint() {
+      List<String> endPointComponents = Arrays.asList(getIndex(), getType(), "_bulk");

Review comment:
       This piece of code looks strange to me. I would prefer a more readable impl like this:
   ```suggestion
         StringBuilder sb = new StringBuilder();
         if (!Strings.isNullOrEmpty(getIndex())){
           sb.append("/").append(getIndex());
         }
         if (!Strings.isNullOrEmpty(getType())){
           sb.append("/").append(getType());
         }
         sb.append("/").append("_bulk");
         return sb.toString();
   
   ```

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1030,115 +1135,92 @@ public boolean test(HttpEntity responseEntity) {
     }
   }
 
-  /** A {@link PTransform} writing data to Elasticsearch. */
+  /** A {@link PTransform} converting docs to their Bulk API counterparts. */
   @AutoValue
-  public abstract static class Write extends PTransform<PCollection<String>, PDone> {
+  public abstract static class DocToBulk
+      extends PTransform<PCollection<String>, PCollection<String>> {
 
-    /**
-     * Interface allowing a specific field value to be returned from a parsed JSON document. This is
-     * used for using explicit document ids, and for dynamic routing (index/Type) on a document
-     * basis. A null response will result in default behaviour and an exception will be propagated
-     * as a failure.
-     */
-    public interface FieldValueExtractFn extends SerializableFunction<JsonNode, String> {}
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final int DEFAULT_RETRY_ON_CONFLICT = 5; // race conditions on updates
 
-    public interface BooleanFieldValueExtractFn extends SerializableFunction<JsonNode, Boolean> {}
+    static {
+      SimpleModule module = new SimpleModule();
+      module.addSerializer(DocumentMetadata.class, new DocumentMetadataSerializer());
+      OBJECT_MAPPER.registerModule(module);
+    }
 
     abstract @Nullable ConnectionConfiguration getConnectionConfiguration();
 
-    abstract long getMaxBatchSize();
+    abstract Write.@Nullable FieldValueExtractFn getIdFn();
 
-    abstract long getMaxBatchSizeBytes();
+    abstract Write.@Nullable FieldValueExtractFn getIndexFn();
 
-    abstract @Nullable FieldValueExtractFn getIdFn();
+    abstract Write.@Nullable FieldValueExtractFn getRoutingFn();
 
-    abstract @Nullable FieldValueExtractFn getIndexFn();
+    abstract Write.@Nullable FieldValueExtractFn getTypeFn();
 
-    abstract @Nullable FieldValueExtractFn getTypeFn();
+    abstract Write.@Nullable FieldValueExtractFn getDocVersionFn();
 
-    abstract @Nullable RetryConfiguration getRetryConfiguration();
+    abstract @Nullable String getDocVersionType();
 
-    abstract boolean getUsePartialUpdate();
+    abstract @Nullable String getUpsertScript();

Review comment:
       good addition !

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {

Review comment:
       as _backendVersion_ is optional it looks good to me but what if set version and actual version mismatch ?

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1269,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {
+          backendVersion = ElasticsearchIO.getBackendVersion(connectionConfiguration);
+        } else {
+          backendVersion = spec.getBackendVersion();
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext c) throws IOException {
+        c.output(createBulkApiEntity(spec, c.element(), backendVersion));
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransform} convenience wrapper for doing both document to bulk API serialization as

Review comment:
       With the separation, this transform will still be the entry point for almost all the users. So I would copy the javadoc of original Write transform there. And add this javadoc as details with a sentence such as "in fact, it is a convinience ...."

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1203,88 +1796,160 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. If data is globally windowed and this
+     * configuration is set to 1, there will only ever be 1 request in flight. Having only a single
+     * request in flight can be beneficial for ensuring an Elasticsearch cluster is not overwhelmed
+     * by parallel requests, but may not work for all use cases. If this number is less than the
+     * number of maximum workers in your pipeline, the IO work may not be distributed across all
+     * workers.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =
+            GroupIntoBatches.ofSize(getMaxBatchSize());
+
+        if (getMaxBufferingDuration() != null) {
+          groupIntoBatches = groupIntoBatches.withMaxBufferingDuration(getMaxBufferingDuration());
+        }
+        input
+            .apply(ParDo.of(new AssignShardFn<>(getMaxParallelRequestsPerWindow())))

Review comment:
       It is true that you need to provide a ` PCollection<KV>` to use `GroupIntoBatches` (because inner state in GIB is per key). Also true that you cannot use runner.parallelism to know the number of workers because it would be a leak from the runner to the SDK which is forbidden. But using `AssignShardFn<>(getMaxParallelRequestsPerWindow())` seems risky for parallelism because the key will be an int modulo `MaxParallelRequestsPerWindow`. If `MaxParallelRequestsPerWindow` set by the user is very low for example 1, then the key will be 0 or 1. And runners such as spark will do object.hascode() to determin to which spark partition the KV should go. I fear that it reduces the parallelism to 2 in that case. But I think you mentioned something similar in the javadoc. Nevertheless, I think forcing the backend engine to reduce concurrency that way is the only way you have to control ES requests concurrency (because no central point in parallel systems) so I think you need to put a big war
 ning for users so that they are aware that using that will reduce concurrency on the last write step of their pipeline: runners will maintain engine configured concurrency starting at the source until the end (if no dynamic repartitioning) but there is very good chance that some partitions will be empty in final write step after setting MaxParallelRequestsPerWindow = very low. 

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1203,88 +1796,160 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. If data is globally windowed and this
+     * configuration is set to 1, there will only ever be 1 request in flight. Having only a single
+     * request in flight can be beneficial for ensuring an Elasticsearch cluster is not overwhelmed
+     * by parallel requests, but may not work for all use cases. If this number is less than the
+     * number of maximum workers in your pipeline, the IO work may not be distributed across all
+     * workers.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =

Review comment:
       once again, nice feature to avoid tiny bulks !

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1203,88 +1796,160 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. If data is globally windowed and this
+     * configuration is set to 1, there will only ever be 1 request in flight. Having only a single
+     * request in flight can be beneficial for ensuring an Elasticsearch cluster is not overwhelmed
+     * by parallel requests, but may not work for all use cases. If this number is less than the
+     * number of maximum workers in your pipeline, the IO work may not be distributed across all
+     * workers.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =
+            GroupIntoBatches.ofSize(getMaxBatchSize());
+
+        if (getMaxBufferingDuration() != null) {
+          groupIntoBatches = groupIntoBatches.withMaxBufferingDuration(getMaxBufferingDuration());
+        }
+        input
+            .apply(ParDo.of(new AssignShardFn<>(getMaxParallelRequestsPerWindow())))
+            .apply(groupIntoBatches)
+            .apply(
+                "Remove key no longer needed",
+                MapElements.into(TypeDescriptors.iterables(TypeDescriptors.strings()))
+                    .via(KV::getValue))
+            .apply(ParDo.of(new BulkIOFn(this)));
+      } else {
+
+        input
+            .apply(
+                "Make elements iterable",
+                MapElements.into(TypeDescriptors.iterables(TypeDescriptors.strings()))

Review comment:
       I know why you do this: this is because you want BulkIOFn to have the same signature when used with GroupIntoBatches and without so you need BulkIOFn to take an Iterable as input because GIB produces an Iterable as output. I'm not a big fan of extra steps just to be able to reuse code. Maybe it would be better to have a BulkIOFn impl that takes a single String and another impl that takes Iterable<String> (for outputs of GIB case) that relies on the first one.

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -265,6 +275,13 @@ void testWriteWithErrors() throws Exception {
     List<String> input =
         ElasticsearchIOTestUtils.createDocuments(
             numDocs, ElasticsearchIOTestUtils.InjectionMode.INJECT_SOME_INVALID_DOCS);
+

Review comment:
       duplicated

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
##########
@@ -156,6 +156,16 @@ public void testWriteWithErrors() throws Exception {
     elasticsearchIOTestCommon.testWriteWithErrors();
   }
 
+  @Test
+  public void testWriteWithAllowableErrors() throws Exception {

Review comment:
       it is a left over I think, it is the same as below

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1269,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {
+          backendVersion = ElasticsearchIO.getBackendVersion(connectionConfiguration);
+        } else {
+          backendVersion = spec.getBackendVersion();
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext c) throws IOException {
+        c.output(createBulkApiEntity(spec, c.element(), backendVersion));
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransform} convenience wrapper for doing both document to bulk API serialization as
+   * well as batching those Bulk API entities and writing them to an Elasticsearch cluster. This
+   * class is effectively a thin proxy for DocToBulk->BulkIO all-in-one for convenience and backward
+   * compatibility.
+   */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<String>, PDone> {
+    public interface FieldValueExtractFn extends SerializableFunction<JsonNode, String> {}
+
+    public interface BooleanFieldValueExtractFn extends SerializableFunction<JsonNode, Boolean> {}
+
+    public abstract DocToBulk getDocToBulk();
+
+    public abstract BulkIO getBulkIO();
+
+    abstract Builder writeBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {

Review comment:
       I don't think a builder pattern is needed here as it is just to store BulkIO and DocToBulk fields and the user will never have to set these fields. I would just store the fields and leave the builder proxy to DocToBulk and BulkIO. In addition, it would avoid having to call the getters each time.

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1453,14 +2062,23 @@ private HttpEntity handleRetry(
         // while retry policy exists
         while (BackOffUtils.next(sleeper, backoff)) {
           LOG.warn(String.format(RETRY_ATTEMPT_LOG, ++attempt));
-          Request request = new Request(method, endpoint);
-          request.addParameters(params);
-          request.setEntity(requestBody);
-          response = restClient.performRequest(request);
-          responseEntity = new BufferedHttpEntity(response.getEntity());
+          try {
+            Request request = new Request(method, endpoint);
+            request.addParameters(params);
+            request.setEntity(requestBody);
+            response = restClient.performRequest(request);
+            responseEntity = new BufferedHttpEntity(response.getEntity());
+          } catch (java.io.IOException ex) {

Review comment:
       same here

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1310,135 +1970,84 @@ public void startBundle(StartBundleContext context) {
         currentBatchSizeBytes = 0;
       }
 
-      private class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
-
-        private DocumentMetadataSerializer() {
-          super(DocumentMetadata.class);
-        }
-
-        @Override
-        public void serialize(
-            DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
-            throws IOException {
-          gen.writeStartObject();
-          if (value.index != null) {
-            gen.writeStringField("_index", value.index);
-          }
-          if (value.type != null) {
-            gen.writeStringField("_type", value.type);
-          }
-          if (value.id != null) {
-            gen.writeStringField("_id", value.id);
-          }
-          if (value.retryOnConflict != null && (backendVersion <= 6)) {
-            gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
-          }
-          if (value.retryOnConflict != null && backendVersion >= 7) {
-            gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
-          }
-          gen.writeEndObject();
-        }
-      }
-      /**
-       * Extracts the components that comprise the document address from the document using the
-       * {@link FieldValueExtractFn} configured. This allows any or all of the index, type and
-       * document id to be controlled on a per document basis. Sanitization of the index is
-       * performed, automatically lower-casing the value as required by Elasticsearch.
-       *
-       * @param parsedDocument the json from which the index, type and id may be extracted
-       * @return the document address as JSON or the default
-       * @throws IOException if the document cannot be parsed as JSON
-       */
-      private String getDocumentMetadata(JsonNode parsedDocument) throws IOException {
-        DocumentMetadata metadata =
-            new DocumentMetadata(
-                spec.getIndexFn() != null
-                    ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
-                    : null,
-                spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
-                spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
-                spec.getUsePartialUpdate() ? DEFAULT_RETRY_ON_CONFLICT : null);
-        return OBJECT_MAPPER.writeValueAsString(metadata);
-      }
-
-      private static String lowerCaseOrNull(String input) {
-        return input == null ? null : input.toLowerCase();
+      @FinishBundle
+      public void finishBundle(FinishBundleContext context)
+          throws IOException, InterruptedException {
+        flushBatch();
       }
 
       @ProcessElement
-      public void processElement(ProcessContext context) throws Exception {
-        String document = context.element(); // use configuration and auto-generated document IDs
-        String documentMetadata = "{}";
-        boolean isDelete = false;
-        if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
-          // parse once and reused for efficiency
-          JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
-          documentMetadata = getDocumentMetadata(parsedDocument);
-          if (spec.getIsDeleteFn() != null) {
-            isDelete = spec.getIsDeleteFn().apply(parsedDocument);
-          }
+      public void processElement(@Element @NonNull Iterable<String> bulkApiEntities)
+          throws Exception {
+        for (String bulkApiEntity : bulkApiEntities) {
+          addAndMaybeFlush(bulkApiEntity);
         }
+      }
 
-        if (isDelete) {
-          // delete request used for deleting a document.
-          batch.add(String.format("{ \"delete\" : %s }%n", documentMetadata));
-        } else {
-          // index is an insert/upsert and update is a partial update (or insert if not existing)
-          if (spec.getUsePartialUpdate()) {
-            batch.add(
-                String.format(
-                    "{ \"update\" : %s }%n{ \"doc\" : %s, \"doc_as_upsert\" : true }%n",
-                    documentMetadata, document));
-          } else {
-            batch.add(String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document));
-          }
-        }
+      protected void addAndMaybeFlush(String bulkApiEntity)
+          throws IOException, InterruptedException {
+        batch.add(bulkApiEntity);
+        currentBatchSizeBytes += bulkApiEntity.getBytes(StandardCharsets.UTF_8).length;
 
-        currentBatchSizeBytes += document.getBytes(StandardCharsets.UTF_8).length;
         if (batch.size() >= spec.getMaxBatchSize()
             || currentBatchSizeBytes >= spec.getMaxBatchSizeBytes()) {
           flushBatch();
         }
       }
 
-      @FinishBundle
-      public void finishBundle(FinishBundleContext context)
-          throws IOException, InterruptedException {
-        flushBatch();
-      }
-
       private void flushBatch() throws IOException, InterruptedException {
         if (batch.isEmpty()) {
           return;
         }
+
+        LOG.info(
+            "ElasticsearchIO batch size: {}, batch size bytes: {}",
+            batch.size(),
+            currentBatchSizeBytes);
+
         StringBuilder bulkRequest = new StringBuilder();
         for (String json : batch) {
           bulkRequest.append(json);
         }
+
         batch.clear();
-        currentBatchSizeBytes = 0;
-        Response response;
-        HttpEntity responseEntity;
-        // Elasticsearch will default to the index/type provided here if none are set in the
-        // document meta (i.e. using ElasticsearchIO$Write#withIndexFn and
-        // ElasticsearchIO$Write#withTypeFn options)
-        String endPoint =
-            String.format(
-                "/%s/%s/_bulk",
-                spec.getConnectionConfiguration().getIndex(),
-                spec.getConnectionConfiguration().getType());
+        currentBatchSizeBytes = 0L;
+
+        Response response = null;
+        HttpEntity responseEntity = null;
+
+        // Elasticsearch will default to the index/type provided the {@link
+        // ConnectionConfiguration} if none are set in the document meta (i.e.
+        // using ElasticsearchIO$DocToBulk#withIndexFn and
+        // ElasticsearchIO$DocToBulk#withTypeFn options)
+        String endPoint = spec.getConnectionConfiguration().getBulkEndPoint();
+
         HttpEntity requestBody =
             new NStringEntity(bulkRequest.toString(), ContentType.APPLICATION_JSON);
-        Request request = new Request("POST", endPoint);
-        request.addParameters(Collections.emptyMap());
-        request.setEntity(requestBody);
-        response = restClient.performRequest(request);
-        responseEntity = new BufferedHttpEntity(response.getEntity());
+        try {
+          Request request = new Request("POST", endPoint);
+          request.addParameters(Collections.emptyMap());
+          request.setEntity(requestBody);
+          response = restClient.performRequest(request);
+          responseEntity = new BufferedHttpEntity(response.getEntity());
+        } catch (java.io.IOException ex) {

Review comment:
       Before we just threw the exception and there would be retrials only on http 429 (predicate). Now retrials are also done when receiving IOException. Are you sure all IOException cases can be retried ? I'm not sure they are all timeouts: a misconfigured IO will throw IOException and will be retried. It is good to retry on Timeouts IMHO but please filter on only timeouts.

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1269,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {
+          backendVersion = ElasticsearchIO.getBackendVersion(connectionConfiguration);
+        } else {
+          backendVersion = spec.getBackendVersion();
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext c) throws IOException {
+        c.output(createBulkApiEntity(spec, c.element(), backendVersion));
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransform} convenience wrapper for doing both document to bulk API serialization as
+   * well as batching those Bulk API entities and writing them to an Elasticsearch cluster. This
+   * class is effectively a thin proxy for DocToBulk->BulkIO all-in-one for convenience and backward
+   * compatibility.
+   */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<String>, PDone> {
+    public interface FieldValueExtractFn extends SerializableFunction<JsonNode, String> {}
+
+    public interface BooleanFieldValueExtractFn extends SerializableFunction<JsonNode, Boolean> {}
+
+    public abstract DocToBulk getDocToBulk();
+
+    public abstract BulkIO getBulkIO();
+
+    abstract Builder writeBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setDocToBulk(DocToBulk docToBulk);
+
+      abstract Builder setBulkIO(BulkIO bulkIO);
+
+      abstract Write build();
+    }
+
+    // For building Doc2Bulk
+    /** Refer to {@link DocToBulk#withIdFn}. */
+    public Write withIdFn(FieldValueExtractFn idFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIdFn(idFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withIndexFn}. */
+    public Write withIndexFn(FieldValueExtractFn indexFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIndexFn(indexFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withRoutingFn}. */
+    public Write withRoutingFn(FieldValueExtractFn routingFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withRoutingFn(routingFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withTypeFn}. */
+    public Write withTypeFn(FieldValueExtractFn typeFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withTypeFn(typeFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withDocVersionFn}. */
+    public Write withDocVersionFn(FieldValueExtractFn docVersionFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withDocVersionFn(docVersionFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withDocVersionType}. */
+    public Write withDocVersionType(String docVersionType) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withDocVersionType(docVersionType)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withUsePartialUpdate}. */
+    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+      return writeBuilder()
+          .setDocToBulk(getDocToBulk().withUsePartialUpdate(usePartialUpdate))
+          .build();
+    }
+
+    /** Refer to {@link DocToBulk#withUpsertScript}. */
+    public Write withUpsertScript(String source) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withUpsertScript(source)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withBackendVersion}. */
+    public Write withBackendVersion(int backendVersion) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withBackendVersion(backendVersion)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withIsDeleteFn}. */
+    public Write withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIsDeleteFn(isDeleteFn)).build();
+    }
+    // End building Doc2Bulk
+
+    /** Refer to {@link BulkIO#withConnectionConfiguration}. */
+    public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) {
+      checkArgument(connectionConfiguration != null, "connectionConfiguration can not be null");
+
+      return writeBuilder()
+          .setDocToBulk(getDocToBulk().withConnectionConfiguration(connectionConfiguration))
+          .setBulkIO(getBulkIO().withConnectionConfiguration(connectionConfiguration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBatchSize}. */
+    public Write withMaxBatchSize(long batchSize) {
+      return writeBuilder().setBulkIO(getBulkIO().withMaxBatchSize(batchSize)).build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBatchSizeBytes}. */
+    public Write withMaxBatchSizeBytes(long batchSizeBytes) {
+      return writeBuilder().setBulkIO(getBulkIO().withMaxBatchSizeBytes(batchSizeBytes)).build();
+    }
+
+    /** Refer to {@link BulkIO#withRetryConfiguration}. */
+    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withRetryConfiguration(retryConfiguration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withIgnoreVersionConflicts}. */
+    public Write withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withIgnoreVersionConflicts(ignoreVersionConflicts))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withUseStatefulBatches}. */
+    public Write withUseStatefulBatches(boolean useStatefulBatches) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withUseStatefulBatches(useStatefulBatches))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBufferingDuration}. */
+    public Write withMaxBufferingDuration(Duration maxBufferingDuration) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withMaxBufferingDuration(maxBufferingDuration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxParallelRequestsPerWindow}. */
+    public Write withMaxParallelRquestsPerWindow(int maxParallelRquestsPerWindow) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withMaxParallelRequestsPerWindow(maxParallelRquestsPerWindow))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withAllowableResponseErrors}. */
+    public Write withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrors) {
+      if (allowableResponseErrors == null) {
+        allowableResponseErrors = new HashSet<>();
+      }
+
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withAllowableResponseErrors(allowableResponseErrors))
+          .build();
+    }
+
+    @Override
+    public PDone expand(PCollection<String> input) {
+      input.apply(getDocToBulk()).apply(getBulkIO());
+      return PDone.in(input.getPipeline());
+    }
+  }
+
+  /** A {@link PTransform} writing data to Elasticsearch. */

Review comment:
       I think we should make it clear for the user here that it is an internal transformation (Write is still the main transform entry point). We should also describe that it takes results of bulk serialization entities (bulkApiEntity) as input otherwise the user will be either confused or mislead to use this transform to write his simple json documents.

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -297,16 +342,27 @@ void testWriteWithMaxBatchSize() throws Exception {
         ElasticsearchIO.write()
             .withConnectionConfiguration(connectionConfiguration)
             .withMaxBatchSize(BATCH_SIZE);
+
     // write bundles size is the runner decision, we cannot force a bundle size,
     // so we test the Writer as a DoFn outside of a runner.
-    try (DoFnTester<String, Void> fnTester = DoFnTester.of(new Write.WriteFn(write))) {
+    try (DoFnTester<Iterable<String>, Void> fnTester =
+        DoFnTester.of(new BulkIO.BulkIOFn(write.getBulkIO()))) {
       List<String> input =
           ElasticsearchIOTestUtils.createDocuments(
               numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+      List<String> serializedInput = new ArrayList<>();
+      for (String doc : input) {
+        serializedInput.add(
+            DocToBulk.createBulkApiEntity(
+                write.getDocToBulk(), doc, getBackendVersion(connectionConfiguration)));
+      }
       long numDocsProcessed = 0;
       long numDocsInserted = 0;
-      for (String document : input) {
-        fnTester.processElement(document);
+      for (String document : serializedInput) {
+        // It's a tad strange to iterate over a list and then make a list of each element, but

Review comment:
       cf my comment in production code: please provide a BulkIOFn that uses `String` and not `Iterable<String>`

##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1203,88 +1796,160 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. If data is globally windowed and this
+     * configuration is set to 1, there will only ever be 1 request in flight. Having only a single
+     * request in flight can be beneficial for ensuring an Elasticsearch cluster is not overwhelmed
+     * by parallel requests, but may not work for all use cases. If this number is less than the
+     * number of maximum workers in your pipeline, the IO work may not be distributed across all
+     * workers.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =
+            GroupIntoBatches.ofSize(getMaxBatchSize());
+
+        if (getMaxBufferingDuration() != null) {
+          groupIntoBatches = groupIntoBatches.withMaxBufferingDuration(getMaxBufferingDuration());
+        }
+        input
+            .apply(ParDo.of(new AssignShardFn<>(getMaxParallelRequestsPerWindow())))
+            .apply(groupIntoBatches)
+            .apply(
+                "Remove key no longer needed",
+                MapElements.into(TypeDescriptors.iterables(TypeDescriptors.strings()))
+                    .via(KV::getValue))
+            .apply(ParDo.of(new BulkIOFn(this)));
+      } else {
+
+        input
+            .apply(
+                "Make elements iterable",
+                MapElements.into(TypeDescriptors.iterables(TypeDescriptors.strings()))
+                    .via(Collections::singletonList))
+            .apply(ParDo.of(new BulkIOFn(this)));
+      }
+
       return PDone.in(input.getPipeline());

Review comment:
       it will be used only with Write which already does the PDone.in

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -257,6 +259,14 @@ void testWrite() throws Exception {
     executeWriteTest(write);
   }
 
+  void testWriteStateful() throws Exception {

Review comment:
       you need to add this test in all ElasticSearchTests.java otherwise they it is not tested.

##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -257,6 +259,14 @@ void testWrite() throws Exception {
     executeWriteTest(write);
   }
 
+  void testWriteStateful() throws Exception {

Review comment:
       please also add test coverage for new features that are not already covered: test upsert, test routing, test doc version, test withMaxParallelRquestsPerWindow.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629649040



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {

Review comment:
       > please to the check with cluster only if the version is not specified and state in the javadoc of withBackendVersion
   
   Done! https://github.com/apache/beam/pull/14347/commits/54f2fbca29b5eac2b74fbca280886c1c15027090
   
   > plus,
   if we offer the ability to have withBackendVersion() to write, it needs to be uniform and offer it to read as well.
   
   Do you mean that we need to `withBackendVersion()` for ElasticsearchIO.Read in this PR? Could we do that in a follow-up instead?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r617016322



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {
+          backendVersion = ElasticsearchIO.getBackendVersion(connectionConfiguration);
+        } else {
+          backendVersion = spec.getBackendVersion();
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext c) throws IOException {
+        c.output(createBulkApiEntity(spec, c.element(), backendVersion));
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransform} convenience wrapper for doing both document to bulk API serialization as
+   * well as batching those Bulk API entities and writing them to an Elasticsearch cluster. This
+   * class is effectively a thin proxy for DocToBulk->BulkIO all-in-one for convenience and backward
+   * compatibility.
+   */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<String>, PDone> {
+    public interface FieldValueExtractFn extends SerializableFunction<JsonNode, String> {}
+
+    public interface BooleanFieldValueExtractFn extends SerializableFunction<JsonNode, Boolean> {}
+
+    public abstract DocToBulk getDocToBulk();
+
+    public abstract BulkIO getBulkIO();
+
+    abstract Builder writeBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setDocToBulk(DocToBulk docToBulk);
+
+      abstract Builder setBulkIO(BulkIO bulkIO);
+
+      abstract Write build();
+    }
+
+    // For building Doc2Bulk
+    /** Refer to {@link DocToBulk#withIdFn}. */
+    public Write withIdFn(FieldValueExtractFn idFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIdFn(idFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withIndexFn}. */
+    public Write withIndexFn(FieldValueExtractFn indexFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIndexFn(indexFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withRoutingFn}. */
+    public Write withRoutingFn(FieldValueExtractFn routingFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withRoutingFn(routingFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withTypeFn}. */
+    public Write withTypeFn(FieldValueExtractFn typeFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withTypeFn(typeFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withDocVersionFn}. */
+    public Write withDocVersionFn(FieldValueExtractFn docVersionFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withDocVersionFn(docVersionFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withDocVersionType}. */
+    public Write withDocVersionType(String docVersionType) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withDocVersionType(docVersionType)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withUsePartialUpdate}. */
+    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+      return writeBuilder()
+          .setDocToBulk(getDocToBulk().withUsePartialUpdate(usePartialUpdate))
+          .build();
+    }
+
+    /** Refer to {@link DocToBulk#withUpsertScript}. */
+    public Write withUpsertScript(String source) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withUpsertScript(source)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withBackendVersion}. */
+    public Write withBackendVersion(int backendVersion) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withBackendVersion(backendVersion)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withIsDeleteFn}. */
+    public Write withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIsDeleteFn(isDeleteFn)).build();
+    }
+    // End building Doc2Bulk
+
+    /** Refer to {@link BulkIO#withConnectionConfiguration}. */
+    public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) {
+      checkArgument(connectionConfiguration != null, "connectionConfiguration can not be null");
+
+      return writeBuilder()
+          .setDocToBulk(getDocToBulk().withConnectionConfiguration(connectionConfiguration))
+          .setBulkIO(getBulkIO().withConnectionConfiguration(connectionConfiguration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBatchSize}. */
+    public Write withMaxBatchSize(long batchSize) {
+      return writeBuilder().setBulkIO(getBulkIO().withMaxBatchSize(batchSize)).build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBatchSizeBytes}. */
+    public Write withMaxBatchSizeBytes(long batchSizeBytes) {
+      return writeBuilder().setBulkIO(getBulkIO().withMaxBatchSizeBytes(batchSizeBytes)).build();
+    }
+
+    /** Refer to {@link BulkIO#withRetryConfiguration}. */
+    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withRetryConfiguration(retryConfiguration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withIgnoreVersionConflicts}. */
+    public Write withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withIgnoreVersionConflicts(ignoreVersionConflicts))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withUseStatefulBatches}. */
+    public Write withUseStatefulBatches(boolean useStatefulBatches) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withUseStatefulBatches(useStatefulBatches))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBufferingDuration}. */
+    public Write withMaxBufferingDuration(Duration maxBufferingDuration) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withMaxBufferingDuration(maxBufferingDuration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxParallelRequestsPerWindow}. */
+    public Write withMaxParallelRquestsPerWindow(int maxParallelRquestsPerWindow) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withMaxParallelRequestsPerWindow(maxParallelRquestsPerWindow))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withAllowableResponseErrors}. */
+    public Write withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrors) {

Review comment:
       This was added for a specific use case with respect to document versioning in particular where the ID is monotonically increasing for new versions of a given document i.e. higher ID always means newer doc.
   
   In that case, it doesn't matter if an older document arrives at an ES cluster after a document with a larger version ID has already been indexed for the same doc ID. So ignoring version conflicts can allow for batches containing an out-of-date documents to complete without raising an error and getting stuck in an infinite retry loop 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r618172398



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -303,6 +341,73 @@ public static ConnectionConfiguration create(String[] addresses, String index, S
           .build();
     }
 
+    /**
+     * Creates a new Elasticsearch connection configuration with no default type.
+     *
+     * @param addresses list of addresses of Elasticsearch nodes
+     * @param index the index toward which the requests will be issued
+     * @return the connection configuration object
+     */
+    public static ConnectionConfiguration create(String[] addresses, String index) {
+      checkArgument(addresses != null, "addresses can not be null");
+      checkArgument(addresses.length > 0, "addresses can not be empty");
+      checkArgument(index != null, "index can not be null");
+      return new AutoValue_ElasticsearchIO_ConnectionConfiguration.Builder()
+          .setAddresses(Arrays.asList(addresses))
+          .setIndex(index)
+          .setType("")
+          .setTrustSelfSignedCerts(false)
+          .build();
+    }
+
+    /**
+     * Creates a new Elasticsearch connection configuration with no default index nor type.
+     *
+     * @param addresses list of addresses of Elasticsearch nodes
+     * @return the connection configuration object
+     */
+    public static ConnectionConfiguration create(String[] addresses) {
+      checkArgument(addresses != null, "addresses can not be null");
+      checkArgument(addresses.length > 0, "addresses can not be empty");
+      return new AutoValue_ElasticsearchIO_ConnectionConfiguration.Builder()
+          .setAddresses(Arrays.asList(addresses))
+          .setIndex("")
+          .setType("")
+          .setTrustSelfSignedCerts(false)
+          .build();
+    }
+
+    /**
+     * Generates the bulk API endpoint based on the set values.
+     *
+     * <p>Based on ConnectionConfiguration constructors, we know that one of the following is true:
+     *
+     * <ul>
+     *   <li>index and type are non-empty strings
+     *   <li>index is non-empty string, type is empty string
+     *   <li>index and type are empty string
+     * </ul>
+     *
+     * <p>Valid endpoints therefore include:
+     *
+     * <ul>
+     *   <li>/_bulk
+     *   <li>/index_name/_bulk
+     *   <li>/index_name/type_name/_bulk
+     * </ul>
+     */
+    public String getBulkEndPoint() {

Review comment:
       ah, yes, the index can be specified in the bulk payload I forgot that.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r618173156



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {

Review comment:
       yes + a warning that mismatch would not work




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-847886759


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-847857771


   @echauchot all set for a final look-over πŸ™‚ 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-827625943


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r640603193



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -568,11 +670,202 @@ void testWritePartialUpdate() throws Exception {
     assertEquals(numDocs, currentNumDocs);
     assertEquals(
         numDocs / NUM_SCIENTISTS,
-        countByScientistName(connectionConfiguration, restClient, "Einstein"));
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
 
     // Partial update assertions
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0"));
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testWriteWithDocVersion() throws Exception {
+    List<ObjectNode> jsonData =
+        ElasticsearchIOTestUtils.createJsonDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    List<String> data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      doc.put("my_version", "1");
+      data.add(doc.toString());
+    }
+
+    insertTestDocuments(connectionConfiguration, data, restClient);
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+    // Check that all docs have the same "my_version"
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withDocVersionFn(new ExtractValueFn("my_version"))
+            .withDocVersionType("external");
+
+    data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      // Set version to larger number than originally set, and larger than next logical version
+      // number set by default by ES.
+      doc.put("my_version", "3");
+      data.add(doc.toString());
+    }
+
+    // Test that documents with lower version are rejected, but rejections ignored when specified
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+
+    // my_version and doc version should have changed
+    assertEquals(
+        0,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "3", null, KV.of(3, numDocs)));
+  }
+
+  /**
+   * Tests upsert script by adding a group field to each document in the standard test set. The
+   * group field is populated as the modulo 2 of the document id allowing for a test to ensure the
+   * documents are split into 2 groups.
+   */
+  void testWriteScriptedUpsert() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withUpsertScript(SCRIPT_SOURCE);
+
+    // Test that documents can be inserted/created by using withUpsertScript
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    // defensive coding to ensure our initial state is as expected
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // All docs should have have group = 0 added by the script upon creation
+    assertEquals(
+        numDocs, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+
+    // Run the same data again. This time, because all docs exist in the index already, scripted
+    // updates should happen rather than scripted inserts.
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+
+    // check we have not unwittingly modified existing behaviour
+    assertEquals(numDocs, currentNumDocs);
+    assertEquals(
+        numDocs / NUM_SCIENTISTS,
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
+
+    // The script will set either 0 or 1 for the group value on update operations
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testMaxParallelRequestsPerWindow() throws Exception {
+    List<String> data =
+        ElasticsearchIOTestUtils.createDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withMaxParallelRequestsPerWindow(1);
+
+    PCollection<KV<Integer, Iterable<String>>> batches =
+        pipeline.apply(Create.of(data)).apply(StatefulBatching.fromSpec(write.getBulkIO()));
+
+    PCollection<Integer> keyValues =
+        batches
+            .apply(GroupByKey.create())

Review comment:
       perfect! 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-824632383


   @egalpin as a general review tip, could you please add or a emoji on each of my comments so that I can know that you have seen/addressed it ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-817680405


   @egalpin starting first round of review sorry for the delay
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r623774847



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1246,88 +1891,182 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. BE AWARE that low values for @param
+     * maxParallelRequestsPerWindow, in particular if the input data has a finite number of windows,
+     * can reduce parallelism greatly. If data is globally windowed and @param
+     * maxParallelRequestsPerWindow is set to 1,there will only ever be 1 request in flight. Having
+     * only a single request in flight can be beneficial for ensuring an Elasticsearch cluster is
+     * not overwhelmed by parallel requests,but may not work for all use cases. If this number is
+     * less than the number of maximum workers in your pipeline, the IO work may not be distributed

Review comment:
       I would say: it will result in an sub-distribution of the last write step with most of the runners.

##########
File path: sdks/java/io/elasticsearch/OWNERS
##########
@@ -4,3 +4,4 @@ reviewers:
   - echauchot
   - jbonofre
   - timrobertson100
+  - egalpin

Review comment:
       you should put your name first IMHO otherwise very little chance to be pinged as reviewer




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r617022230



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1453,14 +2062,23 @@ private HttpEntity handleRetry(
         // while retry policy exists
         while (BackOffUtils.next(sleeper, backoff)) {
           LOG.warn(String.format(RETRY_ATTEMPT_LOG, ++attempt));
-          Request request = new Request(method, endpoint);
-          request.addParameters(params);
-          request.setEntity(requestBody);
-          response = restClient.performRequest(request);
-          responseEntity = new BufferedHttpEntity(response.getEntity());
+          try {
+            Request request = new Request(method, endpoint);
+            request.addParameters(params);
+            request.setEntity(requestBody);
+            response = restClient.performRequest(request);
+            responseEntity = new BufferedHttpEntity(response.getEntity());
+          } catch (java.io.IOException ex) {

Review comment:
       Thanks, great suggestion. I just checked and unfortunately `restClient#performRequest` only throws IOException so we can't explicitly catch something more specific. It does however wrap other exceptions as the cause of the IOException, so I added function to determine what's retryable and what's not. Hope it's alright




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] timrobertson100 commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
timrobertson100 commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-1019269678


   The tests might also be worth looking at @mattwelke 
   
   E.g. 
   https://github.com/apache/beam/blob/master/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java#L770 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-847936490


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-844093298


   Ready to roll! πŸ™‚  


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r620643712



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {

Review comment:
       The only qualm I have with verifying the version is that it creates a dependency on a cluster being available. I had hoped to keep the ability (depending on explicitly setting `withBackendVersion`) for `DocToBulk` to be fully free from IO/dependency on an available cluster.
   
   I could do the checking in `ElasticsearchIO.Write`. Thoughts?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-839771459


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot merged pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot merged pull request #14347:
URL: https://github.com/apache/beam/pull/14347


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r616999346



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -303,6 +341,73 @@ public static ConnectionConfiguration create(String[] addresses, String index, S
           .build();
     }
 
+    /**
+     * Creates a new Elasticsearch connection configuration with no default type.
+     *
+     * @param addresses list of addresses of Elasticsearch nodes
+     * @param index the index toward which the requests will be issued
+     * @return the connection configuration object
+     */
+    public static ConnectionConfiguration create(String[] addresses, String index) {
+      checkArgument(addresses != null, "addresses can not be null");
+      checkArgument(addresses.length > 0, "addresses can not be empty");
+      checkArgument(index != null, "index can not be null");
+      return new AutoValue_ElasticsearchIO_ConnectionConfiguration.Builder()
+          .setAddresses(Arrays.asList(addresses))
+          .setIndex(index)
+          .setType("")
+          .setTrustSelfSignedCerts(false)
+          .build();
+    }
+
+    /**
+     * Creates a new Elasticsearch connection configuration with no default index nor type.
+     *
+     * @param addresses list of addresses of Elasticsearch nodes
+     * @return the connection configuration object
+     */
+    public static ConnectionConfiguration create(String[] addresses) {
+      checkArgument(addresses != null, "addresses can not be null");
+      checkArgument(addresses.length > 0, "addresses can not be empty");
+      return new AutoValue_ElasticsearchIO_ConnectionConfiguration.Builder()
+          .setAddresses(Arrays.asList(addresses))
+          .setIndex("")
+          .setType("")
+          .setTrustSelfSignedCerts(false)
+          .build();
+    }
+
+    /**
+     * Generates the bulk API endpoint based on the set values.
+     *
+     * <p>Based on ConnectionConfiguration constructors, we know that one of the following is true:
+     *
+     * <ul>
+     *   <li>index and type are non-empty strings
+     *   <li>index is non-empty string, type is empty string
+     *   <li>index and type are empty string
+     * </ul>
+     *
+     * <p>Valid endpoints therefore include:
+     *
+     * <ul>
+     *   <li>/_bulk
+     *   <li>/index_name/_bulk
+     *   <li>/index_name/type_name/_bulk
+     * </ul>
+     */
+    public String getBulkEndPoint() {

Review comment:
       I just went to check the API docs and it looks like there's mention of the 3 formats in the 1.x docs[1] Β―\\\_(ツ)_/Β― This should help alleviate some compatibility issues with ES 8.x where the notion of `type` will be fully removed.
   
   [1] https://www.elastic.co/guide/en/elasticsearch/reference/1.7/docs-bulk.html#docs-bulk




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-827572166


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629669794



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -265,6 +275,13 @@ void testWriteWithErrors() throws Exception {
     List<String> input =
         ElasticsearchIOTestUtils.createDocuments(
             numDocs, ElasticsearchIOTestUtils.InjectionMode.INJECT_SOME_INVALID_DOCS);
+

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-827258229


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629659689



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -568,11 +666,167 @@ void testWritePartialUpdate() throws Exception {
     assertEquals(numDocs, currentNumDocs);
     assertEquals(
         numDocs / NUM_SCIENTISTS,
-        countByScientistName(connectionConfiguration, restClient, "Einstein"));
+        countByScientistName(connectionConfiguration, restClient, "Einstein", null));
 
     // Partial update assertions
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0"));
-    assertEquals(numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1"));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "0", null, null));
+    assertEquals(
+        numDocs / 2, countByMatch(connectionConfiguration, restClient, "group", "1", null, null));
+  }
+
+  void testWriteWithDocVersion() throws Exception {
+    List<ObjectNode> jsonData =
+        ElasticsearchIOTestUtils.createJsonDocuments(
+            numDocs, ElasticsearchIOTestUtils.InjectionMode.DO_NOT_INJECT_INVALID_DOCS);
+
+    List<String> data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      doc.put("my_version", "1");
+      data.add(doc.toString());
+    }
+
+    insertTestDocuments(connectionConfiguration, data, restClient);
+    long currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+    // Check that all docs have the same "my_version"
+    assertEquals(
+        numDocs,
+        countByMatch(
+            connectionConfiguration, restClient, "my_version", "1", null, KV.of(1, numDocs)));
+
+    Write write =
+        ElasticsearchIO.write()
+            .withConnectionConfiguration(connectionConfiguration)
+            .withIdFn(new ExtractValueFn("id"))
+            .withDocVersionFn(new ExtractValueFn("my_version"))
+            .withDocVersionType("external");
+
+    data = new ArrayList<>();
+    for (ObjectNode doc : jsonData) {
+      // Set version to larger number than originally set, and larger than next logical version
+      // number set by default by ES.
+      doc.put("my_version", "3");
+      data.add(doc.toString());
+    }
+
+    // Test that documents with lower version are rejected, but rejections ignored when specified
+    pipeline.apply(Create.of(data)).apply(write);
+    pipeline.run();
+
+    currentNumDocs = refreshIndexAndGetCurrentNumDocs(connectionConfiguration, restClient);
+    assertEquals(numDocs, currentNumDocs);
+
+    // my_version and doc version should have changed
+    assertEquals(

Review comment:
       The version numbers are checked via `countByMatch`[1] πŸ‘ 
   
   [1] https://github.com/apache/beam/pull/14347/files#diff-0971ee817a63d0687f2f2977c2c247d5d56c654efa036718cb616f0d49a30f20R379-R388




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r617019346



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1203,88 +1796,160 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. If data is globally windowed and this
+     * configuration is set to 1, there will only ever be 1 request in flight. Having only a single
+     * request in flight can be beneficial for ensuring an Elasticsearch cluster is not overwhelmed
+     * by parallel requests, but may not work for all use cases. If this number is less than the
+     * number of maximum workers in your pipeline, the IO work may not be distributed across all
+     * workers.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =
+            GroupIntoBatches.ofSize(getMaxBatchSize());
+
+        if (getMaxBufferingDuration() != null) {
+          groupIntoBatches = groupIntoBatches.withMaxBufferingDuration(getMaxBufferingDuration());
+        }
+        input
+            .apply(ParDo.of(new AssignShardFn<>(getMaxParallelRequestsPerWindow())))

Review comment:
       Ok I'll revert to having 2 separate classes and a base class πŸ‘ 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r620385967



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -158,10 +167,16 @@
 })
 public class ElasticsearchIO {

Review comment:
       Added in a local commit, just sorting out one last test before I push updates.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r616962376



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1203,88 +1796,160 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. If data is globally windowed and this
+     * configuration is set to 1, there will only ever be 1 request in flight. Having only a single
+     * request in flight can be beneficial for ensuring an Elasticsearch cluster is not overwhelmed
+     * by parallel requests, but may not work for all use cases. If this number is less than the
+     * number of maximum workers in your pipeline, the IO work may not be distributed across all
+     * workers.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =
+            GroupIntoBatches.ofSize(getMaxBatchSize());
+
+        if (getMaxBufferingDuration() != null) {
+          groupIntoBatches = groupIntoBatches.withMaxBufferingDuration(getMaxBufferingDuration());
+        }
+        input
+            .apply(ParDo.of(new AssignShardFn<>(getMaxParallelRequestsPerWindow())))
+            .apply(groupIntoBatches)
+            .apply(
+                "Remove key no longer needed",
+                MapElements.into(TypeDescriptors.iterables(TypeDescriptors.strings()))
+                    .via(KV::getValue))
+            .apply(ParDo.of(new BulkIOFn(this)));
+      } else {
+
+        input
+            .apply(
+                "Make elements iterable",
+                MapElements.into(TypeDescriptors.iterables(TypeDescriptors.strings()))
+                    .via(Collections::singletonList))
+            .apply(ParDo.of(new BulkIOFn(this)));
+      }
+
       return PDone.in(input.getPipeline());

Review comment:
       In cases where a user were to use separate DocToBulk and BulkIO transforms, would we still need this PDone? In that case, maybe I should remove the PDone from `Write` ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-818754924


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-824628922


   > @echauchot Thanks for the review, I'll work my way through your comments and suggestions.
   > 
   > > Besides, Evan, as you know ES very well, and you seem to be interested in contributing. Would you be interested in putting yourself in ES Owners file and jira ES label ?
   > 
   > I'd be very happy to  I've added myself to the ES owners file now, happy to lend a hand reviewing! Thanks 
   > 
   > With respect to Jira, could you please add appropriate permissions for me to either assign myself to the ES label, or assign me to the label yourself if that is the preferred workflow. I have an account on issues.apache.org/jira but only with permission to create tickets I believe.
   
   I just added you to the contributor role and as lead on elasticsearch component. @kennknowles please let me know if there is any problem assigning a non-committer to leader on a component. If such I'll put myself back. If not, @egalpin feel free to ping me if you have questions related to the IO in the future.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-827609998


   @echauchot The ES tests when running in Jenkins seem non-deterministic. I'm not sure if that's caused by something new I've introduced, or if that was always the case previously. Running in the dockerized build environment container, all ElasticsearchIOTest suites tests pass reliably. I'm not sure what's different in Jenkins but it seems that every run has a different result. Any thoughts or advice?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r617019601



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1203,88 +1796,160 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. If data is globally windowed and this
+     * configuration is set to 1, there will only ever be 1 request in flight. Having only a single
+     * request in flight can be beneficial for ensuring an Elasticsearch cluster is not overwhelmed
+     * by parallel requests, but may not work for all use cases. If this number is less than the
+     * number of maximum workers in your pipeline, the IO work may not be distributed across all
+     * workers.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =
+            GroupIntoBatches.ofSize(getMaxBatchSize());
+
+        if (getMaxBufferingDuration() != null) {
+          groupIntoBatches = groupIntoBatches.withMaxBufferingDuration(getMaxBufferingDuration());
+        }
+        input
+            .apply(ParDo.of(new AssignShardFn<>(getMaxParallelRequestsPerWindow())))
+            .apply(groupIntoBatches)
+            .apply(
+                "Remove key no longer needed",
+                MapElements.into(TypeDescriptors.iterables(TypeDescriptors.strings()))
+                    .via(KV::getValue))
+            .apply(ParDo.of(new BulkIOFn(this)));
+      } else {
+
+        input
+            .apply(
+                "Make elements iterable",
+                MapElements.into(TypeDescriptors.iterables(TypeDescriptors.strings()))

Review comment:
       Ok I'll revert to having 2 separate classes and a base class πŸ‘ 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r618163429



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1203,88 +1796,160 @@ public Write withUsePartialUpdate(boolean usePartialUpdate) {
      * }</pre>
      *
      * @param retryConfiguration the rules which govern the retry behavior
-     * @return the {@link Write} with retrying configured
+     * @return the {@link BulkIO} with retrying configured
      */
-    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+    public BulkIO withRetryConfiguration(RetryConfiguration retryConfiguration) {
       checkArgument(retryConfiguration != null, "retryConfiguration is required");
       return builder().setRetryConfiguration(retryConfiguration).build();
     }
 
     /**
-     * Provide a function to extract the target operation either upsert or delete from the document
-     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
-     * taken care that the document's id extraction is defined using the withIdFn function or else
-     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
-     * will fail and the exception propagated.
+     * Whether or not to suppress version conflict errors in a Bulk API response. This can be useful
+     * if your use case involves using external version types.
      *
-     * @param isDeleteFn set to true for deleting the specific document
-     * @return the {@link Write} with the function set
+     * @param ignoreVersionConflicts true to suppress version conflicts, false to surface version
+     *     conflict errors.
+     * @return the {@link BulkIO} with version conflict handling configured
      */
-    public Write withIsDeleteFn(BooleanFieldValueExtractFn isDeleteFn) {
-      checkArgument(isDeleteFn != null, "deleteFn is required");
-      return builder().setIsDeleteFn(isDeleteFn).build();
+    public BulkIO withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      Set<String> allowedResponseErrors = getAllowedResponseErrors();
+      if (allowedResponseErrors == null) {
+        allowedResponseErrors = new HashSet<>();
+      }
+      if (ignoreVersionConflicts) {
+        allowedResponseErrors.add(VERSION_CONFLICT_ERROR);
+      }
+
+      return builder().setAllowedResponseErrors(allowedResponseErrors).build();
+    }
+
+    /**
+     * Provide a set of textual error types which can be contained in Bulk API response
+     * items[].error.type field. Any element in @param allowableResponseErrorTypes will suppress
+     * errors of the same type in Bulk responses.
+     *
+     * <p>See also
+     * https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-failures-ex
+     *
+     * @param allowableResponseErrorTypes
+     * @return the {@link BulkIO} with allowable response errors set
+     */
+    public BulkIO withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrorTypes) {
+      if (allowableResponseErrorTypes == null) {
+        allowableResponseErrorTypes = new HashSet<>();
+      }
+
+      return builder().setAllowedResponseErrors(allowableResponseErrorTypes).build();
+    }
+
+    /**
+     * If using {@link BulkIO#withUseStatefulBatches}, this can be used to set a maximum elapsed
+     * time before buffered elements are emitted to Elasticsearch as a Bulk API request. If this
+     * config is not set, Bulk requests will not be issued until {@link BulkIO#getMaxBatchSize}
+     * number of documents have been buffered. This may result in higher latency in particular if
+     * your max batch size is set to a large value and your pipeline input is low volume.
+     *
+     * @param maxBufferingDuration the maximum duration to wait before sending any buffered
+     *     documents to Elasticsearch, regardless of maxBatchSize.
+     * @return the {@link BulkIO} with maximum buffering duration set
+     */
+    public BulkIO withMaxBufferingDuration(Duration maxBufferingDuration) {
+      LOG.warn(
+          "Use of withMaxBufferingDuration requires withUseStatefulBatches(true). "
+              + "Setting that automatically.");
+      return builder()
+          .setUseStatefulBatches(true)
+          .setMaxBufferingDuration(maxBufferingDuration)
+          .build();
+    }
+
+    /**
+     * Whether or not to use Stateful Processing to ensure bulk requests have the desired number of
+     * entities i.e. as close to the maxBatchSize as possible. By default without this feature
+     * enabled, Bulk requests will not contain more than maxBatchSize entities, but the lower bound
+     * of batch size is determined by Beam Runner bundle sizes, which may be as few as 1.
+     *
+     * @param useStatefulBatches true enables the use of Stateful Processing to ensure that batches
+     *     are as close to the maxBatchSize as possible.
+     * @return the {@link BulkIO} with Stateful Processing enabled or disabled
+     */
+    public BulkIO withUseStatefulBatches(boolean useStatefulBatches) {
+      return builder().setUseStatefulBatches(useStatefulBatches).build();
+    }
+
+    /**
+     * When using {@link BulkIO#withUseStatefulBatches} Stateful Processing, states and therefore
+     * batches are maintained per-key-per-window. If data is globally windowed and this
+     * configuration is set to 1, there will only ever be 1 request in flight. Having only a single
+     * request in flight can be beneficial for ensuring an Elasticsearch cluster is not overwhelmed
+     * by parallel requests, but may not work for all use cases. If this number is less than the
+     * number of maximum workers in your pipeline, the IO work may not be distributed across all
+     * workers.
+     *
+     * @param maxParallelRequestsPerWindow the maximum number of parallel bulk requests for a window
+     *     of data
+     * @return the {@link BulkIO} with maximum parallel bulk requests per window set
+     */
+    public BulkIO withMaxParallelRequestsPerWindow(int maxParallelRequestsPerWindow) {
+      checkArgument(
+          maxParallelRequestsPerWindow > 0, "parameter value must be positive " + "a integer");
+      return builder().setMaxParallelRequestsPerWindow(maxParallelRequestsPerWindow).build();
     }
 
     @Override
     public PDone expand(PCollection<String> input) {
       ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
-      FieldValueExtractFn idFn = getIdFn();
-      BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
       checkState(connectionConfiguration != null, "withConnectionConfiguration() is required");
-      checkArgument(
-          isDeleteFn == null || idFn != null,
-          "Id needs to be specified by withIdFn for delete operation");
-      input.apply(ParDo.of(new WriteFn(this)));
+
+      if (getUseStatefulBatches()) {
+        GroupIntoBatches<Integer, String> groupIntoBatches =
+            GroupIntoBatches.ofSize(getMaxBatchSize());
+
+        if (getMaxBufferingDuration() != null) {
+          groupIntoBatches = groupIntoBatches.withMaxBufferingDuration(getMaxBufferingDuration());
+        }
+        input
+            .apply(ParDo.of(new AssignShardFn<>(getMaxParallelRequestsPerWindow())))
+            .apply(groupIntoBatches)
+            .apply(
+                "Remove key no longer needed",
+                MapElements.into(TypeDescriptors.iterables(TypeDescriptors.strings()))
+                    .via(KV::getValue))
+            .apply(ParDo.of(new BulkIOFn(this)));
+      } else {
+
+        input
+            .apply(
+                "Make elements iterable",
+                MapElements.into(TypeDescriptors.iterables(TypeDescriptors.strings()))
+                    .via(Collections::singletonList))
+            .apply(ParDo.of(new BulkIOFn(this)));
+      }
+
       return PDone.in(input.getPipeline());

Review comment:
       yes remove from write: if BulkIO is used separately it will work and if used in Write it will also work as BulkIO is the last transform applied in Write




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629351489



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1453,14 +2062,23 @@ private HttpEntity handleRetry(
         // while retry policy exists
         while (BackOffUtils.next(sleeper, backoff)) {
           LOG.warn(String.format(RETRY_ATTEMPT_LOG, ++attempt));
-          Request request = new Request(method, endpoint);
-          request.addParameters(params);
-          request.setEntity(requestBody);
-          response = restClient.performRequest(request);
-          responseEntity = new BufferedHttpEntity(response.getEntity());
+          try {
+            Request request = new Request(method, endpoint);
+            request.addParameters(params);
+            request.setEntity(requestBody);
+            response = restClient.performRequest(request);
+            responseEntity = new BufferedHttpEntity(response.getEntity());
+          } catch (java.io.IOException ex) {

Review comment:
       seems good in general concept, but please check that neither _ConnectTimeoutException_ nor _ConnectException_ are thrown in case of bad configuration.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r618176482



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1177,12 +1272,513 @@ public Write withTypeFn(FieldValueExtractFn typeFn) {
      * Elasticsearch.
      *
      * @param usePartialUpdate set to true to issue partial updates
-     * @return the {@link Write} with the partial update control set
+     * @return the {@link DocToBulk} with the partial update control set
      */
-    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+    public DocToBulk withUsePartialUpdate(boolean usePartialUpdate) {
       return builder().setUsePartialUpdate(usePartialUpdate).build();
     }
 
+    /**
+     * Whether to use scripted updates and what script to use.
+     *
+     * @param source set to the value of the script source, painless lang
+     * @return the {@link DocToBulk} with the scripted updates set
+     */
+    public DocToBulk withUpsertScript(String source) {
+      return builder().setUsePartialUpdate(false).setUpsertScript(source).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionFn to extract the document version
+     * @return the {@link DocToBulk} with the function set
+     */
+    public DocToBulk withDocVersionFn(Write.FieldValueExtractFn docVersionFn) {
+      checkArgument(docVersionFn != null, "docVersionFn must not be null");
+      return builder().setDocVersionFn(docVersionFn).build();
+    }
+
+    /**
+     * Provide a function to extract the target operation either upsert or delete from the document
+     * fields allowing dynamic bulk operation decision. While using withIsDeleteFn, it should be
+     * taken care that the document's id extraction is defined using the withIdFn function or else
+     * IllegalArgumentException is thrown. Should the function throw an Exception then the batch
+     * will fail and the exception propagated.
+     *
+     * @param isDeleteFn set to true for deleting the specific document
+     * @return the {@link Write} with the function set
+     */
+    public DocToBulk withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      checkArgument(isDeleteFn != null, "deleteFn is required");
+      return builder().setIsDeleteFn(isDeleteFn).build();
+    }
+
+    /**
+     * Provide a function to extract the doc version from the document. This version number will be
+     * used as the document version in Elasticsearch. Should the function throw an Exception then
+     * the batch will fail and the exception propagated. Incompatible with update operations and
+     * should only be used with withUsePartialUpdate(false)
+     *
+     * @param docVersionType the version type to use, one of {@value ElasticsearchIO#VERSION_TYPES}
+     * @return the {@link DocToBulk} with the doc version type set
+     */
+    public DocToBulk withDocVersionType(String docVersionType) {
+      checkArgument(
+          VERSION_TYPES.contains(docVersionType),
+          "docVersionType must be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setDocVersionType(docVersionType).build();
+    }
+
+    /**
+     * Use to set explicitly which version of Elasticsearch the destination cluster is running.
+     * Providing this hint means there is no need for setting {@link
+     * DocToBulk#withConnectionConfiguration}. This can also be very useful for testing purposes.
+     *
+     * @param backendVersion the major version number of the version of Elasticsearch being run in
+     *     the cluster where documents will be indexed.
+     * @return the {@link DocToBulk} with the Elasticsearch major version number set
+     */
+    public DocToBulk withBackendVersion(int backendVersion) {
+      checkArgument(
+          VALID_CLUSTER_VERSIONS.contains(backendVersion),
+          "Backend version may only be one of " + "%s",
+          String.join(", ", VERSION_TYPES));
+      return builder().setBackendVersion(backendVersion).build();
+    }
+
+    @Override
+    public PCollection<String> expand(PCollection<String> docs) {
+      ConnectionConfiguration connectionConfiguration = getConnectionConfiguration();
+      Integer backendVersion = getBackendVersion();
+      Write.FieldValueExtractFn idFn = getIdFn();
+      Write.BooleanFieldValueExtractFn isDeleteFn = getIsDeleteFn();
+      checkState(
+          (backendVersion != null || connectionConfiguration != null),
+          "withBackendVersion() or withConnectionConfiguration() is required");
+      checkArgument(
+          isDeleteFn == null || idFn != null,
+          "Id needs to be specified by withIdFn for delete operation");
+
+      return docs.apply(ParDo.of(new DocToBulkFn(this)));
+    }
+
+    // Encapsulates the elements which form the metadata for an Elasticsearch bulk operation
+    private static class DocumentMetadata implements Serializable {
+      final String index;
+      final String type;
+      final String id;
+      final Integer retryOnConflict;
+      final String routing;
+      final Integer backendVersion;
+      final String version;
+      final String versionType;
+
+      DocumentMetadata(
+          String index,
+          String type,
+          String id,
+          Integer retryOnConflict,
+          String routing,
+          Integer backendVersion,
+          String version,
+          String versionType) {
+        this.index = index;
+        this.id = id;
+        this.type = type;
+        this.retryOnConflict = retryOnConflict;
+        this.routing = routing;
+        this.backendVersion = backendVersion;
+        this.version = version;
+        this.versionType = versionType;
+      }
+    }
+
+    private static class DocumentMetadataSerializer extends StdSerializer<DocumentMetadata> {
+      private DocumentMetadataSerializer() {
+        super(DocumentMetadata.class);
+      }
+
+      @Override
+      public void serialize(DocumentMetadata value, JsonGenerator gen, SerializerProvider provider)
+          throws IOException {
+        gen.writeStartObject();
+        if (value.index != null) {
+          gen.writeStringField("_index", value.index);
+        }
+        if (value.type != null) {
+          gen.writeStringField("_type", value.type);
+        }
+        if (value.id != null) {
+          gen.writeStringField("_id", value.id);
+        }
+        if (value.routing != null) {
+          gen.writeStringField("routing", value.routing);
+        }
+        if (value.retryOnConflict != null && value.backendVersion <= 6) {
+          gen.writeNumberField("_retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.retryOnConflict != null && value.backendVersion >= 7) {
+          gen.writeNumberField("retry_on_conflict", value.retryOnConflict);
+        }
+        if (value.version != null) {
+          gen.writeStringField("version", value.version);
+        }
+        if (value.versionType != null) {
+          gen.writeStringField("version_type", value.versionType);
+        }
+        gen.writeEndObject();
+      }
+    }
+
+    @VisibleForTesting
+    static String createBulkApiEntity(DocToBulk spec, String document, int backendVersion)
+        throws IOException {
+      String documentMetadata = "{}";
+      boolean isDelete = false;
+      if (spec.getIndexFn() != null || spec.getTypeFn() != null || spec.getIdFn() != null) {
+        // parse once and reused for efficiency
+        JsonNode parsedDocument = OBJECT_MAPPER.readTree(document);
+        documentMetadata = getDocumentMetadata(spec, parsedDocument, backendVersion);
+        if (spec.getIsDeleteFn() != null) {
+          isDelete = spec.getIsDeleteFn().apply(parsedDocument);
+        }
+      }
+
+      if (isDelete) {
+        // delete request used for deleting a document
+        return String.format("{ \"delete\" : %s }%n", documentMetadata);
+      } else {
+        // index is an insert/upsert and update is a partial update (or insert if not
+        // existing)
+        if (spec.getUsePartialUpdate()) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"doc\" : %s, " + "\"doc_as_upsert\" : true }%n",
+              documentMetadata, document);
+        } else if (spec.getUpsertScript() != null) {
+          return String.format(
+              "{ \"update\" : %s }%n{ \"script\" : {\"source\": \"%s\", "
+                  + "\"params\": %s}, \"upsert\" : %s }%n",
+              documentMetadata, spec.getUpsertScript(), document, document);
+        } else {
+          return String.format("{ \"index\" : %s }%n%s%n", documentMetadata, document);
+        }
+      }
+    }
+
+    private static String lowerCaseOrNull(String input) {
+      return input == null ? null : input.toLowerCase();
+    }
+
+    /**
+     * Extracts the components that comprise the document address from the document using the {@link
+     * Write.FieldValueExtractFn} configured. This allows any or all of the index, type and document
+     * id to be controlled on a per document basis. If none are provided then an empty default of
+     * {@code {}} is returned. Sanitization of the index is performed, automatically lower-casing
+     * the value as required by Elasticsearch.
+     *
+     * @param parsedDocument the json from which the index, type and id may be extracted
+     * @return the document address as JSON or the default
+     * @throws IOException if the document cannot be parsed as JSON
+     */
+    private static String getDocumentMetadata(
+        DocToBulk spec, JsonNode parsedDocument, int backendVersion) throws IOException {
+      DocumentMetadata metadata =
+          new DocumentMetadata(
+              spec.getIndexFn() != null
+                  ? lowerCaseOrNull(spec.getIndexFn().apply(parsedDocument))
+                  : null,
+              spec.getTypeFn() != null ? spec.getTypeFn().apply(parsedDocument) : null,
+              spec.getIdFn() != null ? spec.getIdFn().apply(parsedDocument) : null,
+              (spec.getUsePartialUpdate()
+                      || (spec.getUpsertScript() != null && !spec.getUpsertScript().isEmpty()))
+                  ? DEFAULT_RETRY_ON_CONFLICT
+                  : null,
+              spec.getRoutingFn() != null ? spec.getRoutingFn().apply(parsedDocument) : null,
+              backendVersion,
+              spec.getDocVersionFn() != null ? spec.getDocVersionFn().apply(parsedDocument) : null,
+              spec.getDocVersionType());
+      return OBJECT_MAPPER.writeValueAsString(metadata);
+    }
+
+    /** {@link DoFn} to for the {@link DocToBulk} transform. */
+    @VisibleForTesting
+    static class DocToBulkFn extends DoFn<String, String> {
+      private final DocToBulk spec;
+      private int backendVersion;
+
+      public DocToBulkFn(DocToBulk spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() throws IOException {
+        ConnectionConfiguration connectionConfiguration = spec.getConnectionConfiguration();
+        if (spec.getBackendVersion() == null) {
+          backendVersion = ElasticsearchIO.getBackendVersion(connectionConfiguration);
+        } else {
+          backendVersion = spec.getBackendVersion();
+        }
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext c) throws IOException {
+        c.output(createBulkApiEntity(spec, c.element(), backendVersion));
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransform} convenience wrapper for doing both document to bulk API serialization as
+   * well as batching those Bulk API entities and writing them to an Elasticsearch cluster. This
+   * class is effectively a thin proxy for DocToBulk->BulkIO all-in-one for convenience and backward
+   * compatibility.
+   */
+  @AutoValue
+  public abstract static class Write extends PTransform<PCollection<String>, PDone> {
+    public interface FieldValueExtractFn extends SerializableFunction<JsonNode, String> {}
+
+    public interface BooleanFieldValueExtractFn extends SerializableFunction<JsonNode, Boolean> {}
+
+    public abstract DocToBulk getDocToBulk();
+
+    public abstract BulkIO getBulkIO();
+
+    abstract Builder writeBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder {
+      abstract Builder setDocToBulk(DocToBulk docToBulk);
+
+      abstract Builder setBulkIO(BulkIO bulkIO);
+
+      abstract Write build();
+    }
+
+    // For building Doc2Bulk
+    /** Refer to {@link DocToBulk#withIdFn}. */
+    public Write withIdFn(FieldValueExtractFn idFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIdFn(idFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withIndexFn}. */
+    public Write withIndexFn(FieldValueExtractFn indexFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIndexFn(indexFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withRoutingFn}. */
+    public Write withRoutingFn(FieldValueExtractFn routingFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withRoutingFn(routingFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withTypeFn}. */
+    public Write withTypeFn(FieldValueExtractFn typeFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withTypeFn(typeFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withDocVersionFn}. */
+    public Write withDocVersionFn(FieldValueExtractFn docVersionFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withDocVersionFn(docVersionFn)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withDocVersionType}. */
+    public Write withDocVersionType(String docVersionType) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withDocVersionType(docVersionType)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withUsePartialUpdate}. */
+    public Write withUsePartialUpdate(boolean usePartialUpdate) {
+      return writeBuilder()
+          .setDocToBulk(getDocToBulk().withUsePartialUpdate(usePartialUpdate))
+          .build();
+    }
+
+    /** Refer to {@link DocToBulk#withUpsertScript}. */
+    public Write withUpsertScript(String source) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withUpsertScript(source)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withBackendVersion}. */
+    public Write withBackendVersion(int backendVersion) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withBackendVersion(backendVersion)).build();
+    }
+
+    /** Refer to {@link DocToBulk#withIsDeleteFn}. */
+    public Write withIsDeleteFn(Write.BooleanFieldValueExtractFn isDeleteFn) {
+      return writeBuilder().setDocToBulk(getDocToBulk().withIsDeleteFn(isDeleteFn)).build();
+    }
+    // End building Doc2Bulk
+
+    /** Refer to {@link BulkIO#withConnectionConfiguration}. */
+    public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) {
+      checkArgument(connectionConfiguration != null, "connectionConfiguration can not be null");
+
+      return writeBuilder()
+          .setDocToBulk(getDocToBulk().withConnectionConfiguration(connectionConfiguration))
+          .setBulkIO(getBulkIO().withConnectionConfiguration(connectionConfiguration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBatchSize}. */
+    public Write withMaxBatchSize(long batchSize) {
+      return writeBuilder().setBulkIO(getBulkIO().withMaxBatchSize(batchSize)).build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBatchSizeBytes}. */
+    public Write withMaxBatchSizeBytes(long batchSizeBytes) {
+      return writeBuilder().setBulkIO(getBulkIO().withMaxBatchSizeBytes(batchSizeBytes)).build();
+    }
+
+    /** Refer to {@link BulkIO#withRetryConfiguration}. */
+    public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withRetryConfiguration(retryConfiguration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withIgnoreVersionConflicts}. */
+    public Write withIgnoreVersionConflicts(boolean ignoreVersionConflicts) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withIgnoreVersionConflicts(ignoreVersionConflicts))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withUseStatefulBatches}. */
+    public Write withUseStatefulBatches(boolean useStatefulBatches) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withUseStatefulBatches(useStatefulBatches))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxBufferingDuration}. */
+    public Write withMaxBufferingDuration(Duration maxBufferingDuration) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withMaxBufferingDuration(maxBufferingDuration))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withMaxParallelRequestsPerWindow}. */
+    public Write withMaxParallelRquestsPerWindow(int maxParallelRquestsPerWindow) {
+      return writeBuilder()
+          .setBulkIO(getBulkIO().withMaxParallelRequestsPerWindow(maxParallelRquestsPerWindow))
+          .build();
+    }
+
+    /** Refer to {@link BulkIO#withAllowableResponseErrors}. */
+    public Write withAllowableResponseErrors(@Nullable Set<String> allowableResponseErrors) {

Review comment:
       LGTM




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-831993549


   note that behavior of `withMaxParallelRequestsPerWindow` depends on https://github.com/apache/beam/pull/14720


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r637044281



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -532,6 +602,34 @@ void testWriteWithFullAddressing() throws Exception {
     }
   }
 
+  /**
+   * Tests that documents are correctly routed when index, type and document ID functions are
+   * provided to overwrite the defaults of using the configuration and auto-generation of the
+   * document IDs by Elasticsearch. The scientist name is used for the index, type and document ID.

Review comment:
       same as above it asserts count = numDocs / NUM_SCIENTISTS so javadoc is to be updated no ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] egalpin commented on pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
egalpin commented on pull request #14347:
URL: https://github.com/apache/beam/pull/14347#issuecomment-815993586


   @echauchot FYI build is passing now πŸ™‚ 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r618164724



##########
File path: sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -1453,14 +2062,23 @@ private HttpEntity handleRetry(
         // while retry policy exists
         while (BackOffUtils.next(sleeper, backoff)) {
           LOG.warn(String.format(RETRY_ATTEMPT_LOG, ++attempt));
-          Request request = new Request(method, endpoint);
-          request.addParameters(params);
-          request.setEntity(requestBody);
-          response = restClient.performRequest(request);
-          responseEntity = new BufferedHttpEntity(response.getEntity());
+          try {
+            Request request = new Request(method, endpoint);
+            request.addParameters(params);
+            request.setEntity(requestBody);
+            response = restClient.performRequest(request);
+            responseEntity = new BufferedHttpEntity(response.getEntity());
+          } catch (java.io.IOException ex) {

Review comment:
       that seems good to me, need to check the code




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] echauchot commented on a change in pull request #14347: [BEAM-12093] Overhaul ElasticsearchIO.Write

Posted by GitBox <gi...@apache.org>.
echauchot commented on a change in pull request #14347:
URL: https://github.com/apache/beam/pull/14347#discussion_r629433650



##########
File path: sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
##########
@@ -257,6 +259,14 @@ void testWrite() throws Exception {
     executeWriteTest(write);
   }
 
+  void testWriteStateful() throws Exception {

Review comment:
       modulo minor fixes in code comments:
   upsert test: LGTM
   routing test: LGTM
   doc version test: LGTM
   parallel test: see comments




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org