Class WithTimestamps<T>
- All Implemented Interfaces:
Serializable
,HasDisplayData
PTransform
for assigning timestamps to all the elements of a PCollection
.
Timestamps are used to assign Windows
to elements within the Window.into(org.apache.beam.sdk.transforms.windowing.WindowFn)
PTransform
. Assigning
timestamps is useful when the input data set comes from a Source
without implicit
timestamps (such as TextIO
).
- See Also:
-
Field Summary
Fields inherited from class org.apache.beam.sdk.transforms.PTransform
annotations, displayData, name, resourceHints
-
Method Summary
Modifier and TypeMethodDescriptionexpand
(PCollection<T> input) Override this method to specify how thisPTransform
should be expanded on the givenInputT
.Deprecated.This method permits a to elements to be emitted behind the watermark.static <T> WithTimestamps
<T> of
(SerializableFunction<T, Instant> fn) For aSerializableFunction
fn
fromT
toInstant
, outputs aPTransform
that takes an inputPCollection<T>
and outputs aPCollection<T>
containing every elementv
in the input where each element is output with a timestamp obtained as the result offn.apply(v)
.withAllowedTimestampSkew
(Duration allowedTimestampSkew) Deprecated.This method permits a to elements to be emitted behind the watermark.Methods inherited from class org.apache.beam.sdk.transforms.PTransform
addAnnotation, compose, compose, getAdditionalInputs, getAnnotations, getDefaultOutputCoder, getDefaultOutputCoder, getDefaultOutputCoder, getKindString, getName, getResourceHints, populateDisplayData, setDisplayData, setResourceHints, toString, validate, validate
-
Method Details
-
of
For aSerializableFunction
fn
fromT
toInstant
, outputs aPTransform
that takes an inputPCollection<T>
and outputs aPCollection<T>
containing every elementv
in the input where each element is output with a timestamp obtained as the result offn.apply(v)
.If the input
PCollection
elements have timestamps, the output timestamp for each element must not be before the input element's timestamp minus the value ofgetAllowedTimestampSkew()
. If an output timestamp is before this time, the transform will throw anIllegalArgumentException
when executed. UsewithAllowedTimestampSkew(Duration)
to update the allowed skew.CAUTION: Use of
withAllowedTimestampSkew(Duration)
permits elements to be emitted behind the watermark. These elements are considered late, and if behind theallowed lateness
of a downstreamPCollection
may be silently dropped. See https://github.com/apache/beam/issues/18065 for details on a replacement.Each output element will be in the same windows as the input element. If a new window based on the new output timestamp is desired, apply a new instance of
Window.into(WindowFn)
.This transform will fail at execution time with a
NullPointerException
if for any input element the result offn.apply(v)
isnull
.Example of use in Java 8:
PCollection<Record> timestampedRecords = records.apply( WithTimestamps.of((Record rec) -> rec.getInstant());
-
withAllowedTimestampSkew
Deprecated.This method permits a to elements to be emitted behind the watermark. These elements are considered late, and if behind theallowed lateness
of a downstreamPCollection
may be silently dropped. See https://github.com/apache/beam/issues/18065 for details on a replacement.Return a new WithTimestamps like this one with updated allowed timestamp skew, which is the maximum duration that timestamps can be shifted backward. Does not modify this object.The default value is
Duration.ZERO
, allowing timestamps to only be shifted into the future. For infinite skew, usenew Duration(Long.MAX_VALUE)
. -
getAllowedTimestampSkew
Deprecated.This method permits a to elements to be emitted behind the watermark. These elements are considered late, and if behind theallowed lateness
of a downstreamPCollection
may be silently dropped. See https://github.com/apache/beam/issues/18065 for details on a replacement.Returns the allowed timestamp skew duration, which is the maximum duration that timestamps can be shifted backwards from the timestamp of the input element.- See Also:
-
expand
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<PCollection<T>,
PCollection<T>>
-