-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[Spanner Change Streams] Fix potential data loss issue by ensuring to only claim timestamps that have been fully processed from the restriction tracker. #37326
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from 2 commits
def9bf9
dc9179d
065a7bd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -168,7 +168,6 @@ public class QueryChangeStreamAction { | |
| * @return a {@link ProcessContinuation#stop()} if a record timestamp could not be claimed or if | ||
| * the partition processing has finished | ||
| */ | ||
| @SuppressWarnings("nullness") | ||
| @VisibleForTesting | ||
| public ProcessContinuation run( | ||
| PartitionMetadata partition, | ||
|
|
@@ -179,10 +178,9 @@ public ProcessContinuation run( | |
| final String token = partition.getPartitionToken(); | ||
| final Timestamp startTimestamp = tracker.currentRestriction().getFrom(); | ||
| final Timestamp endTimestamp = partition.getEndTimestamp(); | ||
| final boolean readToEndTimestamp = !endTimestamp.equals(MAX_INCLUSIVE_END_AT); | ||
|
||
| final Timestamp changeStreamQueryEndTimestamp = | ||
|
||
| endTimestamp.equals(MAX_INCLUSIVE_END_AT) | ||
| ? getNextReadChangeStreamEndTimestamp() | ||
| : endTimestamp; | ||
| readToEndTimestamp ? endTimestamp : getNextReadChangeStreamEndTimestamp(); | ||
|
||
|
|
||
| // TODO: Potentially we can avoid this fetch, by enriching the runningAt timestamp when the | ||
| // ReadChangeStreamPartitionDoFn#processElement is called | ||
|
|
@@ -198,6 +196,7 @@ public ProcessContinuation run( | |
| RestrictionInterrupter<Timestamp> interrupter = | ||
| RestrictionInterrupter.withSoftTimeout(RESTRICTION_TRACKER_TIMEOUT); | ||
|
|
||
| boolean stopAfterQuerySucceeds = readToEndTimestamp; | ||
| try (ChangeStreamResultSet resultSet = | ||
| changeStreamDao.changeStreamQuery( | ||
| token, startTimestamp, changeStreamQueryEndTimestamp, partition.getHeartbeatMillis())) { | ||
|
|
@@ -250,6 +249,9 @@ public ProcessContinuation run( | |
| tracker, | ||
| interrupter, | ||
| watermarkEstimator); | ||
| // The PartitionEndRecord indicates that there are no more records expected | ||
| // for this partition. | ||
| stopAfterQuerySucceeds = true; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seems like we don't need it here. Only change stream v2 has PartitionEndRecord, while change stream v1 has ChildPartitionRecord (might be more than one), which also indicate the partition ends.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. See below, I need the variable to determine whether the query made from (start, now+2m) stopped due to reaching now+2m or due to being done. Since if it's just because it reached the artificial end timestamp we need to reschedule. For v1, it sounds like this approach is not sufficient. I believe the change to the artificial end timestamp was made for v2 change streams. Is it possible to see whether v1 or v2 is being used and we can use changeStreamQueryEndTimestamp=endTimestamp for v1?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Looking at https://docs.cloud.google.com/spanner/docs/change-streams/details#query it seems that ChildPartitionRecords always indicate the end of a query too and that all of such records will have the same timestamp. I've updated the logic to set stopAfterQuerySucceeds in that case too.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Following on this comment, if we add stopAfterQuerySucceeds = true for both ChildPartitionRecords and PartitionEndRecord, then we have assumption relying on that spanner always returns both records in the end of partition. Should we not make this assumption and rely on OutOfRange as the indicator of partition end? Thanks! There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. LGTM |
||
| } else if (record instanceof PartitionEventRecord) { | ||
| maybeContinuation = | ||
| partitionEventRecordAction.run( | ||
|
|
@@ -272,27 +274,23 @@ public ProcessContinuation run( | |
| } | ||
| } | ||
| } | ||
| bundleFinalizer.afterBundleCommit( | ||
| Instant.now().plus(BUNDLE_FINALIZER_TIMEOUT), | ||
| updateWatermarkCallback(token, watermarkEstimator)); | ||
|
|
||
| } catch (SpannerException e) { | ||
| /* | ||
| If there is a split when a partition is supposed to be finished, the residual will try | ||
| to perform a change stream query for an out of range interval. We ignore this error | ||
| here, and the residual should be able to claim the end of the timestamp range, finishing | ||
| the partition. | ||
| */ | ||
| if (isTimestampOutOfRange(e)) { | ||
| LOG.info( | ||
| "[{}] query change stream is out of range for {} to {}, finishing stream.", | ||
| token, | ||
| startTimestamp, | ||
| endTimestamp, | ||
| e); | ||
| } else { | ||
| if (!isTimestampOutOfRange(e)) { | ||
| throw e; | ||
| } | ||
| LOG.info( | ||
| "[{}] query change stream is out of range for {} to {}, finishing stream.", | ||
| token, | ||
| startTimestamp, | ||
| endTimestamp, | ||
| e); | ||
| stopAfterQuerySucceeds = true; | ||
| } catch (Exception e) { | ||
| LOG.error( | ||
| "[{}] query change stream had exception processing range {} to {}.", | ||
|
|
@@ -303,13 +301,28 @@ public ProcessContinuation run( | |
| throw e; | ||
| } | ||
|
|
||
| LOG.debug("[{}] change stream completed successfully", token); | ||
| if (tracker.tryClaim(endTimestamp)) { | ||
| LOG.debug("[{}] Finishing partition", token); | ||
| partitionMetadataDao.updateToFinished(token); | ||
| metrics.decActivePartitionReadCounter(); | ||
| LOG.info("[{}] After attempting to finish the partition", token); | ||
| LOG.debug( | ||
| "[{}] change stream completed successfully up to {}", token, changeStreamQueryEndTimestamp); | ||
| Timestamp claimTimestamp = | ||
| stopAfterQuerySucceeds ? endTimestamp : changeStreamQueryEndTimestamp; | ||
| if (!tracker.tryClaim(claimTimestamp)) { | ||
| return ProcessContinuation.stop(); | ||
| } | ||
|
||
| bundleFinalizer.afterBundleCommit( | ||
| Instant.now().plus(BUNDLE_FINALIZER_TIMEOUT), | ||
| updateWatermarkCallback(token, watermarkEstimator)); | ||
|
|
||
| if (!stopAfterQuerySucceeds) { | ||
| LOG.debug("[{}] Rescheduling partition to resume reading", token); | ||
| return ProcessContinuation.resume(); | ||
| } | ||
|
|
||
| LOG.debug("[{}] Finishing partition", token); | ||
| // TODO: This should be performed after the commit succeeds. Since bundle finalizers are not | ||
| // guaranteed to be called, this needs to be performed in a subsequent fused stage. | ||
| partitionMetadataDao.updateToFinished(token); | ||
| metrics.decActivePartitionReadCounter(); | ||
| LOG.info("[{}] After attempting to finish the partition", token); | ||
| return ProcessContinuation.stop(); | ||
| } | ||
|
|
||
|
|
@@ -339,8 +352,8 @@ private boolean isTimestampOutOfRange(SpannerException e) { | |
| } | ||
|
|
||
| // Return (now + 2 mins) as the end timestamp for reading change streams. This is only used if | ||
| // users want to run the connector forever. This approach works because Google Dataflow | ||
| // checkpoints every 5s or 5MB output provided and the change stream query has deadline for 1 min. | ||
| // users want to run the connector forever. If the end timestamp is reached, we will resume | ||
| // processing from that timestamp on a subsequent DoFn execution. | ||
| private Timestamp getNextReadChangeStreamEndTimestamp() { | ||
| final Timestamp current = Timestamp.now(); | ||
| return Timestamp.ofTimeSecondsAndNanos(current.getSeconds() + 2 * 60, current.getNanos()); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We would like to get the right behavior agreed through this comment first.
It may need multiple PR's to fix.
Here endTs from the partition can be
If bounded,
1.1) For change stream V1, we can use the bounded endTs to query.
1.2) For change stream V2, we can not use the bounded endTs to query as validation may fail endTs <= max(startTs, now) + 30m and return exception. So we should use min(endTs, now+2m) to query.
We can unify 1.1) and 1.2) as min(endTs, now+2m) to query.
If unbounded,
2.1) For change stream V1, we can use the unbounded endTs to query.
2.2) For change stream V2 we can not use the unbounded endTs to query as validation may fail endTs <= max(startTs, now) + 30min and return exception. So we should use now+2m instead.
We can unify 2.1) and 2.2) as min(endTs, now+2m) to query as well.
So in summary, we do not need to know v1 or v2, and consistently use min(endTs, now+2m) to query no matter v1 or v2, bounded or unbounded.
Feel free to poke a hole if this comment is not accurate. But I agree that unify to min(endTs, now+2m) can be risky as it changes V1 existing behaviors and may need more testing and more follow up PR's.