/** * 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(); }
/** Specifies the coder for the output key. */ public Growth<InputT, OutputT, KeyT> withOutputKeyCoder(Coder<KeyT> outputKeyCoder) { return toBuilder().setOutputKeyCoder(outputKeyCoder).build(); }
/** * Specifies how long to wait after a call to {@link PollFn} before calling it again (if at all * - according to {@link PollResult} and the {@link TerminationCondition}). */ public Growth<InputT, OutputT, KeyT> withPollInterval(Duration pollInterval) { return toBuilder().setPollInterval(pollInterval).build(); }
/** Specifies a {@link TerminationCondition} that will be independently used for every input. */ public Growth<InputT, OutputT, KeyT> withTerminationPerInput( TerminationCondition<InputT, ?> terminationPerInput) { return toBuilder().setTerminationPerInput(terminationPerInput).build(); }
/** * Specifies a {@link Coder} to use for the outputs. If unspecified, it will be inferred from * the output type of {@link PollFn} whenever possible. * * <p>The coder must be deterministic, because the transform will compare encoded outputs for * deduplication between polling rounds. */ public Growth<InputT, OutputT, KeyT> withOutputCoder(Coder<OutputT> outputCoder) { return toBuilder().setOutputCoder(outputCoder).build(); }
/** Watches the growth of the given poll function. See class documentation for more details. */ public static <InputT, OutputT> Growth<InputT, OutputT, OutputT> growthOf( Growth.PollFn<InputT, OutputT> pollFn, Requirements requirements) { return new AutoValue_Watch_Growth.Builder<InputT, OutputT, OutputT>() .setTerminationPerInput(Growth.never()) .setPollFn(Contextful.of(pollFn, requirements)) // use null as a signal that this is the identity function and output coder can be // reused as key coder .setOutputKeyFn(null) .build(); }