|
22 | 22 | import java.io.Serializable; |
23 | 23 | import java.util.Collections; |
24 | 24 | import java.util.List; |
| 25 | +import java.util.Set; |
25 | 26 | import java.util.concurrent.atomic.AtomicReference; |
| 27 | +import java.util.stream.Collectors; |
| 28 | +import java.util.stream.LongStream; |
26 | 29 | 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; |
27 | 35 | import org.apache.beam.sdk.testing.NeedsRunner; |
28 | 36 | import org.apache.beam.sdk.testing.PAssert; |
29 | 37 | import org.apache.beam.sdk.testing.TestPipeline; |
30 | 38 | import org.apache.beam.sdk.testing.TestStream; |
| 39 | +import org.apache.beam.sdk.testing.UsesTestStream; |
31 | 40 | import org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime; |
32 | 41 | import org.apache.beam.sdk.transforms.windowing.AfterPane; |
33 | 42 | import org.apache.beam.sdk.transforms.windowing.BoundedWindow; |
|
38 | 47 | import org.apache.beam.sdk.transforms.windowing.Window; |
39 | 48 | import org.apache.beam.sdk.transforms.windowing.WindowFn; |
40 | 49 | import org.apache.beam.sdk.values.PCollection; |
| 50 | +import org.apache.beam.sdk.values.PCollectionTuple; |
41 | 51 | 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; |
42 | 55 | 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; |
43 | 57 | 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; |
44 | 59 | import org.checkerframework.checker.nullness.qual.Nullable; |
45 | 60 | import org.joda.time.Duration; |
46 | 61 | import org.joda.time.Instant; |
@@ -210,6 +225,126 @@ public void testWaitWithSomeSignalWindowsEmpty() { |
210 | 225 | FixedWindows.of(Duration.standardSeconds(1))); |
211 | 226 | } |
212 | 227 |
|
| 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 | + |
213 | 348 | /** |
214 | 349 | * Tests the {@link Wait} transform with a given configuration of the main input and the signal |
215 | 350 | * input. Specifically, generates random streams with bounded lateness for main and signal inputs |
|
0 commit comments