standardSeconds(3) /* timeToDeclareOutputFinal */, standardSeconds(30) /* timeToFail */)) .withPollInterval(Duration.millis(500)) .withOutputCoder(VarIntCoder.of())) .apply(Reify.timestampsInValue())
.withPollInterval(Duration.millis(1)) .withOutputCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))) .apply("Drop input", Values.create());
@Test @Category(NeedsRunner.class) public void testSinglePollMultipleInputs() { PCollection<KV<String, String>> res = p.apply(Create.of("a", "b")) .apply( Watch.growthOf( new PollFn<String, String>() { @Override public PollResult<String> apply(String element, Context c) throws Exception { return PollResult.complete( Instant.now(), Arrays.asList(element + ".foo", element + ".bar")); } }) .withPollInterval(Duration.ZERO)); PAssert.that(res) .containsInAnyOrder( Arrays.asList( KV.of("a", "a.foo"), KV.of("a", "a.bar"), KV.of("b", "b.foo"), KV.of("b", "b.bar"))); p.run(); }
KV::getKey) .withTerminationPerInput(Growth.afterTotalOf(standardSeconds(5))) .withPollInterval(Duration.millis(100)) .withOutputCoder(KvCoder.of(VarIntCoder.of(), StringUtf8Coder.of())) .withOutputKeyCoder(VarIntCoder.of()))
@Test @Category(NeedsRunner.class) public void testSinglePollMultipleInputsWithSideInput() { final PCollectionView<String> moo = p.apply("moo", Create.of("moo")).apply("moo singleton", View.asSingleton()); final PCollectionView<String> zoo = p.apply("zoo", Create.of("zoo")).apply("zoo singleton", View.asSingleton()); PCollection<KV<String, String>> res = p.apply("input", Create.of("a", "b")) .apply( Watch.growthOf( new PollFn<String, String>() { @Override public PollResult<String> apply(String element, Context c) throws Exception { return PollResult.complete( Instant.now(), Arrays.asList( element + " " + c.sideInput(moo) + " " + c.sideInput(zoo))); } }, requiresSideInputs(moo, zoo)) .withPollInterval(Duration.ZERO)); PAssert.that(res) .containsInAnyOrder(Arrays.asList(KV.of("a", "a moo zoo"), KV.of("b", "b moo zoo"))); p.run(); }
private void testMultiplePolls(boolean terminationConditionElapsesBeforeOutputIsFinal) { List<Integer> all = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); PCollection<Integer> res = p.apply(Create.of("a")) .apply( Watch.growthOf( new TimedPollFn<String, Integer>( all, standardSeconds(1) /* timeToOutputEverything */, standardSeconds(3) /* timeToDeclareOutputFinal */, standardSeconds(30) /* timeToFail */)) .withTerminationPerInput( Growth.afterTotalOf( standardSeconds( // At 2 seconds, all output has been yielded, but not yet // declared final - so polling should terminate per termination // condition. // At 3 seconds, all output has been yielded (and declared final), // so polling should terminate because of that without waiting for // 100 seconds. terminationConditionElapsesBeforeOutputIsFinal ? 2 : 100))) .withPollInterval(Duration.millis(300)) .withOutputCoder(VarIntCoder.of())) .apply("Drop input", Values.create()); PAssert.that(res).containsInAnyOrder(all); p.run(); }
@Test @Category(NeedsRunner.class) public void testMultiplePollsStopAfterTimeSinceNewOutput() { List<Integer> all = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); PCollection<Integer> res = p.apply(Create.of("a")) .apply( Watch.growthOf( new TimedPollFn<String, Integer>( all, standardSeconds(1) /* timeToOutputEverything */, // Never declare output final standardSeconds(1000) /* timeToDeclareOutputFinal */, standardSeconds(30) /* timeToFail */)) // Should terminate after 4 seconds - earlier than timeToFail .withTerminationPerInput(afterTimeSinceNewOutput(standardSeconds(3))) .withPollInterval(Duration.millis(300)) .withOutputCoder(VarIntCoder.of())) .apply("Drop input", Values.create()); PAssert.that(res).containsInAnyOrder(all); p.run(); }
@Override public PCollection<MatchResult.Metadata> expand(PCollection<String> input) { PCollection<MatchResult.Metadata> res; if (getConfiguration().getWatchInterval() == null) { res = input.apply( "Match filepatterns", ParDo.of(new MatchFn(getConfiguration().getEmptyMatchTreatment()))); } else { res = input .apply( "Continuously match filepatterns", Watch.growthOf( Contextful.of(new MatchPollFn(), Requirements.empty()), new ExtractFilenameFn()) .withPollInterval(getConfiguration().getWatchInterval()) .withTerminationPerInput(getConfiguration().getWatchTerminationCondition())) .apply(Values.create()); } return res.apply(Reshuffle.viaRandomKey()); }