-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
changefeedccl: time-bound iterator and CDC in 2.1 #32799
Comments
I ran some performance tests for option (1) last week. Summary: I'm not sure that disabling time-bound iterators completely is workable. Changefeeds in 2.1 work by repeatedly polling all the "watched" data for any changes. This is done by picking a new high-water mark and sending ExportRequests with the previous and new high-water as time bounds. This poll happens every Setting the time bounds in ExportRequest triggers the time-bound iterator optimization. Internally, it uses the bounds along with the metadata on the sstables in the RocksDB LSM to avoid even looking at ones that can't possibly relevant. Since changefeeds are constantly looking at only the most recent data, this means they get to skip over most of the larger sstables in the lower levels of the LSM. Running tpcc warehouses=1000 on a 3 node n1-standard-16 cluster, a full poll usually takes ~100ms. Most of the time, it easily finishes within the 1s poll duration. Last week, I ran the above test with TBIs completely disabled. Initially, the max ExportRequest duration for a single range stays around 10-20s (contrast this to 100ms for all ~5k ranges with TBI). The requests are run with some limited amount of concurrency and it all works out to a new poll every 10-15m. Over time, the max duration for a single range continually creeps up. I ran the test on and off (kept hitting roachtest timeouts that I thought I disabled) for 3 days (before my cluster met its premature demise 😢) and the max was up to ~300s. p50/p90 started at 1.6s/4.8s and got to 7.0s/20.4s. One theory for this is at the start of the test, we've just done a restore and all data is in L6 tables. As the test runs, the intermediate LSM levels fill in. If the load and changfeed are stopped and restarted, it doesn't drop back down, so I don't think its resource leakage. Ultimately, the good news is that somehow, the most recent resolved timestamp of the changefeeds seems to stay around 8-15m behind realtime. However, the time these ExportRequests are taking (and the growth over several days) makes it hard to be confidant that we could productionize it. |
For reasons described in cockroachdb#28358, a time-bound iterator will sometimes see an unresolved intent where there is none. A normal iterator doesn't have this problem, so we work around it in MVCCIncrementalIterator by double checking all non-value keys. If the normal iterator has a different value for the key, it's used instead. If the normal iterator doesn't have that key, it's skipped. This fixes both changefeeds and incremental backup. Closes cockroachdb#32104 Closes cockroachdb#32799 Release note (bug fix): `CHANGEFEED`s and incremental `BACKUP`s no longer indefinitely hang under an infrequent condition.
For reasons described in cockroachdb#28358, a time-bound iterator will sometimes see an unresolved intent where there is none. A normal iterator doesn't have this problem, so we work around it in MVCCIncrementalIterator by double checking all non-value keys. If the normal iterator has a different value for the key, it's used instead. If the normal iterator doesn't have that key, it's skipped. This fixes both changefeeds and incremental backup. Closes cockroachdb#32104 Closes cockroachdb#32799 Release note (bug fix): `CHANGEFEED`s and incremental `BACKUP`s no longer indefinitely hang under an infrequent condition.
32909: engineccl/mvcc: work around time-bound iterator bug r=bdarnell,benesch a=danhhz For reasons described in #28358, a time-bound iterator will sometimes see an unresolved intent where there is none. A normal iterator doesn't have this problem, so we work around it in MVCCIncrementalIterator by double checking all non-value keys. If the normal iterator has a different value for the key, it's used instead. If the normal iterator doesn't have that key, it's skipped. This fixes both changefeeds and incremental backup. Closes #32104 Closes #32799 Release note (bug fix): `CHANGEFEED`s and incremental `BACKUP`s no longer indefinitely hang under an infrequent condition. Co-authored-by: Daniel Harrison <[email protected]>
For reasons described in cockroachdb#28358, a time-bound iterator will sometimes see an unresolved intent where there is none. A normal iterator doesn't have this problem, so we work around it in MVCCIncrementalIterator by double checking all non-value keys. If the normal iterator has a different value for the key, it's used instead. If the normal iterator doesn't have that key, it's skipped. This fixes both changefeeds and incremental backup. Closes cockroachdb#32104 Closes cockroachdb#32799 Release note (bug fix): `CHANGEFEED`s and incremental `BACKUP`s no longer indefinitely hang under an infrequent condition.
There is an unresolved bug in time-bound iterator. It manifests as incorrectly seeing an unresolved intent (the bug is that deletion of the intent key is missed). All current ideas for fixing this are too scary to backport to 2.1, but changefeeds use them (via ExportRequest). See much more detail on the bug in #28358. We have a customer commitment for changefeeds in 2.1 so something has to be done here. Some possible options:
The text was updated successfully, but these errors were encountered: