je-ik commented on a change in pull request #17112:
URL: https://github.com/apache/beam/pull/17112#discussion_r830875900
##########
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:
Looking at the code in `SimpleDoFnRunner` I think that:
a) `DoFnProcessContext.outputWithTimestamp` results in call to
`checkTimestamp`
(https://github.com/apache/beam/blob/096aeeff833679af3068eea2f00b280a589256e8/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java#L409)
b) the check will throw exception if the output timestamp is before the
current element's timestamp
(https://github.com/apache/beam/blob/096aeeff833679af3068eea2f00b280a589256e8/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java#L444)
I *think* you can fix that by overriding `getAllowedTimestampSkew()`
(returning some large value), and *provided* that - as you write - output
watermark is held between bundles, then this should be correct. But I actually
really doubt it is the case, I don't know how flink runner for instance would
ensure that, bundle elements and watermarks are completely interleaved in this
particular runner, because the runner does not have a "natural" concept of a
bundle.
@robertwb maybe you could give some more insights?
> Are there testing utilities that can synthesize a bundle to test this?
You could probably use `TestStream` to verify the behavior on DirectRunner.
I _believe_, that all elements that out add using `addElements` will end up in
the same bundle.
--
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]