engineccl: ignore intents beneath start in MVCCIncrementalIterator#32421
engineccl: ignore intents beneath start in MVCCIncrementalIterator#32421craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
|
Note that if we ever want to use TBI elsewhere, like RefreshRange, we'll need to either teach it to use an MVCCIncrementalIterator instead of a TBI directly, or fold the MVCCIncrementalIterator into the normal mvccScanner. Put another way: nothing besides MVCCIncrementalIterator can possibly use a TBI correctly. All uses of a raw TBI should be viewed with extreme suspicion. |
petermattis
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/ccl/storageccl/engineccl/mvcc_test.go, line 190 at r1 (raw file):
t.Run("intents3", assertEqualKVs(e, fn, keyMin, keyMax, tsMin, ts3, kvs(kv1_3Deleted, kv2_2_2))) // Intents beneath the start bound must be ignored (#28358). t.Run("intents4", assertEqualKVs(e, fn, keyMin, keyMax, ts4, tsMax, kvs()))
These tests are not easy to read. Just a comment in passing. Not the fault of this PR.
nvb
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/ccl/storageccl/engineccl/mvcc.go, line 154 at r1 (raw file):
metaTimestamp := hlc.Timestamp(i.meta.Timestamp) if i.meta.Txn != nil { if i.startTime.Less(metaTimestamp) && !i.endTime.Less(metaTimestamp) {
Should this lower-bound be inclusive or exclusive? How about the upper-bound? Either way, let's add tests for these end-cases.
|
I'm running a tpcc-1000 + cdc test with this patch and haven't yet seen the characteristic multi-second ExportRequest stalls that happen without it. I need to let the test run for a while (ideally overnight) to be sure, but so far so good. |
As determined in cockroachdb#28358, using time-bound iterators is rife with pitfalls. Specifically, the keys returned outside of the time bounds might be wildly inconsistent. A iteration over time bounds [ts3, ts4] might observe an intent at time ts2 or ts5 as still pending when in fact it was resolved, just in an SST that was not considered by the iterator. The only guarantee is that the snapshot of keys within the [ts3, ts4] time bounds is consistent. (Currently this isn't quite true, thanks to another bug, but this is the guarantee that time-bound iterators should be providing.) MVCCIncrementalIterator mostly handled these pitfalls correctly. It properly ignored all non-metadata keys outside of the timestamp bounds, as well as metadata keys (i.e., intents) above the upper timestamp bound. It was not, however, ignoring intents beneath the lower timestamp bound. Since these intents might be inconsistent (i.e., they might already be resolved), the iterator must ignore them. The most problematic symptom was that ExportRequests, which use an MVCCIncrementalIterator, could get stuck trying to resolve an already-resolved intent. The ExportRequest would return a WriteIntentError because it would observe the intent and not its resolution, but resolving the intent would be a no-op because the intent was, in fact, already resolved, and so retrying the ExportRequest would run into the same problem. The situation would eventually unstick when a RocksDB compaction rearranged SSTs such that the ExportRequest observed both the intent and its resolution. Note that 1eb3b2a disabled the use of time-bound iterators in all other code paths that would have had a similar problem. Release note: None
9d8c564 to
765df40
Compare
benesch
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained
pkg/ccl/storageccl/engineccl/mvcc.go, line 154 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Should this lower-bound be inclusive or exclusive? How about the upper-bound? Either way, let's add tests for these end-cases.
Incremental iterators are "backwards:" they have an exclusive start time bound and an inclusive end time bound. I enhanced the test cases around this.
nvb
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 1 of 0 LGTMs obtained
|
bors r=petermattis,nvanbenschoten |
32421: engineccl: ignore intents beneath start in MVCCIncrementalIterator r=petermattis,nvanbenschoten a=benesch As determined in #28358, using time-bound iterators is rife with pitfalls. Specifically, the keys returned outside of the time bounds might be wildly inconsistent. A iteration over time bounds [ts3, ts4] might observe an intent at time ts2 or ts5 as still pending when in fact it was resolved, just in an SST that was not considered by the iterator. The only guarantee is that the snapshot of keys within the [ts3, ts4] time bounds is consistent. (Currently this isn't quite true, thanks to another bug, but this is the guarantee that time-bound iterators should be providing.) MVCCIncrementalIterator mostly handled these pitfalls correctly. It properly ignored all non-metadata keys outside of the timestamp bounds, as well as metadata keys (i.e., intents) above the upper timestamp bound. It was not, however, ignoring intents beneath the lower timestamp bound. Since these intents might be inconsistent (i.e., they might already be resolved), the iterator must ignore them. The most problematic symptom was that ExportRequests, which use an MVCCIncrementalIterator, could get stuck trying to resolve an already-resolved intent. The ExportRequest would return a WriteIntentError because it would observe the intent and not its resolution, but resolving the intent would be a no-op because the intent was, in fact, already resolved, and so retrying the ExportRequest would run into the same problem. The situation would eventually unstick when a RocksDB compaction rearranged SSTs such that the ExportRequest observed both the intent and its resolution. Note that 1eb3b2a disabled the use of time-bound iterators in all other code paths that would have had a similar problem. Release note: None Co-authored-by: Nikhil Benesch <nikhil.benesch@gmail.com>
Build succeeded |
…rator" This reverts commit 765df40. Unfortunately, cockroachdb#32421 has turned > make test TESTS=TestChangefeedSchemaChangeNoBackfill PKG=github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl TESTTIMEOUT=30s TESTFLAGS=-v and various other tests into failing propositions (they take forever). This slows down the CDC pipeline and the symptoms affect various other CDC tests so that skipping the test isn't an attractive option. Besides, this was clearly not intended, so there's probably something wrong with the code being reverted here. We should investigate and submit a fix. Closes cockroachdb#32461. Fixes cockroachdb#32433. Fixes cockroachdb#32444. Release note: None
32465: Revert "engineccl: ignore intents beneath start in MVCCIncrementalIterator" r=benesch a=tbg This reverts commit 765df40. Unfortunately, #32421 has turned > make test TESTS=TestChangefeedSchemaChangeNoBackfill PKG=github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl TESTTIMEOUT=30s TESTFLAGS=-v and various other tests into failing propositions (they take forever). This slows down the CDC pipeline and the symptoms affect various other CDC tests so that skipping the test isn't an attractive option. Besides, this was clearly not intended, so there's probably something wrong with the code being reverted here. We should investigate and submit a fix. Closes #32461. Fixes #32433. Fixes #32444. Release note: None Co-authored-by: Tobias Schottdorf <tobias.schottdorf@gmail.com>
As determined in #28358, using time-bound iterators is rife with
pitfalls. Specifically, the keys returned outside of the time bounds
might be wildly inconsistent. A iteration over time bounds [ts3, ts4]
might observe an intent at time ts2 or ts5 as still pending when in fact
it was resolved, just in an SST that was not considered by the iterator.
The only guarantee is that the snapshot of keys within the [ts3, ts4]
time bounds is consistent. (Currently this isn't quite true, thanks to
another bug, but this is the guarantee that time-bound iterators should
be providing.)
MVCCIncrementalIterator mostly handled these pitfalls correctly. It
properly ignored all non-metadata keys outside of the timestamp bounds,
as well as metadata keys (i.e., intents) above the upper timestamp
bound. It was not, however, ignoring intents beneath the lower timestamp
bound. Since these intents might be inconsistent (i.e., they might
already be resolved), the iterator must ignore them.
The most problematic symptom was that ExportRequests, which use an
MVCCIncrementalIterator, could get stuck trying to resolve an
already-resolved intent. The ExportRequest would return a
WriteIntentError because it would observe the intent and not its
resolution, but resolving the intent would be a no-op because the intent
was, in fact, already resolved, and so retrying the ExportRequest would
run into the same problem. The situation would eventually unstick when a
RocksDB compaction rearranged SSTs such that the ExportRequest observed
both the intent and its resolution.
Note that 1eb3b2a disabled the use of time-bound iterators in all
other code paths that would have had a similar problem.
Release note: None