Skip to content

Commit 23ba9fc

Browse files
authored
Merge pull request #33989: Fix never trigger
1 parent eea972f commit 23ba9fc

File tree

7 files changed

+162
-18
lines changed

7 files changed

+162
-18
lines changed

runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import java.util.concurrent.CountDownLatch;
2828
import java.util.concurrent.Executor;
2929
import javax.annotation.Nonnull;
30+
import org.apache.beam.runners.core.LateDataUtils;
3031
import org.apache.beam.sdk.runners.AppliedPTransform;
3132
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
3233
import org.apache.beam.sdk.values.WindowingStrategy;
@@ -154,8 +155,11 @@ public void fireForWatermark(AppliedPTransform<?, ?, ?> step, Instant watermark)
154155
private static class WatermarkCallback {
155156
public static <W extends BoundedWindow> WatermarkCallback onGuaranteedFiring(
156157
BoundedWindow window, WindowingStrategy<?, W> strategy, Runnable callback) {
157-
@SuppressWarnings("unchecked")
158-
Instant firingAfter = strategy.getTrigger().getWatermarkThatGuaranteesFiring((W) window);
158+
Instant firingAfter =
159+
Ordering.natural()
160+
.min(
161+
LateDataUtils.garbageCollectionTime(window, strategy),
162+
strategy.getTrigger().getWatermarkThatGuaranteesFiring((W) window));
159163
return new WatermarkCallback(firingAfter, callback);
160164
}
161165

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import java.util.Map;
2727
import java.util.Set;
2828
import org.apache.beam.runners.core.DoFnRunner;
29+
import org.apache.beam.runners.core.LateDataUtils;
2930
import org.apache.beam.runners.core.StateNamespaces;
3031
import org.apache.beam.runners.core.StateNamespaces.WindowNamespace;
3132
import org.apache.beam.runners.core.StateTag;
@@ -54,6 +55,8 @@
5455
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
5556
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
5657
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
58+
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Ordering;
59+
import org.joda.time.Instant;
5760

5861
/** A class that handles streaming side inputs in a {@link DoFnRunner}. */
5962
@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://github.com/apache/beam/issues/20497)
@@ -312,15 +315,19 @@ private <SideWindowT extends BoundedWindow> Windmill.GlobalDataRequest buildGlob
312315
throw new RuntimeException(e);
313316
}
314317

318+
Instant firingAfter =
319+
Ordering.natural()
320+
.min(
321+
LateDataUtils.garbageCollectionTime(sideInputWindow, sideWindowStrategy),
322+
sideWindowStrategy.getTrigger().getWatermarkThatGuaranteesFiring(sideInputWindow));
323+
315324
return Windmill.GlobalDataRequest.newBuilder()
316325
.setDataId(
317326
Windmill.GlobalDataId.newBuilder()
318327
.setTag(view.getTagInternal().getId())
319328
.setVersion(windowStream.toByteString())
320329
.build())
321-
.setExistenceWatermarkDeadline(
322-
WindmillTimeUtils.harnessToWindmillTimestamp(
323-
sideWindowStrategy.getTrigger().getWatermarkThatGuaranteesFiring(sideInputWindow)))
330+
.setExistenceWatermarkDeadline(WindmillTimeUtils.harnessToWindmillTimestamp(firingAfter))
324331
.build();
325332
}
326333

runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/BoundedTrieImpl.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,7 @@ public void add(Iterable<String> values) {
4242
}
4343

4444
@Override
45-
public void add(String ... values) {
45+
public void add(String... values) {
4646
add(Arrays.asList(values));
4747
}
4848
}

runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricsContainer.java

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -89,8 +89,11 @@ public BoundedTrie getBoundedTrie(MetricName metricName) {
8989

9090
@SuppressWarnings("FutureReturnValueIgnored")
9191
public void flush(boolean async) {
92-
if (counters.isEmpty() && distributions.isEmpty() && gauges.isEmpty() &&
93-
stringSets.isEmpty() && boundedTries.isEmpty()) {
92+
if (counters.isEmpty()
93+
&& distributions.isEmpty()
94+
&& gauges.isEmpty()
95+
&& stringSets.isEmpty()
96+
&& boundedTries.isEmpty()) {
9497
return;
9598
}
9699

@@ -102,7 +105,8 @@ public void flush(boolean async) {
102105
extractUpdates(this.stringSets);
103106
ImmutableList<MetricUpdates.MetricUpdate<BoundedTrieData>> boundedTries =
104107
extractUpdates(this.boundedTries);
105-
MetricUpdates updates = new MetricUpdatesImpl(counters, distributions, gauges, stringSets, boundedTries);
108+
MetricUpdates updates =
109+
new MetricUpdatesImpl(counters, distributions, gauges, stringSets, boundedTries);
106110

107111
if (async) {
108112
accumulator.setAsync(metricsKey, updates);

sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java

Lines changed: 1 addition & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -1309,12 +1309,7 @@ public StackManipulation.Size apply(MethodVisitor mv, Context context) {
13091309
if (returnVarIndex != null) {
13101310
// Drop the return type from the locals
13111311
mv.visitLocalVariable(
1312-
"res",
1313-
returnType.getDescriptor(),
1314-
null,
1315-
wrapStart,
1316-
wrapEnd,
1317-
returnVarIndex);
1312+
"res", returnType.getDescriptor(), null, wrapStart, wrapEnd, returnVarIndex);
13181313
}
13191314

13201315
return size;

sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WaitTest.java

Lines changed: 135 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,12 +22,21 @@
2222
import java.io.Serializable;
2323
import java.util.Collections;
2424
import java.util.List;
25+
import java.util.Set;
2526
import java.util.concurrent.atomic.AtomicReference;
27+
import java.util.stream.Collectors;
28+
import java.util.stream.LongStream;
2629
import org.apache.beam.sdk.coders.VarLongCoder;
30+
import org.apache.beam.sdk.schemas.JavaFieldSchema;
31+
import org.apache.beam.sdk.schemas.NoSuchSchemaException;
32+
import org.apache.beam.sdk.schemas.SchemaCoder;
33+
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
34+
import org.apache.beam.sdk.schemas.annotations.SchemaCreate;
2735
import org.apache.beam.sdk.testing.NeedsRunner;
2836
import org.apache.beam.sdk.testing.PAssert;
2937
import org.apache.beam.sdk.testing.TestPipeline;
3038
import org.apache.beam.sdk.testing.TestStream;
39+
import org.apache.beam.sdk.testing.UsesTestStream;
3140
import org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime;
3241
import org.apache.beam.sdk.transforms.windowing.AfterPane;
3342
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -38,9 +47,15 @@
3847
import org.apache.beam.sdk.transforms.windowing.Window;
3948
import org.apache.beam.sdk.transforms.windowing.WindowFn;
4049
import org.apache.beam.sdk.values.PCollection;
50+
import org.apache.beam.sdk.values.PCollectionTuple;
4151
import org.apache.beam.sdk.values.TimestampedValue;
52+
import org.apache.beam.sdk.values.TupleTag;
53+
import org.apache.beam.sdk.values.TupleTagList;
54+
import org.apache.beam.sdk.values.TypeDescriptors;
4255
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects;
56+
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
4357
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists;
58+
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets;
4459
import org.checkerframework.checker.nullness.qual.Nullable;
4560
import org.joda.time.Duration;
4661
import org.joda.time.Instant;
@@ -210,6 +225,126 @@ public void testWaitWithSomeSignalWindowsEmpty() {
210225
FixedWindows.of(Duration.standardSeconds(1)));
211226
}
212227

228+
private static final Set<Long> PROCESSED_LONGS = Sets.newConcurrentHashSet();
229+
private static final Set<Long> VERIFIED_LONGS = Sets.newConcurrentHashSet();
230+
231+
@DefaultSchema(JavaFieldSchema.class)
232+
static class WindowExpirationValue {
233+
public final @Nullable Instant watermarkAdvance;
234+
public final long value;
235+
236+
@SchemaCreate
237+
public WindowExpirationValue(@Nullable Instant watermarkAdvance, long value) {
238+
this.watermarkAdvance = watermarkAdvance;
239+
this.value = value;
240+
}
241+
}
242+
243+
@Test
244+
@Category({NeedsRunner.class, UsesTestStream.class})
245+
public void testWindowExpiration() throws NoSuchSchemaException {
246+
PROCESSED_LONGS.clear();
247+
VERIFIED_LONGS.clear();
248+
249+
SchemaCoder<WindowExpirationValue> schemaCoder =
250+
p.getSchemaRegistry().getSchemaCoder(WindowExpirationValue.class);
251+
List<Long> allLongs = LongStream.range(0, 200).boxed().collect(Collectors.toList());
252+
TestStream.Builder<WindowExpirationValue> streamBuilder =
253+
TestStream.create(schemaCoder).advanceWatermarkTo(Instant.EPOCH);
254+
for (long i : allLongs) {
255+
if (i > 0 && (i % 2) == 0) {
256+
Instant watermarkValue = Instant.ofEpochMilli(i * 1000);
257+
streamBuilder = streamBuilder.advanceWatermarkTo(watermarkValue);
258+
streamBuilder =
259+
streamBuilder.addElements(
260+
TimestampedValue.of(
261+
new WindowExpirationValue(watermarkValue, -1), Instant.ofEpochSecond(i)));
262+
}
263+
streamBuilder =
264+
streamBuilder.addElements(
265+
TimestampedValue.of(new WindowExpirationValue(null, i), Instant.ofEpochSecond(i)));
266+
}
267+
Instant watermarkValue = Instant.ofEpochMilli(200 * 1000);
268+
streamBuilder = streamBuilder.advanceWatermarkTo(watermarkValue);
269+
streamBuilder =
270+
streamBuilder.addElements(
271+
TimestampedValue.of(
272+
new WindowExpirationValue(watermarkValue, -1), Instant.ofEpochSecond(200)));
273+
274+
PCollection<WindowExpirationValue> longs = p.apply(streamBuilder.advanceWatermarkToInfinity());
275+
276+
TupleTag<Long> signalOutputTag = new TupleTag<>();
277+
TupleTag<Long> verifiedOutputTag = new TupleTag<>();
278+
// Keeps track of values processed.
279+
PCollectionTuple pCollectionTuple =
280+
longs.apply(
281+
ParDo.of(
282+
new DoFn<WindowExpirationValue, Long>() {
283+
@ProcessElement
284+
public void process(
285+
@Element WindowExpirationValue element, MultiOutputReceiver o) {
286+
if (element.watermarkAdvance != null) {
287+
// Since TestStream is synchronous, we can assume that the Wait has
288+
// released the previous
289+
// window. Each window contains two elements, so verify that these two
290+
// elements have been
291+
// verified by the ParDo following the Wait.
292+
long elementUpperBound = element.watermarkAdvance.getMillis() / 1000;
293+
// This means the watermark has advanced. We expect the previous window to
294+
// have been verified.
295+
OutputReceiver<Long> verified = o.get(verifiedOutputTag);
296+
if (VERIFIED_LONGS.contains(elementUpperBound - 1)) {
297+
verified.output(elementUpperBound - 1);
298+
}
299+
if (VERIFIED_LONGS.contains(elementUpperBound - 2)) {
300+
verified.output(elementUpperBound - 2);
301+
}
302+
}
303+
PROCESSED_LONGS.add(element.value);
304+
o.get(signalOutputTag).output(element.value);
305+
}
306+
})
307+
.withOutputTags(signalOutputTag, TupleTagList.of(verifiedOutputTag)));
308+
pCollectionTuple.get(verifiedOutputTag).setCoder(VarLongCoder.of());
309+
310+
FixedWindows fixedWindows = FixedWindows.of(Duration.standardSeconds(2));
311+
PCollection<Long> verifiedInts =
312+
longs
313+
.apply(
314+
"flatmap",
315+
FlatMapElements.into(TypeDescriptors.longs())
316+
.via(
317+
value ->
318+
value.watermarkAdvance == null
319+
? Collections.singletonList(value.value)
320+
: Collections.emptyList()))
321+
.apply("w1", Window.<Long>into(fixedWindows).withAllowedLateness(Duration.ZERO))
322+
.apply(
323+
Wait.on(
324+
pCollectionTuple
325+
.get(signalOutputTag)
326+
.apply(
327+
"w2",
328+
Window.<Long>into(fixedWindows).withAllowedLateness(Duration.ZERO))))
329+
.apply(
330+
"verify",
331+
ParDo.of(
332+
new DoFn<Long, Long>() {
333+
@ProcessElement
334+
public void process(@Element Long element, OutputReceiver<Long> o) {
335+
if (PROCESSED_LONGS.contains(element)) {
336+
VERIFIED_LONGS.add(element);
337+
o.output(element);
338+
}
339+
}
340+
}));
341+
PAssert.that(verifiedInts).containsInAnyOrder(Iterables.toArray(allLongs, Long.class));
342+
343+
PAssert.that(pCollectionTuple.get(verifiedOutputTag))
344+
.containsInAnyOrder(Iterables.toArray(allLongs, Long.class));
345+
p.run();
346+
}
347+
213348
/**
214349
* Tests the {@link Wait} transform with a given configuration of the main input and the signal
215350
* input. Specifically, generates random streams with bounded lateness for main and signal inputs

sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -29,10 +29,9 @@
2929
public class NeverTest {
3030
@Test
3131
public void testFireDeadline() throws Exception {
32+
IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10));
3233
assertEquals(
33-
BoundedWindow.TIMESTAMP_MAX_VALUE,
34-
Never.ever()
35-
.getWatermarkThatGuaranteesFiring(new IntervalWindow(new Instant(0), new Instant(10))));
34+
BoundedWindow.TIMESTAMP_MAX_VALUE, Never.ever().getWatermarkThatGuaranteesFiring(window));
3635
}
3736

3837
@Test

0 commit comments

Comments
 (0)