Class GrowableOffsetRangeTracker
- All Implemented Interfaces:
RestrictionTracker.HasProgress
- Direct Known Subclasses:
DetectNewPartitionsTracker
OffsetRangeTracker
for tracking a growable offset range. Long.MAX_VALUE
is
used as the end of the range to indicate infinity.
An offset range is considered growable when the end offset could grow (or change) during execution time (e.g., Kafka topic partition offset, appended file, ...).
The growable range is marked as done by claiming Long.MAX_VALUE
.
-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionstatic interface
Provides the estimated end offset of the range.Nested classes/interfaces inherited from class org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker
RestrictionTracker.HasProgress, RestrictionTracker.IsBounded, RestrictionTracker.Progress, RestrictionTracker.TruncateResult<RestrictionT>
-
Field Summary
Fields inherited from class org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker
lastAttemptedOffset, lastClaimedOffset, range
-
Constructor Summary
ConstructorsConstructorDescriptionGrowableOffsetRangeTracker
(long start, GrowableOffsetRangeTracker.RangeEndEstimator rangeEndEstimator) -
Method Summary
Modifier and TypeMethodDescriptionA representation for the amount of known completed and known remaining work.Return the boundedness of the current restriction.trySplit
(double fractionOfRemainder) Splits current restriction based onfractionOfRemainder
.Methods inherited from class org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker
checkDone, currentRestriction, toString, tryClaim
-
Constructor Details
-
GrowableOffsetRangeTracker
public GrowableOffsetRangeTracker(long start, GrowableOffsetRangeTracker.RangeEndEstimator rangeEndEstimator)
-
-
Method Details
-
trySplit
Description copied from class:RestrictionTracker
Splits current restriction based onfractionOfRemainder
.If splitting the current restriction is possible, the current restriction is split into a primary and residual restriction pair. This invocation updates the
RestrictionTracker.currentRestriction()
to be the primary restriction effectively having the currentDoFn.ProcessElement
execution responsible for performing the work that the primary restriction represents. The residual restriction will be executed in a separateDoFn.ProcessElement
invocation (likely in a different process). The work performed by executing the primary and residual restrictions as separateDoFn.ProcessElement
invocations MUST be equivalent to the work performed as if this split never occurred.The
fractionOfRemainder
should be used in a best effort manner to choose a primary and residual restriction based upon the fraction of the remaining work that the currentDoFn.ProcessElement
invocation is responsible for. For example, if aDoFn.ProcessElement
was reading a file with a restriction representing the offset range[100, 200)
and has processed up to offset 130 with afractionOfRemainder
of0.7
, the primary and residual restrictions returned would be[100, 179), [179, 200)
(note:currentOffset + fractionOfRemainder * remainingWork = 130 + 0.7 * 70 = 179
).fractionOfRemainder = 0
means a checkpoint is required.The API is recommended to be implemented for a batch pipeline to improve parallel processing performance.
The API is recommended to be implemented for batch pipeline given that it is very important for pipeline scaling and end to end pipeline execution.
The API is required to be implemented for a streaming pipeline.
- Overrides:
trySplit
in classOffsetRangeTracker
- Parameters:
fractionOfRemainder
- A hint as to the fraction of work the primary restriction should represent based upon the current known remaining amount of work.- Returns:
- a
SplitResult
if a split was possible, otherwise returnsnull
. If thefractionOfRemainder == 0
, anull
result MUST imply that the restriction tracker is done and there is no more work left to do.
-
getProgress
Description copied from interface:RestrictionTracker.HasProgress
A representation for the amount of known completed and known remaining work.It is up to each restriction tracker to convert between their natural representation of completed and remaining work and the
double
representation. For example:- Block based file source (e.g. Avro): The number of bytes from the beginning of the restriction to the current block and the number of bytes from the current block to the end of the restriction.
- Pull based queue based source (e.g. Pubsub): The local/global size available in number
of messages or number of
message bytes
that have processed and the number of messages or number ofmessage bytes
that are outstanding. - Key range based source (e.g. BigQuery, Bigtable, ...): Scale the start key to be one and end key to be zero and interpolate the position of the next splittable key as a position. If information about the probability density function or cumulative distribution function is available, work completed and work remaining interpolation can be improved. Alternatively, if the number of encoded bytes for the keys and values is known for the key range, the number of completed and remaining bytes can be used.
The work completed and work remaining must be of the same scale whether that be number of messages or number of bytes and should never represent two distinct unit types.
- Specified by:
getProgress
in interfaceRestrictionTracker.HasProgress
- Overrides:
getProgress
in classOffsetRangeTracker
-
isBounded
Description copied from class:RestrictionTracker
Return the boundedness of the current restriction. If the current restriction represents a finite amount of work, it should returnRestrictionTracker.IsBounded.BOUNDED
. Otherwise, it should returnRestrictionTracker.IsBounded.UNBOUNDED
.It is valid to return
RestrictionTracker.IsBounded.BOUNDED
after returningRestrictionTracker.IsBounded.UNBOUNDED
once the end of a restriction is discovered. It is not valid to returnRestrictionTracker.IsBounded.UNBOUNDED
after returningRestrictionTracker.IsBounded.BOUNDED
.This method is required to be implemented.
- Overrides:
isBounded
in classOffsetRangeTracker
-