Class GenerateSequence
- java.lang.Object
-
- org.apache.beam.sdk.transforms.PTransform<PBegin,PCollection<java.lang.Long>>
-
- org.apache.beam.sdk.io.GenerateSequence
-
- All Implemented Interfaces:
java.io.Serializable
,HasDisplayData
public abstract class GenerateSequence extends PTransform<PBegin,PCollection<java.lang.Long>>
APTransform
that produces longs starting from the given value, and either up to the given limit or untilLong.MAX_VALUE
/ until the given time elapses.The bounded
GenerateSequence
is implemented based onOffsetBasedSource
andOffsetBasedSource.OffsetBasedReader
, so it performs efficient initial splitting and it supports dynamic work rebalancing.To produce a bounded
PCollection<Long>
:Pipeline p = ... PCollection<Long> bounded = p.apply(GenerateSequence.from(0).to(1000));
To produce an unbounded
PCollection<Long>
, simply do not specifyto(long)
, callingwithTimestampFn(SerializableFunction)
to provide values with timestamps other thanInstant.now()
.Pipeline p = ... // To use processing time as the element timestamp. PCollection<Long> unbounded = p.apply(GenerateSequence.from(0)); // Or, to use a provided function to set the element timestamp. PCollection<Long> unboundedWithTimestamps = p.apply(GenerateSequence.from(0).withTimestampFn(someFn));
In all cases, the sequence of numbers is generated in parallel, so there is no inherent ordering between the generated values - it is only guaranteed that all values in the given range will be present in the resulting
PCollection
.- See Also:
- Serialized Form
-
-
Nested Class Summary
Nested Classes Modifier and Type Class Description static class
GenerateSequence.External
Exposes GenerateSequence as an external transform for cross-language usage.
-
Field Summary
-
Fields inherited from class org.apache.beam.sdk.transforms.PTransform
name, resourceHints
-
-
Constructor Summary
Constructors Constructor Description GenerateSequence()
-
Method Summary
All Methods Static Methods Instance Methods Concrete Methods Modifier and Type Method Description PCollection<java.lang.Long>
expand(PBegin input)
Override this method to specify how thisPTransform
should be expanded on the givenInputT
.static GenerateSequence
from(long from)
Specifies the minimum number to generate (inclusive).void
populateDisplayData(DisplayData.Builder builder)
Register display data for the given transform or component.GenerateSequence
to(long to)
Specifies the maximum number to generate (exclusive).GenerateSequence
withMaxReadTime(org.joda.time.Duration maxReadTime)
Specifies to stop generating elements after the given time.GenerateSequence
withRate(long numElements, org.joda.time.Duration periodLength)
Specifies to generate at most a given number of elements per a given period.GenerateSequence
withTimestampFn(SerializableFunction<java.lang.Long,org.joda.time.Instant> timestampFn)
Specifies the function to use to assign timestamps to the elements.-
Methods inherited from class org.apache.beam.sdk.transforms.PTransform
compose, compose, getAdditionalInputs, getDefaultOutputCoder, getDefaultOutputCoder, getDefaultOutputCoder, getKindString, getName, getResourceHints, setResourceHints, toString, validate, validate
-
-
-
-
Method Detail
-
from
public static GenerateSequence from(long from)
Specifies the minimum number to generate (inclusive).
-
to
public GenerateSequence to(long to)
Specifies the maximum number to generate (exclusive).
-
withTimestampFn
public GenerateSequence withTimestampFn(SerializableFunction<java.lang.Long,org.joda.time.Instant> timestampFn)
Specifies the function to use to assign timestamps to the elements.
-
withRate
public GenerateSequence withRate(long numElements, org.joda.time.Duration periodLength)
Specifies to generate at most a given number of elements per a given period.
-
withMaxReadTime
public GenerateSequence withMaxReadTime(org.joda.time.Duration maxReadTime)
Specifies to stop generating elements after the given time.
-
expand
public PCollection<java.lang.Long> expand(PBegin input)
Description copied from class:PTransform
Override this method to specify how thisPTransform
should be expanded on the givenInputT
.NOTE: This method should not be called directly. Instead apply the
PTransform
should be applied to theInputT
using theapply
method.Composite transforms, which are defined in terms of other transforms, should return the output of one of the composed transforms. Non-composite transforms, which do not apply any transforms internally, should return a new unbound output and register evaluators (via backend-specific registration methods).
- Specified by:
expand
in classPTransform<PBegin,PCollection<java.lang.Long>>
-
populateDisplayData
public void populateDisplayData(DisplayData.Builder builder)
Description copied from class:PTransform
Register display data for the given transform or component.populateDisplayData(DisplayData.Builder)
is invoked by Pipeline runners to collect display data viaDisplayData.from(HasDisplayData)
. Implementations may callsuper.populateDisplayData(builder)
in order to register display data in the current namespace, but should otherwise usesubcomponent.populateDisplayData(builder)
to use the namespace of the subcomponent.By default, does not register any display data. Implementors may override this method to provide their own display data.
- Specified by:
populateDisplayData
in interfaceHasDisplayData
- Overrides:
populateDisplayData
in classPTransform<PBegin,PCollection<java.lang.Long>>
- Parameters:
builder
- The builder to populate with display data.- See Also:
HasDisplayData
-
-