Skip to content

kv: don't mix prefix and non-prefix iters when collecting intents#47247

Merged
craig[bot] merged 2 commits intocockroachdb:masterfrom
nvb:nvanbenschoten/collectIntentIter
Apr 9, 2020
Merged

kv: don't mix prefix and non-prefix iters when collecting intents#47247
craig[bot] merged 2 commits intocockroachdb:masterfrom
nvb:nvanbenschoten/collectIntentIter

Conversation

@nvb
Copy link
Copy Markdown
Contributor

@nvb nvb commented Apr 9, 2020

Fixes #47219.

This commit addresses the bug diagnosed and explained in #47219. In that issue, we saw an assertion failure all the way up in the concurrency manager because a READ_UNCOMMITTED scan was hitting a WriteIntentError, which should not be possible. The root cause of this issue was that READ_UNCOMMITTED scans were mixing prefix and non-prefix iterators pulled from a read-only engine between the time that they were collecting intent keys and they were returning to fetch the provisional values for those keys. This mixing of iterators did not guarantee that the two stages of the operation would observe a consistent snapshot of the underlying engine, and because the READ_UNCOMMITTED scans also did not acquire latches, writes were able to slip in and change the intent while the scan wasn't looking. This caused the scan to throw a WriteIntentError for the new intent transaction, which badly confused other parts of the system (rightfully so).

This commit fixes this issue in a few different ways:

  1. it ensures that we always use the same iterator type (prefix or non-prefix) when retrieving the provisional values for a collection of intents retrieved by an earlier scan during READ_UNCOMMITTED operations.
  2. it adds an assertion inside of batcheval.CollectIntentRows that the function never returns a WriteIntentError. This would have caught the bug much more easily, especially back before we had the concurrency manager assertion and this bug could have materialized as stuck range lookups and potentially even deadlocked splits due to the dependency cycle between those two operations.
  3. it documents the limited guarantees that read-only engines provide with respect to consistent engine snapshots across iterator instances.

We'll want to backport this fix as far back as possible. It won't crash earlier releases of Cockroach, but as stated above, it might cause even more disastrous results. REMINDER: when backporting, remember to change the release note.

Release notes (bug fix): a bug that could cause Cockroach processes to crash due to an assertion failure with the text "expected latches held, found none" has been fixed.

Release justification: fixes a high-priority bug in existing functionality. The bug became louder (now crashes servers) due to recent changes that added new assertions into the code.

@nvb nvb requested a review from petermattis April 9, 2020 03:05
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

Copy link
Copy Markdown
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The plumbing of usePrefixIter :lgtm:. I'm not familiar with CollectIntentRows and its usage, though, so it might be worthwhile for someone else to give this change a look as well.

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @nvanbenschoten and @petermattis)


pkg/kv/kvserver/batcheval/intent.go, line 41 at r1 (raw file):

// TODO(nvanbenschoten): remove the usePrefixIter complexity when we're fully on
// Pebble and can guarantee that all iterators created from a read-only engine
// are consistent.

Note that there is additional work needed in Pebble to make this work. Specifically, we need to add an Iterator.Clone() method. Doing so should be straightforward. I mention this only because right now using Pebble would encounter this same problem.

Copy link
Copy Markdown
Contributor Author

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TFTR! There's no one who is particularly familiar with CollectIntentRows because it hasn't been touched in years.

@irfansharif do you mind giving this a pass as well? There are only three callers of CollectIntentRows and only two uses of roachpb.READ_UNCOMMITTED.

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @petermattis)


pkg/kv/kvserver/batcheval/intent.go, line 41 at r1 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Note that there is additional work needed in Pebble to make this work. Specifically, we need to add an Iterator.Clone() method. Doing so should be straightforward. I mention this only because right now using Pebble would encounter this same problem.

Yeah, I mentioned that in the TODO in engine.go. It seemed too specific for this TODO though.

@nvb
Copy link
Copy Markdown
Contributor Author

nvb commented Apr 9, 2020

Actually, @irfansharif is on the L2 rotation this week, so he's got a lot of interrupts on his plate already. @ajwerner said he'd take a quick look.

