public DoFn<String, String> newInnerAnonymousDoFnWithTimers() { return new DoFn<String, String>() { @TimerId(TIMER_ID) private final TimerSpec myTimer = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); @OnTimer(TIMER_ID) public void onTimer(BoundedWindow w) {} @ProcessElement public void process(ProcessContext c) {} }; }
public static DoFn<String, String> newStaticAnonymousDoFnWithTimers() { return new DoFn<String, String>() { private BoundedWindow invokedWindow; @ProcessElement public void process(ProcessContext c) {} @TimerId(TIMER_ID) private final TimerSpec myTimer = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); @OnTimer(TIMER_ID) public void onTimer(BoundedWindow window) { assertNull("Should have been invoked just once", invokedWindow); invokedWindow = window; } @SuppressWarnings("unused") public void verify(BoundedWindow window) { assertEquals(window, invokedWindow); } }; }
@Test public void testProcessElementWithOnTimerContextRejected() throws Exception { thrown.expect(IllegalArgumentException.class); // The message should at least mention @ProcessElement and OnTimerContext thrown.expectMessage("@" + DoFn.ProcessElement.class.getSimpleName()); thrown.expectMessage(DoFn.OnTimerContext.class.getSimpleName()); DoFnSignatures.getSignature( new DoFn<KV<String, Integer>, Long>() { @TimerId("foo") private final TimerSpec bizzle = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void foo(ProcessContext context, OnTimerContext bogus) {} @OnTimer("foo") public void onFoo() {} }.getClass()); }
@Test public void testWindowParamOnTimer() throws Exception { final String timerId = "some-timer-id"; DoFnSignature sig = DoFnSignatures.getSignature( new DoFn<String, String>() { @TimerId(timerId) private final TimerSpec myfield1 = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void process(ProcessContext c) {} @OnTimer(timerId) public void onTimer(BoundedWindow w) {} }.getClass()); assertThat(sig.onTimerMethods().get(timerId).extraParameters().size(), equalTo(1)); assertThat( sig.onTimerMethods().get(timerId).extraParameters().get(0), instanceOf(WindowParameter.class)); }
@Test public void testSimpleTimerIdAnonymousDoFn() throws Exception { DoFnSignature sig = DoFnSignatures.getSignature( new DoFn<KV<String, Integer>, Long>() { @TimerId("foo") private final TimerSpec bizzle = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void foo(ProcessContext context) {} @OnTimer("foo") public void onFoo() {} }.getClass()); assertThat(sig.timerDeclarations().size(), equalTo(1)); DoFnSignature.TimerDeclaration decl = sig.timerDeclarations().get("foo"); assertThat(decl.id(), equalTo("foo")); assertThat(decl.field().getName(), equalTo("bizzle")); }
@Test public void testSimpleTimerWithContext() throws Exception { DoFnSignature sig = DoFnSignatures.getSignature( new DoFn<KV<String, Integer>, Long>() { @TimerId("foo") private final TimerSpec bizzle = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void foo(ProcessContext context) {} @OnTimer("foo") public void onFoo(OnTimerContext c) {} }.getClass()); assertThat(sig.timerDeclarations().size(), equalTo(1)); DoFnSignature.TimerDeclaration decl = sig.timerDeclarations().get("foo"); assertThat(decl.id(), equalTo("foo")); assertThat(decl.field().getName(), equalTo("bizzle")); assertThat( sig.onTimerMethods().get("foo").extraParameters().get(0), equalTo((Parameter) Parameter.onTimerContext())); }
@Test public void testAllParamsOnTimer() throws Exception { final String timerId = "some-timer-id"; DoFnSignature sig = DoFnSignatures.getSignature( new DoFn<String, String>() { @TimerId(timerId) private final TimerSpec myfield1 = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void process(ProcessContext c) {} @OnTimer(timerId) public void onTimer( @Timestamp Instant timestamp, TimeDomain timeDomain, BoundedWindow w) {} }.getClass()); assertThat(sig.onTimerMethods().get(timerId).extraParameters().size(), equalTo(3)); assertThat( sig.onTimerMethods().get(timerId).extraParameters().get(0), instanceOf(TimestampParameter.class)); assertThat( sig.onTimerMethods().get(timerId).extraParameters().get(1), instanceOf(TimeDomainParameter.class)); assertThat( sig.onTimerMethods().get(timerId).extraParameters().get(2), instanceOf(WindowParameter.class)); }
@Test public void testTimerIdDuplicate() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Duplicate"); thrown.expectMessage("TimerId"); thrown.expectMessage("my-id"); thrown.expectMessage("myfield1"); thrown.expectMessage("myfield2"); thrown.expectMessage(not(containsString("State"))); // lowercase "state" is in the package name thrown.expectMessage(mentionsTimers()); DoFnSignatures.getSignature( new DoFn<KV<String, Integer>, Long>() { @TimerId("my-id") private final TimerSpec myfield1 = TimerSpecs.timer(TimeDomain.EVENT_TIME); @TimerId("my-id") private final TimerSpec myfield2 = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void foo(ProcessContext context) {} }.getClass()); }
@Test public void testTimerIdNonFinal() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Timer declarations must be final"); thrown.expectMessage("Non-final field"); thrown.expectMessage("myfield"); thrown.expectMessage(not(containsString("State"))); // lowercase "state" is in the package name thrown.expectMessage(mentionsTimers()); DoFnSignatures.getSignature( new DoFn<KV<String, Integer>, Long>() { @TimerId("my-timer-id") private TimerSpec myfield = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); @ProcessElement public void foo(ProcessContext context) {} }.getClass()); }
@Test public void testTimerIdNoCallback() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("No callback registered"); thrown.expectMessage("my-id"); thrown.expectMessage(not(mentionsState())); thrown.expectMessage(mentionsTimers()); DoFnSignatures.getSignature( new DoFn<KV<String, Integer>, Long>() { @TimerId("my-id") private final TimerSpec myfield1 = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void foo(ProcessContext context) {} }.getClass()); }
@Test public void testTimerParameterDuplicate() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("duplicate"); thrown.expectMessage("my-id"); thrown.expectMessage("myProcessElement"); thrown.expectMessage("index 2"); thrown.expectMessage(not(mentionsState())); DoFnSignatures.getSignature( new DoFn<KV<String, Integer>, Long>() { @TimerId("my-id") private final TimerSpec myfield = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); @ProcessElement public void myProcessElement( ProcessContext context, @TimerId("my-id") Timer one, @TimerId("my-id") Timer two) {} @OnTimer("my-id") public void onWhatever() {} }.getClass()); }
@Test public void testOnTimerDeclaredInSubclass() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Callback"); thrown.expectMessage("declared in a different class"); thrown.expectMessage(DoFnWithOnlyCallback.TIMER_ID); thrown.expectMessage(not(mentionsState())); thrown.expectMessage(mentionsTimers()); DoFnSignatures.getSignature( new DoFnWithOnlyCallback() { @TimerId(DoFnWithOnlyCallback.TIMER_ID) private final TimerSpec myfield1 = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement @Override public void foo(ProcessContext context) {} }.getClass()); }
@Test public void testTimerNotKeyed() { final String timerId = "foo"; DoFn<String, Integer> fn = new DoFn<String, Integer>() { @TimerId(timerId) private final TimerSpec timer = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void processElement(ProcessContext c, @TimerId(timerId) Timer timer) {} @OnTimer(timerId) public void onTimer() {} }; thrown.expect(IllegalArgumentException.class); thrown.expectMessage("timer"); thrown.expectMessage("KvCoder"); pipeline.apply(Create.of("hello", "goodbye", "hello again")).apply(ParDo.of(fn)); }
@Test public void testTimerNotDeterministic() { final String timerId = "foo"; // DoubleCoder is not deterministic, so this should crash DoFn<KV<Double, String>, Integer> fn = new DoFn<KV<Double, String>, Integer>() { @TimerId(timerId) private final TimerSpec timer = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void processElement(ProcessContext c, @TimerId(timerId) Timer timer) {} @OnTimer(timerId) public void onTimer() {} }; thrown.expect(IllegalArgumentException.class); thrown.expectMessage("timer"); thrown.expectMessage("deterministic"); pipeline .apply(Create.of(KV.of(1.0, "hello"), KV.of(5.4, "goodbye"), KV.of(7.2, "hello again"))) .apply(ParDo.of(fn)); }
/** * Tests that it is OK to use different window types in the parameter lists to different {@link * DoFn} functions, as long as they are all subtypes of the actual window type of the input. * * <p>Today, the only method other than {@link ProcessElement @ProcessElement} that can accept * extended parameters is {@link OnTimer @OnTimer}, which is rejected before it reaches window * type validation. Rather than delay validation, this test is temporarily disabled. */ @Ignore("ParDo rejects this on account of it using timers") @Test public void testMultipleWindowSubtypesOK() { final String timerId = "gobbledegook"; pipeline .apply(Create.of(1, 2, 3)) .apply(Window.into(FixedWindows.of(Duration.standardSeconds(10)))) .apply( ParDo.of( new DoFn<Integer, Integer>() { @TimerId(timerId) private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void process(ProcessContext c, IntervalWindow w) {} @OnTimer(timerId) public void onTimer(BoundedWindow w) {} })); // If it doesn't crash, we made it! }
private final TimerSpec loopSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
@Test @Category({NeedsRunner.class, UsesTimersInParDo.class, UsesTestStream.class}) public void testSimpleProcessingTimerTimer() throws Exception { final String timerId = "foo"; DoFn<KV<String, Integer>, Integer> fn = new DoFn<KV<String, Integer>, Integer>() { @TimerId(timerId) private final TimerSpec spec = TimerSpecs.timer(TimeDomain.PROCESSING_TIME); @ProcessElement public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Integer> r) { timer.offset(Duration.standardSeconds(1)).setRelative(); r.output(3); } @OnTimer(timerId) public void onTimer(TimeDomain timeDomain, OutputReceiver<Integer> r) { if (timeDomain.equals(TimeDomain.PROCESSING_TIME)) { r.output(42); } } }; TestStream<KV<String, Integer>> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())) .addElements(KV.of("hello", 37)) .advanceProcessingTime(Duration.standardSeconds(2)) .advanceWatermarkToInfinity(); PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn)); PAssert.that(output).containsInAnyOrder(3, 42); pipeline.run(); }
private final TimerSpec spec = TimerSpecs.timer(TimeDomain.PROCESSING_TIME);
private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
@Test @Category({NeedsRunner.class, UsesTimersInParDo.class, UsesTestStream.class}) public void testEventTimeTimerUnbounded() throws Exception { final String timerId = "foo"; DoFn<KV<String, Integer>, Integer> fn = new DoFn<KV<String, Integer>, Integer>() { @TimerId(timerId) private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME); @ProcessElement public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Integer> r) { timer.offset(Duration.standardSeconds(1)).setRelative(); r.output(3); } @OnTimer(timerId) public void onTimer(OutputReceiver<Integer> r) { r.output(42); } }; TestStream<KV<String, Integer>> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())) .advanceWatermarkTo(new Instant(0)) .addElements(KV.of("hello", 37)) .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1))) .advanceWatermarkToInfinity(); PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn)); PAssert.that(output).containsInAnyOrder(3, 42); pipeline.run(); }