-
Notifications
You must be signed in to change notification settings - Fork 3.9k
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
rangefeed: fix premature checkpoint due to intent resolution race #114487
rangefeed: fix premature checkpoint due to intent resolution race #114487
Conversation
676750d
to
170cf84
Compare
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.
Reviewed 5 of 5 files at r1, 3 of 3 files at r2, 1 of 1 files at r3, all commit messages.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @erikgrinaker)
pkg/kv/kvserver/replica_rangefeed.go
line 213 at r2 (raw file):
var ba kvpb.BatchRequest ba.Add(&kvpb.LeaseInfoRequest{
The use of LeaseInfoRequest
in this way feels somewhat fragile in multiple ways. I'm curious if you're sympathetic to any of them. I'm not sure if I am.
The first is that it incurs a clock synchronization dependency. LeaseInfoRequest
can be served by a stale leaseholder if that leaseholder has a slow clock. As a result, the LeaseAppliedIndex
returned here may lag behind the maximum lease applied index in the range. The consequence of this would be that a rangefeed would not wait long enough to forget about a transaction and violate the resolved timestamp invariant.
The second is that it assumes that the leaseholder will have applied the intent resolution locally before allowing the intent's transaction's record to be GCed. We have mechanisms which can acknowledge writes ahead of their local application and we rely on latches to ensure linearizability of subsequent reads.
The third is similar. It assumes that the leaseholder always has applied the maximum index which has been acknowledged by the range. We have projects like #88699 which would break this guarantee . That again would probably rely on latching to ensure linearizability of any subsequent reads.
So the use of LeaseInfoRequest
and LeaseAppliedIndex
doesn't feel rock solid because it depends on clock synchronization and internal implementation details of our use of Raft. At the same time, this wouldn't be the first time we're relying on this — we have the same kind of dependency in the range merge protocol, though I think we're at least acquiring the correct latches there to ensure we synchronize with any prior acknowledged-but-not-applied writes.
We could consider a quorum-based linearization method without a clock dependency to be extra certain in these cases. For example, you could imagine a QueryAppliedRaftIndex
method which would get proposed by any replica, committed to the raft log, and "applied". After that point, we'd know what the maximum raft index that any previously acknowledged write could have had. etcd/raft
might actually have an API like this built in.
That's all probably not worth it, but I don't think we'd regret being more deliberate about where exactly we rely on clock synchronization for correctness, and avoiding clock sync dependencies in critical, performance-insensitive operations.
pkg/kv/kvserver/replica_rangefeed.go
line 232 at r2 (raw file):
// older nodes may not set it. Return a 0 LAI in that case. if tp.r.store.ClusterSettings().Version.IsActive(ctx, clusterversion.V24_1) { return 0, errors.AssertionFailedf("leaseholder did not return a lease applied index")
Is this racy? Couldn't the leaseholder have been upgraded between the time that it sent the response and the time that the caller is checking the cluster version?
pkg/kv/kvserver/rangefeed/task.go
line 329 at r2 (raw file):
// See: https://github.com/cockroachdb/cockroach/issues/104309 if sawAborted { leaseholderLAI, err := a.pusher.GetLeaseholderLAI(ctx)
Is this correctly synchronizing with handleLogicalOpLogRaftMuLocked
? Since we're not holding the raftMu, doesn't the LeaseAppliedIndex get incremented before handleLogicalOpLogRaftMuLocked
gets called?
EDIT: never mind, handleLogicalOpLogRaftMuLocked
is performed during raft apply staging, before r.mu.state.LeaseAppliedIndex
is advanced. Mind adding that to the comment?
This returns the current lease applied index of the current leaseholder. This is useful e.g. if followers want to ensure they have applied all pending log entries that have currently been applied on the leaseholder. Epic: none Release note: None
170cf84
to
2e5b55a
Compare
`PushTxn` may return a false `ABORTED` status for a transaction that has in fact been committed, if the transaction record has been removed after resolving all intents. This is possible because the timestamp cache does not retain sufficient information to disambiguate a committed transaction from an aborted one for removed transaction records (see `Replica.CanCreateTxnRecord` and `batcheval.SynthesizeTxnFromMeta`). However, the rangefeed txn pusher trusted this `ABORTED` status, and removed the txn's intents from the rangefeed's resolved timestamp intent queue. This can lead to the following sequence of events: 1. A rangefeed is established on a follower F. 2. A txn writes an intent at time t1, applied on the leaseholder L. 3. L emits a closed timestamp t2. 4. F applies the intent write and closed timestamp t2. 5. The txn commits with write timestamp t1. 6. The txn's intent and record is removed on L, but not yet on F. 7. F pushes the txn on L, receives `ABORTED` and discards the intents. 8. F emits a checkpoint at t2 across the rangefeed. 9. F applies the intent resolution, emits an event at t1 < checkpoint t2. This violates the fundamental checkpoint guarantee. Furthermore, a changefeed will drop events below its frontier, so this write may not be emitted to clients at all. This patch fixes the bug by waiting for the local replica to reach the leaseholder's current lease applied index before removing the intents from tracking, which will ensure that any intents that have been resolved on the leaseholder will also be applied on the local replica first. Epic: none Release note (bug fix): fixed a bug where a rangefeed could emit a checkpoint before emitting all committed writes below it. This could happen if the rangefeed runs on a follower replica that lags significantly behind the leaseholder (beyond the closed timestamp target duration), a transaction has been running for more than 10 seconds before committing, and the transaction is able to quickly resolve all intents and remove its transaction record. This may result in a changefeed omitting the write event entirely, logging an error like "cdc ux violation: detected timestamp ... that is less or equal to the local frontier".
Epic: none Release note: None
2e5b55a
to
4af60c2
Compare
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.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
pkg/kv/kvserver/replica_rangefeed.go
line 213 at r2 (raw file):
The first is that it incurs a clock synchronization dependency.
LeaseInfoRequest
can be served by a stale leaseholder if that leaseholder has a slow clock. As a result, theLeaseAppliedIndex
returned here may lag behind the maximum lease applied index in the range.
This kind of seems ok to me, at least in principle. If clock skew exceeds the max offset then reads may violate linearizability, and emitting a checkpoint before an earlier write is basically a linearizability violation -- just a different manifestation. It's worth documenting though.
If we're ok with this, we should at least have changefeeds emit these events anyway, instead of dropping them (like we currently do).
The second is that it assumes that the leaseholder will have applied the intent resolution locally before allowing the intent's transaction's record to be GCed. We have mechanisms which can acknowledge writes ahead of their local application and we rely on latches to ensure linearizability of subsequent reads.
Does it? Here, we first run a txn push to observe the GCed txn record -- if it's not there, the leaseholder must have applied the txn record GC, and thus also have applied all prior intent resolution (because these were committed earlier in the log). The LAI can't regress across lease changes because the incoming leaseholder must actually apply the lease first, which implies applying everything ahead of it in the log.
It's possible that our txn push was what ended up GCing the txn record, and we received an abort before the resolution and GC is applied on the leaseholder, but in this case it's a true abort rather than a false negative so we're ok to prematurely discard the intents from resolved timestamp tracking.
The third is similar. It assumes that the leaseholder always has applied the maximum index which has been acknowledged by the range.
I don't think it does, since we have to observe a removed txn record on the leaseholder first, as described above.
the use of
LeaseInfoRequest
andLeaseAppliedIndex
doesn't feel rock solid because it depends on clock synchronization and internal implementation details of our use of Raft
I'm somewhat sympathetic to this. There are a lot of subtle interactions here, and running a command through Raft is a pretty simple and effective way to serialize with all prior writes. It does seem a bit heavy-handed though, and also far less backportable -- I suppose there could also be concerns around the write volume in certain pathological cases. Let's discuss in our 1:1 later today.
pkg/kv/kvserver/replica_rangefeed.go
line 232 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Is this racy? Couldn't the leaseholder have been upgraded between the time that it sent the response and the time that the caller is checking the cluster version?
This would require a binary upgrade (including a node restart) and subsequent version cluster bump while the response is in transit. If the TCP traffic is delayed longer than this interval, it will presumably exceed the RPC heartbeat timeout, closing the RPC connection from the client side. This seems edge casey enough that we can ignore it? If we should hit this edge case, we'll simply fail the txn push with a log message, and retry in a second.
pkg/kv/kvserver/rangefeed/task.go
line 329 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Is this correctly synchronizing with
handleLogicalOpLogRaftMuLocked
? Since we're not holding the raftMu, doesn't the LeaseAppliedIndex get incremented beforehandleLogicalOpLogRaftMuLocked
gets called?EDIT: never mind,
handleLogicalOpLogRaftMuLocked
is performed during raft apply staging, beforer.mu.state.LeaseAppliedIndex
is advanced. Mind adding that to the comment?
Good catch, added a comment.
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.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
pkg/kv/kvserver/replica_rangefeed.go
line 213 at r2 (raw file):
Here, we first run a txn push to observe the GCed txn record -- if it's not there, the leaseholder must have applied the txn record GC, and thus also have applied all prior intent resolution (because these were committed earlier in the log).
@nvanbenschoten pointed out that the txn record may be anchored on a different range, so we can't rely on this.
We should probably try to flush the Raft pipeline instead by committing a command, and wait for its local application. We can possibly adapt the existing Barrier RPC request to do this.
If we're ok with this, we should at least have changefeeds emit these events anyway, instead of dropping them (like we currently do).
This could possibly lead to excessive duplicates in cases where the rangefeed restarts -- although we should run the catchup scan from the last checkpoint the changefeed has seen anyway, so maybe not.
Closing in favor of #117612, we can reopen this if that doesn't pan out. |
Naïve proof of concept, needs additional testing and improvements. Does this seem reasonable to you @nvanbenschoten?
Given the severity of this, I'm inclined to backport it despite the protocol change.
cc @miretskiy
kvpb: add
LeaseInfoResponse.LeaseAppliedIndex
This returns the current lease applied index of the current leaseholder. This is useful e.g. if followers want to ensure they have applied all pending log entries that have currently been applied on the leaseholder.
rangefeed: fix premature checkpoint due to intent resolution race
PushTxn
may return a falseABORTED
status for a transaction that has in fact been committed, if the transaction record has been removed after resolving all intents. This is possible because the timestamp cache does not retain sufficient information to disambiguate a committed transaction from an aborted one for removed transaction records (seeReplica.CanCreateTxnRecord
andbatcheval.SynthesizeTxnFromMeta
).However, the rangefeed txn pusher trusted this
ABORTED
status, and removed the txn's intents from the rangefeed's resolved timestamp intent queue. This can lead to the following sequence of events:A rangefeed is established on a follower F.
A txn writes an intent at time t1, applied on the leaseholder L.
L emits a closed timestamp t2.
F applies the intent write and closed timestamp t2.
The txn commits with write timestamp t1.
The txn's intent and record is removed on L, but not yet on F.
F pushes the txn on L, receives
ABORTED
and discards the intents.F emits a checkpoint at t2 across the rangefeed.
F applies the intent resolution, emits an event at t1 < checkpoint t2.
This violates the fundamental checkpoint guarantee. Furthermore, a changefeed will drop events below its frontier, so this write may not be emitted to clients at all.
This patch fixes the bug by waiting for the local replica to reach the leaseholder's current lease applied index before removing the intents from tracking, which will ensure that any intents that have been resolved on the leaseholder will also be applied on the local replica first.
Resolves #104309.
Replaces #111218.
Epic: none
Release note (bug fix): fixed a bug where a rangefeed could emit a checkpoint before emitting all committed writes below it. This could happen if the rangefeed runs on a follower replica that lags significantly behind the leaseholder (beyond the closed timestamp target duration), a transaction has been running for more than 10 seconds before committing, and the transaction is able to quickly resolve all intents and remove its transaction record. This may result in a changefeed omitting the write event entirely, logging an error like "cdc ux violation: detected timestamp ... that is less or equal to the local frontier".
rangefeed: assert intent commits above resolved timestamp