@irfansharif
Copy link
Copy Markdown
Contributor

It was L1, and I just rotated off. Taking a look anyway, curious to learn what you learnt here.

Copy link
Copy Markdown
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 6 of 7 files at r1.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @nvanbenschoten and @petermattis)


pkg/kv/kvserver/batcheval/intent.go, line 80 at r1 (raw file):

			return roachpb.KeyValue{}, err
		} else if val == nil {
			return roachpb.KeyValue{}, errors.Errorf("provisional value for intent %+v not found", intent)

This seems like a change in behavior. Before it seems like we used to just skip the intent. Is it the case that actually val is never nil and instead val.Value is empty on deletes and you're fixing an old bug?

Should we add testing for this case? When can it happen?

Copy link
Copy Markdown
Contributor

@irfansharif irfansharif left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

Reviewed 7 of 7 files at r1.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @ajwerner, @nvanbenschoten, and @petermattis)


pkg/kv/kvserver/batcheval/intent.go, line 79 at r1 (raw file):

		if err != nil {
			return roachpb.KeyValue{}, err
		} else if val == nil {

nit: was clearer without the else. Could just be an isolated if.


pkg/kv/kvserver/batcheval/intent.go, line 80 at r1 (raw file):

Previously, ajwerner wrote…

This seems like a change in behavior. Before it seems like we used to just skip the intent. Is it the case that actually val is never nil and instead val.Value is empty on deletes and you're fixing an old bug?

Should we add testing for this case? When can it happen?

Ditto.


pkg/kv/kvserver/batcheval/intent.go, line 84 at r1 (raw file):

		return roachpb.KeyValue{Key: intent.Key, Value: *val}, nil
	}
	res, err := storage.MVCCScanAsTxn(

It seems that MVCCScanAsTxn uses the non-prefix iterator. For my own curiosity, where is the code for that? (what I really want to know is the when one is chosen over the other)


pkg/storage/mvcc.go, line 2547 at r1 (raw file):

// should rarely be used.
//
// The read is carried out without the chance of uncertainty restarts.

Here and MVCCGetAsTxn, is it worth asserting on this?

@nvb nvb force-pushed the nvanbenschoten/collectIntentIter branch from a2b9529 to 7ebf50f Compare April 9, 2020 19:07
Copy link
Copy Markdown
Contributor Author

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It was L1, and I just rotated off.

Ah, sorry, I thought you were ahead of me in the KV L2 rotation, so that's where my mind jumped to.

Thanks for the two reviews!

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @ajwerner, @irfansharif, and @petermattis)


pkg/kv/kvserver/batcheval/intent.go, line 79 at r1 (raw file):

Previously, irfansharif (irfan sharif) wrote…

nit: was clearer without the else. Could just be an isolated if.

Done here and below.


pkg/kv/kvserver/batcheval/intent.go, line 80 at r1 (raw file):

Previously, irfansharif (irfan sharif) wrote…

Ditto.

This is a good catch! I was thinking that the previous code was unnecessary because we would never even see a deletion intent in the first place, but that's incorrect. We'll see the intent when scanning non-transactionally, but then we'll get a nil roachpb.Value from MVCCGet because we're not scanning for tombstones. I've revived the earlier logic and added tests for all of these cases.


pkg/kv/kvserver/batcheval/intent.go, line 84 at r1 (raw file):

Previously, irfansharif (irfan sharif) wrote…

It seems that MVCCScanAsTxn uses the non-prefix iterator. For my own curiosity, where is the code for that? (what I really want to know is the when one is chosen over the other)

That's right at the top of MVCCScan and MVCCGet. Notice how theIterOptions they provide to reader.NewIterator differ.


pkg/storage/mvcc.go, line 2547 at r1 (raw file):

Previously, irfansharif (irfan sharif) wrote…

Here and MVCCGetAsTxn, is it worth asserting on this?

Done at the same place I added the assertion against WriteIntentErrors.

Copy link
Copy Markdown
Contributor

@ajwerner ajwerner left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm:

Reviewed 2 of 2 files at r2.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @irfansharif and @petermattis)


pkg/kv/kvserver/batcheval/intent_test.go, line 113 at r2 (raw file):

	for _, c := range testCases {
		t.Run(c.name, func(t *testing.T) {
			// Test with and without deletion intents. If a READ_UNCOMMITTED request

These semantics seem potentially surprising and probably deserve to be documented somewhere if they aren't already. Is it correct that if you run a read in READ_UNCOMMITTED and there's an uncommitted deletion we'll return the current value?

This is definitely not for this PR though. Always interesting to learn about the edge cases in the bespoke, rarely used modes like READ_UNCOMMITTED

@blathers-crl
Copy link
Copy Markdown

blathers-crl bot commented Apr 9, 2020

❌ The GitHub CI (Cockroach) build has failed on 7ebf50ff.

🦉 Hoot! I am a Blathers, a bot for CockroachDB. I am experimental - my owner is otan.

Fixes cockroachdb#47219.

This commit addresses the bug diagnosed and explained in cockroachdb#47219. In that
issue, we saw an assertion failure all the way up in the concurrency
manager because a READ_UNCOMMITTED scan was hitting a WriteIntentError,
which should not be possible. The root cause of this issue was that
READ_UNCOMMITTED scans were mixing prefix and non-prefix iterators
pulled from a read-only engine between the time that they were
collecting intent keys and they were returning to fetch the provisional
values for those keys. This mixing of iterators did not guarantee that
the two stages of the operation would observe a consistent snapshot of
the underlying engine, and because the READ_UNCOMMITTED scans also did
not acquire latches, writes were able to slip in and change the intent
while the scan wasn't looking. This caused the scan to throw a
WriteIntentError for the new intent transaction, which badly confused
other parts of the system (rightfully so).

This commit fixes this issue in a few different ways:
1. it ensures that we always use the same iterator type (prefix or non-prefix)
   when retrieving the provisional values for a collection of intents retrieved
   by an earlier scan during READ_UNCOMMITTED operations.
2. it adds an assertion inside of batcheval.CollectIntentRows that the
   function never returns a WriteIntentError. This would have caught the bug
   much more easily, especially back before we had the concurrency manager
   assertion and this bug could have materialized as stuck range lookups and
   potentially even deadlocked splits due to the dependency cycle between
   those two operations.
3. it documents the limited guarantees that read-only engines provide with
   respect to consistent engine snapshots across iterator instances.

We'll want to backport this fix as far back as possible. It won't crash
earlier releases of Cockroach, but as stated above, it might cause even
more disastrous results. REMINDER: when backporting, remember to change
the release note.

Release notes (bug fix): a bug that could cause Cockroach processes to
crash due to an assertion failure with the text "expected latches held,
found none" has been fixed.

Release justification: fixes a high-priority bug in existing
functionality. The bug became louder (now crashes servers) due to recent
changes that added new assertions into the code.
@nvb nvb force-pushed the nvanbenschoten/collectIntentIter branch from 7ebf50f to b9e925b Compare April 9, 2020 20:03
Updates documentation on:
- GetResponse.IntentValue
- ScanResponse.IntentRows
- ReverseScanResponse.IntentRows

Release justification: comment-only.
@nvb nvb force-pushed the nvanbenschoten/collectIntentIter branch from 0a9d871 to 81c587f Compare April 9, 2020 20:23
Copy link
Copy Markdown
Contributor Author

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

bors r+

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 2 stale) (waiting on @ajwerner and @petermattis)


pkg/kv/kvserver/batcheval/intent_test.go, line 113 at r2 (raw file):

These semantics seem potentially surprising and probably deserve to be documented somewhere if they aren't already. Is it correct that if you run a read in READ_UNCOMMITTED and there's an uncommitted deletion we'll return the current value?

Yes, that is true. So we probably should be returning deletion tombstones in the IntentRows slice to indicate that the committed value is being deleted. I added a comment to the documentation on these fields.

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Apr 9, 2020

Build succeeded

@craig craig bot merged commit 5f1536f into cockroachdb:master Apr 9, 2020
@nvb nvb deleted the nvanbenschoten/collectIntentIter branch April 10, 2020 16:12
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

kv: READ_UNCOMMITTED scan observes WriteIntentError, hits assertion

5 participants