egalpin commented on a change in pull request #17112:
URL: https://github.com/apache/beam/pull/17112#discussion_r833670820



##########
File path: 
sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
##########
@@ -2246,27 +2286,43 @@ public static StatefulBatching fromSpec(BulkIO spec) {
         }
 
         return input
-            .apply(ParDo.of(new 
Reshuffle.AssignShardFn<>(spec.getMaxParallelRequestsPerWindow())))
+            .apply(ParDo.of(new 
Reshuffle.AssignShardFn<>(spec.getMaxParallelRequests())))
             .apply(groupIntoBatches);
       }
     }
 
     @Override
     public PCollectionTuple expand(PCollection<Document> input) {
       ConnectionConfiguration connectionConfiguration = 
getConnectionConfiguration();
-
       checkState(connectionConfiguration != null, 
"withConnectionConfiguration() is required");
 
+      PCollection<Document> docResults;
+      PCollection<Document> globalDocs = input.apply(Window.into(new 
GlobalWindows()));
+
       if (getUseStatefulBatches()) {
-        return input
-            .apply(StatefulBatching.fromSpec(this))
-            .apply(
-                ParDo.of(new BulkIOStatefulFn(this))
-                    .withOutputTags(Write.SUCCESSFUL_WRITES, 
TupleTagList.of(Write.FAILED_WRITES)));
+        docResults =
+            globalDocs
+                .apply(StatefulBatching.fromSpec(this))
+                .apply(ParDo.of(new BulkIOStatefulFn(this)));
       } else {
-        return input.apply(
-            ParDo.of(new BulkIOBundleFn(this))
-                .withOutputTags(Write.SUCCESSFUL_WRITES, 
TupleTagList.of(Write.FAILED_WRITES)));
+        docResults = globalDocs.apply(ParDo.of(new BulkIOBundleFn(this)));
+      }
+
+      return docResults
+          .setWindowingStrategyInternal(input.getWindowingStrategy())
+          .apply(
+              ParDo.of(new ResultFilteringFn())
+                  .withOutputTags(Write.SUCCESSFUL_WRITES, 
TupleTagList.of(Write.FAILED_WRITES)));
+    }
+
+    private static class ResultFilteringFn extends DoFn<Document, Document> {
+      @ProcessElement
+      public void processElement(@Element Document doc, MultiOutputReceiver 
out) {
+        if (doc.getHasError()) {
+          out.get(Write.FAILED_WRITES).outputWithTimestamp(doc, 
doc.getTimestamp());
+        } else {
+          out.get(Write.SUCCESSFUL_WRITES).outputWithTimestamp(doc, 
doc.getTimestamp());

Review comment:
       @je-ik What I meant to convey was that I think the solution as laid out 
in this PR may be resilient to runners where the watermark is updated after 
every element _iff_ we assume that this implies calling `@StartBundle` and 
`@FinishBundle` for each element (i.e. as if bundle size was enforced to be 
always 1).  Such an environment would cause all buffered elements to be flushed 
(via `@FinishBundle`) in the context of the window from which the elements 
originated before the watermark was updated.
   
   It sounds like this notion is supported based on comments in the related 
dev@ thread[1].
   
   [1] https://lists.apache.org/thread/7foy455spg43xo77zhrs62gc1m383t50 




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to