Class ByteKeyRangeTracker
- All Implemented Interfaces:
RestrictionTracker.HasProgress
RestrictionTracker for claiming ByteKeys in a ByteKeyRange in a
monotonically increasing fashion. The range is a semi-open bounded interval [startKey, endKey)
where the limits are both represented by ByteKey.EMPTY.
Note, one can complete a range by claiming the ByteKey.EMPTY once one runs out of keys
to process.
-
Nested Class Summary
Nested classes/interfaces inherited from class org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker
RestrictionTracker.HasProgress, RestrictionTracker.IsBounded, RestrictionTracker.Progress, RestrictionTracker.TruncateResult<RestrictionT> -
Method Summary
Modifier and TypeMethodDescriptionvoidChecks whether the restriction has been fully processed.Returns a restriction accurately describing the full range of work the currentDoFn.ProcessElementcall will do, including already completed work.A representation for the amount of known completed and known remaining work.Return the boundedness of the current restriction.static ByteKeyRangeTrackerof(ByteKeyRange range) toString()booleanAttempts to claim the given key.trySplit(double fractionOfRemainder) Splits current restriction based onfractionOfRemainder.
-
Method Details
-
of
-
currentRestriction
Description copied from class:RestrictionTrackerReturns a restriction accurately describing the full range of work the currentDoFn.ProcessElementcall will do, including already completed work.The current restriction returned by method may be updated dynamically due to due to concurrent invocation of other methods of the
RestrictionTracker, For example,RestrictionTracker.trySplit(double).This method is required to be implemented.
- Specified by:
currentRestrictionin classRestrictionTracker<ByteKeyRange,ByteKey>
-
trySplit
Description copied from class:RestrictionTrackerSplits 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.ProcessElementexecution responsible for performing the work that the primary restriction represents. The residual restriction will be executed in a separateDoFn.ProcessElementinvocation (likely in a different process). The work performed by executing the primary and residual restrictions as separateDoFn.ProcessElementinvocations MUST be equivalent to the work performed as if this split never occurred.The
fractionOfRemaindershould 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.ProcessElementinvocation is responsible for. For example, if aDoFn.ProcessElementwas reading a file with a restriction representing the offset range[100, 200)and has processed up to offset 130 with afractionOfRemainderof0.7, the primary and residual restrictions returned would be[100, 179), [179, 200)(note:currentOffset + fractionOfRemainder * remainingWork = 130 + 0.7 * 70 = 179).fractionOfRemainder = 0means 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.
- Specified by:
trySplitin classRestrictionTracker<ByteKeyRange,ByteKey> - 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
SplitResultif a split was possible, otherwise returnsnull. If thefractionOfRemainder == 0, anullresult MUST imply that the restriction tracker is done and there is no more work left to do.
-
tryClaim
Attempts to claim the given key.Must be larger than the last attempted key. Since this restriction tracker represents a range over a semi-open bounded interval
[start, end), the last key that was attempted may have failed but still have consumed the interval[lastAttemptedKey, end)since this range tracker processes keys in a monotonically increasing order. Note that passing inByteKey.EMPTYclaims all keys to the end of range and can only be claimed once.- Specified by:
tryClaimin classRestrictionTracker<ByteKeyRange,ByteKey> - Returns:
trueif the key was successfully claimed,falseif it is outside the currentByteKeyRangeof this tracker.
-
checkDone
Description copied from class:RestrictionTrackerChecks whether the restriction has been fully processed.Called by the SDK harness after
DoFn.ProcessElementreturns.Must throw an exception with an informative error message, if there is still any unclaimed work remaining in the restriction.
This method is required to be implemented in order to prevent data loss during SDK processing.
- Specified by:
checkDonein classRestrictionTracker<ByteKeyRange,ByteKey> - Throws:
IllegalStateException
-
isBounded
Description copied from class:RestrictionTrackerReturn 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.BOUNDEDafter returningRestrictionTracker.IsBounded.UNBOUNDEDonce the end of a restriction is discovered. It is not valid to returnRestrictionTracker.IsBounded.UNBOUNDEDafter returningRestrictionTracker.IsBounded.BOUNDED.This method is required to be implemented.
- Specified by:
isBoundedin classRestrictionTracker<ByteKeyRange,ByteKey>
-
toString
-
getProgress
Description copied from interface:RestrictionTracker.HasProgressA 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
doublerepresentation. 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 bytesthat have processed and the number of messages or number ofmessage bytesthat 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:
getProgressin interfaceRestrictionTracker.HasProgress
-