je-ik commented on a change in pull request #17112:
URL: https://github.com/apache/beam/pull/17112#discussion_r834063058
##########
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:
I think we are going in cycles here. :) Maybe we could start a new dev@
thread to help clarifying things out. FlinkRunner definitely does not have
bundles of size 1 (that is what would mean to call @StartBundle and
@FinishBundle) before/after each element. That would make all "in-memory"
batching void on such runner.
I think it implies one of two possibilities:
a) either the watermark update only on boundaries of bundles is *not* part
of the model, or
b) FlinkRunner does not adhere to this model
--
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]