nehsyc commented on a change in pull request #14421:
URL: https://github.com/apache/beam/pull/14421#discussion_r606533742
##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java
##########
@@ -770,73 +740,71 @@ public void processElement(@Element UserT element,
ProcessContext context)
}
}))
.setCoder(KvCoder.of(VarIntCoder.of(), input.getCoder()))
+ .apply("GroupByDestination", GroupByKey.create())
.apply(
- "ShardAndBatch",
- GroupIntoBatches.<Integer,
UserT>ofSize(FILE_TRIGGERING_RECORD_COUNT)
-
.withMaxBufferingDuration(FILE_TRIGGERING_RECORD_BUFFERING_DURATION)
- .withShardedKey())
- .setCoder(
- KvCoder.of(
-
org.apache.beam.sdk.util.ShardedKey.Coder.of(VarIntCoder.of()),
- IterableCoder.of(input.getCoder())));
-
- // Write grouped elements to temp files.
- PCollection<FileResult<DestinationT>> tempFiles =
- shardedInput
- .apply(
- "AddDummyShard",
- MapElements.via(
- new SimpleFunction<
- KV<org.apache.beam.sdk.util.ShardedKey<Integer>,
Iterable<UserT>>,
- KV<ShardedKey<Integer>, Iterable<UserT>>>() {
- @Override
- public KV<ShardedKey<Integer>, Iterable<UserT>> apply(
- KV<org.apache.beam.sdk.util.ShardedKey<Integer>,
Iterable<UserT>>
- input) {
- // Add dummy shard since it is required by
WriteShardsIntoTempFilesFn. It
- // will be dropped after we generate the temp files.
- return KV.of(
- ShardedKey.of(input.getKey().getKey(),
DUMMY_SHARDNUM),
- input.getValue());
+ "ReifyWindowInKey",
+ ParDo.of(
+ new DoFn<
+ KV<Integer, Iterable<UserT>>,
KV<ValueInSingleWindow<Integer>, UserT>>() {
+ @ProcessElement
+ public void processElement(
+ @Element KV<Integer, Iterable<UserT>> element,
+ @Timestamp Instant timestamp,
+ BoundedWindow window,
+ PaneInfo pane,
+ OutputReceiver<KV<ValueInSingleWindow<Integer>,
UserT>> r) {
+ for (UserT value : element.getValue()) {
+ r.output(
+ KV.of(
+ ValueInSingleWindow.of(
+ element.getKey(), timestamp, window,
pane),
+ value));
+ }
}
}))
.setCoder(
KvCoder.of(
- ShardedKeyCoder.of(VarIntCoder.of()),
IterableCoder.of(input.getCoder())))
+ ValueInSingleWindow.Coder.of(VarIntCoder.of(),
windowCoder),
+ input.getCoder()));
+
+ // Write elements to temp files.
+ PCollection<KV<ValueInSingleWindow<DestinationT>,
FileResult<DestinationT>>> tempFiles =
+ groupedWindowedInput
.apply(
- "WriteShardsIntoTempFiles",
- ParDo.of(new
WriteShardsIntoTempFilesFn()).withSideInputs(getSideInputs()))
- .setCoder(fileResultCoder)
+ "RewindowIntoGlobal",
+ Window.<KV<ValueInSingleWindow<Integer>, UserT>>into(new
GlobalWindows())
+ .triggering(DefaultTrigger.of())
+ .discardingFiredPanes())
.apply(
- "DropShardNum",
- ParDo.of(
- new DoFn<FileResult<DestinationT>,
FileResult<DestinationT>>() {
- @ProcessElement
- public void process(ProcessContext c) {
- c.output(c.element().withShard(UNKNOWN_SHARDNUM));
- }
- }));
+ "ShardAndBatch",
+ GroupIntoBatches.<ValueInSingleWindow<Integer>, UserT>ofSize(
+ FILE_TRIGGERING_RECORD_COUNT)
+
.withMaxBufferingDuration(FILE_TRIGGERING_RECORD_BUFFERING_DURATION)
+ .withShardedKey())
+ .apply(
+ "WriteShardsIntoTempFiles",
+ ParDo.of(new WriteWindowedShardsIntoTempFilesFn())
+ .withSideInputs(getSideInputs()))
+ .setCoder(
+ KvCoder.of(
+ ValueInSingleWindow.Coder.of(destinationCoder,
windowCoder),
+ fileResultCoder));
- // Group temp file results by destinations again to gather all the
results in the same window.
+ // Group temp file results by destination again to gather all the files
in the same window.
// This is needed since we don't have shard idx associated with each
temp file so have to rely
// on the indexing within a bundle.
return tempFiles
.apply(
- "KeyedByDestination",
- WithKeys.of(
- new SimpleFunction<FileResult<DestinationT>, DestinationT>()
{
- @Override
- public DestinationT apply(FileResult<DestinationT> input) {
- return input.getDestination();
- }
- }))
- .setCoder(KvCoder.of(destinationCoder, fileResultCoder))
- .apply(GroupByKey.create())
+ Window.<KV<ValueInSingleWindow<DestinationT>,
FileResult<DestinationT>>>into(
Review comment:
I am pretty sure that this is incorrect. But I haven't figured out a way
to collect all the temp files corresponding to the original window. It could be
much simpler if we don't need to worry about collecting all the files in a
window for shard indexing in the file names (say instead replace 0000-of-0005
with a uuid) which doesn't seem to make a lot of sense given that the sharding
is now dynamic and hidden as a runner optimization.
--
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:
[email protected]