Skip to content

kv: use correct sequence number when scanning for conflicting intents#93175

Merged
craig[bot] merged 2 commits intocockroachdb:masterfrom
arulajmani:kvnemesis-92189
Dec 8, 2022
Merged

kv: use correct sequence number when scanning for conflicting intents#93175
craig[bot] merged 2 commits intocockroachdb:masterfrom
arulajmani:kvnemesis-92189

Conversation

@arulajmani
Copy link
Copy Markdown
Collaborator

A read only request scans the lock table before it can proceed with dropping latches. It can only evaluate if no conflicting intents are found. While doing so, it also determines if the MVCC scan evaluation needs to consult intent history (by using the interleaved iterator).

The MVCC scan evaluation needs to consult intent history if we discover an intent by the transaction performing the read operation at a higher sequence number or a higher timestamp. The correct sequence numbers to compare here are those on the BatchRequest, and not on the transaction. Before this patch, we were using the sequence number on the transaction, which could lead us to wrongly conclude that the use of an intent interleaving iterator wasn't required.

Specifically, if the batch of the following construction was retried on the server:

b.Scan(a, e)
b.Put(b, "value")

The scan would end up (erroneously) reading "value" at key b.

As part of this patch, I've also renamed ScanConflictingIntents to ScanConflictingIntentsForDroppingLatchesEarly -- the function isn't as generalized as the old name would suggest.

Closes #92217
Closes #92189

Release note: None

@arulajmani arulajmani requested a review from nvb December 6, 2022 23:05
@arulajmani arulajmani requested review from a team as code owners December 6, 2022 23:05
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@arulajmani
Copy link
Copy Markdown
Collaborator Author

@nvanbenschoten turns out this only happens when we're doing server side retries -- so the hypothesis in #92189 (comment) about why the transaction's sequence number is 2 doesn't quite hold. I'll run this through the debugger and try and figure out what's going on there.

Copy link
Copy Markdown
Contributor

@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.

:lgtm:

Reviewed 3 of 3 files at r1, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @arulajmani)


pkg/storage/engine.go line 1687 at r1 (raw file):

// `txn` at the supplied `ts` are ignored.
//
// The caller must supply the sequence number of the batch request on behalf of

"the request", not "the batch request", that's what was getting us in trouble.


pkg/storage/engine.go line 1698 at r1 (raw file):

	ctx context.Context,
	reader Reader,
	txn *roachpb.Transaction,

To make this kind of mistake harder to make, should we switch to passing the txnID uuid.UUID? We can compare against uuid.Nil for the non-txn case.

@arulajmani arulajmani force-pushed the kvnemesis-92189 branch 2 times, most recently from c8837be to 6bbcc3a Compare December 8, 2022 04:53
Copy link
Copy Markdown
Collaborator Author

@arulajmani arulajmani left a comment

Choose a reason for hiding this comment

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

Like we spoke about offline, I think I was too quick to dismiss the hypothesis in the comment linked above; it turns, out that was indeed why we were seeing sequence number 2 the second time around, when the batch was retried. I mistakenly convinced myself that this was happening on a server side retry, because that's what I expected to happen given our construction, but I was missing the splitting that happens in the DistSender to decompose the batch into 2. Given this was the issue, I added the lines to reset the Sequence on the transaction before returning it in the batch response, like we spoke about earlier today.

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


pkg/storage/engine.go line 1698 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

To make this kind of mistake harder to make, should we switch to passing the txnID uuid.UUID? We can compare against uuid.Nil for the non-txn case.

Done.

@tbg
Copy link
Copy Markdown
Member

tbg commented Dec 8, 2022

Please make sure to remove this in this PR:

// TODO(arul): remove this line when #92189 is addressed.
//
// See: https://github.com/cockroachdb/cockroach/issues/93164.
sqlutils.MakeSQLRunner(sqlDBs[0]).Exec(t, `SET CLUSTER SETTING kv.transaction.dropping_latches_before_eval.enabled = false`)

A read only request scans the lock table before it can proceed with
dropping latches. It can only evaluate if no conflicting intents are
found. While doing so, it also determines if the MVCC scan evaluation
needs to consult intent history (by using the interleaved iterator).

The MVCC scan evaluation needs to consult intent history if we discover
an intent by the transaction performing the read operation at a higher
sequence number or a higher timestamp. The correct sequence numbers to
compare here are those on the `BatchRequest`, and not on the
transaction. Before this patch, we were using the sequence number on the
transaction, which could lead us to wrongly conclude that the use of an
intent interleaving iterator wasn't required.

Specifically, if the batch of the following construction was retried:

```
b.Scan(a, e)
b.Put(b, "value")
```

The scan would end up (erroneously) reading "value" at key b.

As part of this patch, I've also renamed `ScanConflictingIntents` to
`ScanConflictingIntentsForDroppingLatchesEarly` -- the function isn't
as generalized as the old name would suggest.

Closes cockroachdb#92217
Closes cockroachdb#92189

Release note: None
We'd disabled dropping latches early to stabilize kvnemisis, this patch
undoes that.

Closes cockroachdb#93164

Release note: None
@arulajmani
Copy link
Copy Markdown
Collaborator Author

Added a commit to remove the lines that disabled dropping latches early in kvnemesis.

I also had to revert the reset Sequence on the transaction change -- turns out, we actually rely on this behaviour to ascertain if the transaction has performed any writes or not in the TxnCoordSender.

func (tc *TxnCoordSender) hasPerformedWritesLocked() bool {
return tc.mu.txn.Sequence != 0
}

Will bors on green.

Copy link
Copy Markdown
Contributor

@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.

:lgtm:

Reviewed 3 of 4 files at r2, 1 of 1 files at r3, 1 of 1 files at r4, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @arulajmani)

@nvb
Copy link
Copy Markdown
Contributor

nvb commented Dec 8, 2022

I also had to revert the reset Sequence on the transaction change -- turns out, we actually rely on this behaviour to ascertain if the transaction has performed any writes or not in the TxnCoordSender.

This feels potentially buggy to me. There's no guarantee that the coordinator received a response from a write request. I think we'd want to replace the tc.mu.txn.Sequence != 0 with tc.interceptorAlloc.txnPipeliner.hasAcquiredLocks() like we have in TxnCoordSender.Send.

@arulajmani
Copy link
Copy Markdown
Collaborator Author

This feels potentially buggy to me. There's no guarantee that the coordinator received a response from a write request. I think we'd want to replace the tc.mu.txn.Sequence != 0 with tc.interceptorAlloc.txnPipeliner.hasAcquiredLocks() like we have in TxnCoordSender.Send.

Yeah, this didn't seem great, given how we're relying on command evaluation to have written to this field and then making inferences up here. I'll file an issue about this and address it in a quick patch.

bors r=nvanbenschoten

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Dec 8, 2022

Build failed (retrying...):

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Dec 8, 2022

Build succeeded:

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/kvnemesis: TestKVNemesisMultiNode failed kv/kvnemesis: TestKVNemesisSingleNode failed

4 participants