egalpin commented on a change in pull request #17112:
URL: https://github.com/apache/beam/pull/17112#discussion_r831401872
##########
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 thanks for the follow up, I really appreciate your time and input
as I want to make the best fix possible here. Agreed that I would need to add
something to ResultFilterFn like:
```java
@Override
public Duration getAllowedTimestampSkew() {
return Duration.millis(Long.MAX_VALUE);
}
```
But I still wonder, as you point out, whether or not watermarks are updated
during the processing of, or only between the processing of, bundles. In the
case of a runner like Flink with no notion of bundles, or in an imaginary
runner where bundles were always of size 1, I suppose maybe the logic still
holds true? Only after every bundle, i.e. after every element, has been
processed can the watermark be updated. ElasticsearchIO would be ok in that
case as well, because `@FinishBundle` would output all elements before
processing the element was done which would also imply being output before the
watermark was updated? Thinking out loud 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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]