Skip to content

Commit 27f9fe9

Browse files
authored
Merge pull request #33973: Fix timestamp propagation to successful-loads PCollection.
1 parent f6f7d80 commit 27f9fe9

File tree

1 file changed

+5
-2
lines changed
  • sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery

1 file changed

+5
-2
lines changed

sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java

+5-2
Original file line numberDiff line numberDiff line change
@@ -67,6 +67,7 @@
6767
import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
6868
import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
6969
import org.apache.beam.sdk.transforms.windowing.Repeatedly;
70+
import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
7071
import org.apache.beam.sdk.transforms.windowing.Window;
7172
import org.apache.beam.sdk.util.Preconditions;
7273
import org.apache.beam.sdk.values.KV;
@@ -340,7 +341,8 @@ private WriteResult expandTriggered(PCollection<KV<DestinationT, ElementT>> inpu
340341
AfterProcessingTime.pastFirstElementInPane()
341342
.plusDelayOf(triggeringFrequency),
342343
AfterPane.elementCountAtLeast(FILE_TRIGGERING_RECORD_COUNT))))
343-
.discardingFiredPanes());
344+
.discardingFiredPanes()
345+
.withTimestampCombiner(TimestampCombiner.EARLIEST));
344346
results = writeStaticallyShardedFiles(inputInGlobalWindow, tempFilePrefixView);
345347
} else {
346348
// In the case of dynamic sharding, however, we use a default trigger since the transform
@@ -364,7 +366,8 @@ private WriteResult expandTriggered(PCollection<KV<DestinationT, ElementT>> inpu
364366
Repeatedly.forever(
365367
AfterProcessingTime.pastFirstElementInPane()
366368
.plusDelayOf(triggeringFrequency)))
367-
.discardingFiredPanes());
369+
.discardingFiredPanes()
370+
.withTimestampCombiner(TimestampCombiner.EARLIEST));
368371

369372
TupleTag<KV<ShardedKey<DestinationT>, WritePartition.Result>> multiPartitionsTag =
370373
new TupleTag<>("multiPartitionsTag");

0 commit comments

Comments
 (0)