.continuously( Duration.millis(100), Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3)))); PCollection<MatchResult.Metadata> matchAllMetadata = p.apply(Create.of(basePath.resolve("*").toString())) .continuously( Duration.millis(100), Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3)))); assertEquals(PCollection.IsBounded.UNBOUNDED, matchMetadata.isBounded()); assertEquals(PCollection.IsBounded.UNBOUNDED, matchAllMetadata.isBounded());
.withTerminationPerInput(Growth.afterTotalOf(standardSeconds(1))) .withPollInterval(Duration.millis(1)) .withOutputCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))) .apply("Drop input", Values.create());
@Override public PCollection<KV<InputT, OutputT>> expand(PCollection<InputT> input) { checkNotNull(getPollInterval(), "pollInterval"); checkNotNull(getTerminationPerInput(), "terminationPerInput"); Coder<OutputT> outputCoder = getOutputCoder(); if (outputCoder == null) { getPollFn().getClosure(), PollFn.class, new TypeVariableExtractor<PollFn<InputT, OutputT>, OutputT>() {}); Coder<KeyT> outputKeyCoder = getOutputKeyCoder(); SerializableFunction<OutputT, KeyT> outputKeyFn = getOutputKeyFn(); if (getOutputKeyFn() == null) { TypeDescriptor<KeyT> keyT = TypeDescriptors.outputOf(getOutputKeyFn()); try { outputKeyCoder = input.getPipeline().getCoderRegistry().getCoder(keyT); .apply( ParDo.of(new WatchGrowthFn<>(this, outputCoder, outputKeyFn, outputKeyCoder)) .withSideInputs(getPollFn().getRequirements().getSideInputs())) .setCoder(KvCoder.of(input.getCoder(), outputCoder));
TextIO.read() .from(options.getInputFilePattern()) .watchForNewFiles(DEFAULT_POLL_INTERVAL, Growth.never()))
.watchForNewFiles( Duration.millis(100), Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))) .containsInAnyOrder(firstValues); PAssert.that( .watchForNewFiles( Duration.millis(100), Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))))) .containsInAnyOrder(firstValues); .watchForNewFiles( Duration.millis(100), Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))) .withDesiredBundleSizeBytes(10))) .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); .watchForNewFiles( Duration.millis(100), Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3))) .withDesiredBundleSizeBytes(10))) .containsInAnyOrder(Iterables.concat(firstValues, secondValues));
standardSeconds(3) /* timeToDeclareOutputFinal */, standardSeconds(30) /* timeToFail */)) .withPollInterval(Duration.millis(500)) .withOutputCoder(VarIntCoder.of())) .apply(Reify.timestampsInValue()) .apply("Drop timestamped input", Values.create());
Requirements.empty()), KV::getKey) .withTerminationPerInput(Growth.afterTotalOf(standardSeconds(5))) .withPollInterval(Duration.millis(100)) .withOutputCoder(KvCoder.of(VarIntCoder.of(), StringUtf8Coder.of())) .withOutputKeyCoder(VarIntCoder.of())) .apply("Drop input", Values.create()) .apply("Drop auxiliary string", Keys.create());
@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(); }
@Test @Category(NeedsRunner.class) public void testReadWatchForNewFiles() throws IOException, InterruptedException { final Path basePath = tempFolder.getRoot().toPath().resolve("readWatch"); basePath.toFile().mkdir(); p.apply(GenerateSequence.from(0).to(10).withRate(1, Duration.millis(100))) .apply( Window.<Long>into(FixedWindows.of(Duration.millis(150))) .withAllowedLateness(Duration.ZERO) .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1))) .discardingFiredPanes()) .apply(ToString.elements()) .apply( TextIO.write() .to(basePath.resolve("data").toString()) .withNumShards(1) .withWindowedWrites()); PCollection<String> lines = p.apply( TextIO.read() .from(basePath.resolve("*").toString()) .watchForNewFiles( Duration.millis(100), Watch.Growth.afterTimeSinceNewOutput(Duration.standardSeconds(3)))); PAssert.that(lines).containsInAnyOrder("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"); p.run(); } }
/** * Executes the pipeline with the provided execution * parameters. * * @param options The execution parameters. */ public static PipelineResult run(Options options) { // Create the pipeline. Pipeline pipeline = Pipeline.create(options); /* * Steps: * 1) Read from the text source. * 2) Write each text record to Pub/Sub */ pipeline .apply( "Read Text Data", TextIO.read() .from(options.getInputFilePattern()) .watchForNewFiles(DEFAULT_POLL_INTERVAL, Watch.Growth.never())) .apply("Write to PubSub", PubsubIO.writeStrings().to(options.getOutputTopic())); return pipeline.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(); }
@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(); }
@Test public void testTerminationConditionsAfterTimeSinceNewOutput() { Instant now = Instant.now(); Watch.Growth.AfterTimeSinceNewOutput<Object> c = afterTimeSinceNewOutput(standardSeconds(5)); KV<Instant, ReadableDuration> state = c.forNewInput(now, null); assertFalse(c.canStopPolling(now, state)); assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); assertFalse(c.canStopPolling(now.plus(standardSeconds(6)), state)); state = c.onSeenNewOutput(now.plus(standardSeconds(3)), state); assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); assertFalse(c.canStopPolling(now.plus(standardSeconds(6)), state)); assertTrue(c.canStopPolling(now.plus(standardSeconds(9)), state)); state = c.onSeenNewOutput(now.plus(standardSeconds(5)), state); assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); assertFalse(c.canStopPolling(now.plus(standardSeconds(6)), state)); assertFalse(c.canStopPolling(now.plus(standardSeconds(9)), state)); assertTrue(c.canStopPolling(now.plus(standardSeconds(11)), state)); }
/** * Watches the growth of the given poll function, using the given "key function" to deduplicate * outputs. For example, if OutputT is a filename + file size, this can be a function that returns * just the filename, so that if the same file is observed multiple times with different sizes, * only the first observation is emitted. * * <p>By default, this is the identity function, i.e. the output is used as its own key. */ public static <InputT, OutputT, KeyT> Growth<InputT, OutputT, KeyT> growthOf( Contextful<Growth.PollFn<InputT, OutputT>> pollFn, SerializableFunction<OutputT, KeyT> outputKeyFn) { checkArgument(pollFn != null, "pollFn can not be null"); checkArgument(outputKeyFn != null, "outputKeyFn can not be null"); return new AutoValue_Watch_Growth.Builder<InputT, OutputT, KeyT>() .setTerminationPerInput(Watch.Growth.never()) .setPollFn(pollFn) .setOutputKeyFn(outputKeyFn) .build(); }
@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()); }
private static GrowthTracker<String, String, Integer> newTracker() { return newTracker(new GrowthState<>(never().forNewInput(Instant.now(), null))); }
@Test public void testTerminationConditionsAllOf() { Instant now = Instant.now(); Watch.Growth.AfterTotalOf<Object> a = Growth.afterTotalOf(standardSeconds(5)); Watch.Growth.AfterTotalOf<Object> b = Growth.afterTotalOf(standardSeconds(10)); Watch.Growth.BinaryCombined< Object, KV<Instant, ReadableDuration>, KV<Instant, ReadableDuration>> c = allOf(a, b); KV<KV<Instant, ReadableDuration>, KV<Instant, ReadableDuration>> state = c.forNewInput(now, null); assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); assertFalse(c.canStopPolling(now.plus(standardSeconds(7)), state)); assertTrue(c.canStopPolling(now.plus(standardSeconds(12)), state)); }
@Test public void testTerminationConditionsEitherOf() { Instant now = Instant.now(); Watch.Growth.AfterTotalOf<Object> a = Growth.afterTotalOf(standardSeconds(5)); Watch.Growth.AfterTotalOf<Object> b = Growth.afterTotalOf(standardSeconds(10)); Watch.Growth.BinaryCombined< Object, KV<Instant, ReadableDuration>, KV<Instant, ReadableDuration>> c = eitherOf(a, b); KV<KV<Instant, ReadableDuration>, KV<Instant, ReadableDuration>> state = c.forNewInput(now, null); assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); assertTrue(c.canStopPolling(now.plus(standardSeconds(7)), state)); assertTrue(c.canStopPolling(now.plus(standardSeconds(12)), state)); }
/** * Returns a {@link TerminationCondition} that holds after the given time has elapsed after the * current input was seen. */ public static <InputT> AfterTotalOf<InputT> afterTotalOf(ReadableDuration timeSinceInput) { return afterTotalOf(SerializableFunctions.<InputT, ReadableDuration>constant(timeSinceInput)); }