Skip to content

concurrency: emit structured contention information to trace#58444

Merged
craig[bot] merged 2 commits intocockroachdb:masterfrom
tbg:kv-contention-info
Jan 21, 2021
Merged

concurrency: emit structured contention information to trace#58444
craig[bot] merged 2 commits intocockroachdb:masterfrom
tbg:kv-contention-info

Conversation

@tbg
Copy link
Copy Markdown
Member

@tbg tbg commented Jan 5, 2021

This change attaches a protobuf payload to the current Span whenever a
request conflicts with another transaction. The payload contains the
contending txn (i.e. the pushee) at the time at which it was first
encountered, the key on which the conflict occurred (note that this is
not necessarily the key at which the pushee is anchored) and the time
spent waiting on the conflict (excluding intent resolution).

This enables #57114.

Touches #55583. I am not closing that issue yet because we also want
to potentially track the outcome of the conflict.

Release note: None

@tbg tbg requested a review from nvb January 5, 2021 10:57
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@tbg tbg added the A-tracing Relating to tracing in CockroachDB. label Jan 5, 2021
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.

is this basic approach sane or should contention metadata be generated elsewhere (i.e. not in WaitOn)?

Yes, WaitOn seems like the right place for this. The lockTableWaiter is where all of the waiting on locks is coordinated.

In particular, it's unclear to me whether state.txn and/or state.key can change within a single invocation of WaitOn.

Both of these fields can change while iterating in WaitOn. The idea there is that we grab a snapshot of the lockTable, iterate over it, and wait on each lock that's in our way. WaitOn continues to listen to these updates on its newStateC, clearing the way if necessary by pushing, until it eventually reaches a doneWaiting state.

Completing the current approach would likely result in spaghetti code that is easy to get wrong. We seem to want to keep a bit of state around; more than we do with the current structure which intentionally minimizes state held across iterations of the wait loop.

Yeah, I think what you have here is a little different than how I would do it. I was imagining that you'd begin a new span when you receive on newStateC (with kind waitFor/waitForDistinguished) and then close that span when you receive the next update on newStateC (with any kind). So each span's lifetime is the time between updates on newStateC.

Reviewed 2 of 13 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten and @tbg)


pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 165 at r1 (raw file):

			switch state.kind {
			case waitFor, waitForDistinguished:
				// NB: TxnLockTableWaiterWithNonTxn has state.{key,txn}==nil, despite

Yeah, we should fix the test instead of adding this conditional. It really doesn't make sense for either of these fields to be nil.

Copy link
Copy Markdown
Member Author

@tbg tbg left a comment

Choose a reason for hiding this comment

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

Both of these fields can change while iterating in WaitOn. The idea there is that we grab a snapshot of the lockTable, iterate over it, and wait on each lock that's in our way. WaitOn continues to listen to these updates on its newStateC, clearing the way if necessary by pushing, until it eventually reaches a doneWaiting state.

Right, but can something like this happen:

  • waitFor txn1 on key1, decide to do something (maybe we do a liveness push and txn is still live)
  • go back to waiting
  • waitFor gives us txn2 on key 2, decide to do something, go back to waiting
  • now txn1 on key1 comes back
  • ...

I want to avoid emitting a payload for each time we see a given txn; that could be an unbounded amount, no? Plus these individual updates don't mean much to the caller - the aggregate does (maybe that's wrong; not sure). Do I need to maintain a map in this method?

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

Copy link
Copy Markdown
Member Author

@tbg tbg left a comment

Choose a reason for hiding this comment

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

We chatted directly, updated the PR. Still some cleanup remaining but I think a review pass is reasonable. Note that when I rebased and picked up recent PRs in this area, a got a few diffs:

diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents
index 4fdef227d6..8f3468882d 100644
--- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents
+++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents
@@ -116,15 +116,6 @@ on-txn-updated txn=txn2 status=aborted
 [3] sequence req1: scanning lock table for conflicting locks
 [3] sequence req1: sequencing complete, returned guard
 [3] sequence req1: contention metadata: 00000002 @ "a"
-[3] sequence req1: contention metadata: 00000002 @ "b"
-[3] sequence req1: contention metadata: 00000002 @ "c"
-[3] sequence req1: contention metadata: 00000002 @ "d"
-[3] sequence req1: contention metadata: 00000002 @ "e"
-[3] sequence req1: contention metadata: 00000002 @ "f"
-[3] sequence req1: contention metadata: 00000002 @ "g"
-[3] sequence req1: contention metadata: 00000002 @ "h"
-[3] sequence req1: contention metadata: 00000002 @ "i"
-[3] sequence req1: contention metadata: 00000002 @ "j"

I assume they are a consequence of moving the txn status cache query and intent resolution above WaitOn.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained

@tbg tbg force-pushed the kv-contention-info branch 2 times, most recently from 7691a7b to d063a05 Compare January 7, 2021 10:01
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 7, 2021

cc @irfansharif and @knz for pod visibility.

@nvanbenschoten this is now ready for review.

@tbg tbg marked this pull request as ready for review January 7, 2021 10:06
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.

Nice and clean!

Reviewed 1 of 14 files at r2, 17 of 21 files at r3, 2 of 5 files at r4, 1 of 1 files at r7, 21 of 21 files at r8.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @tbg)


pkg/kv/kvserver/concurrency/concurrency_manager_test.go, line 914 at r8 (raw file):

		rec := g.collect()
		for _, span := range rec {
			for _, log := range span.Logs {

Out of curiosity, why did you need to pull out the structured events separately in an earlier revision of this PR, but no longer need to do so?


pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 128 at r8 (raw file):

}

// contentionEventHelper tracks and emits ContentionEvents.

As a matter of style, I think we should move this helper and its method down below the lockTableWaiterImpl methods, maybe right after txnCache and its methods. Most readers of this file will not need to know about this contentionEventHelper, so this is a lot of noise right up top.


pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 163 at r8 (raw file):

		ice.sp.LogStructured(ice.ev)
	}
	ice.ev = nil

nit: move this one line up so that this is more easily read as "if an event already exists, take it and process it".


pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 186 at r8 (raw file):

	var timerWaitingState waitingState

	ice := contentionEventHelper{

Why ice? What does that stand for?


pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 204 at r8 (raw file):

			switch state.kind {
			case waitFor, waitForDistinguished:
				ice.emitAndInit(state.key, *state.txn)

I think we're going to need to handle other waitKinds as well, not just waitFor and waitForDistinguished. While the other waitKinds might not create contention events, they'll certainly want to finalize the existing event.

This also makes it easier to extend this in the future to create contention events for other waitKinds. For instance, waitSelf may be an eventual candidate, though it should be super rare, so I wouldn't both doing anything special for it now.

So that indicates that we may just want to hand contentionEventHelper the entire waitingState (after moving this up two lines) and let it decide what to do for each waitKind.

@tbg tbg force-pushed the kv-contention-info branch from d063a05 to 7af3d8a Compare January 11, 2021 10:16
@tbg tbg requested a review from nvb January 11, 2021 10:54
Copy link
Copy Markdown
Member Author

@tbg tbg left a comment

Choose a reason for hiding this comment

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

TFTR! This should be ready to go now.

I added a commit to handle waitSelf properly. I would like to exercise this through TestConcurrencyManager, but had some trouble (since I also don't exactly understand how to trigger the waitSelf state through its primitives). Help with that is appricated.

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


pkg/kv/kvserver/concurrency/concurrency_manager_test.go, line 914 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Out of curiosity, why did you need to pull out the structured events separately in an earlier revision of this PR, but no longer need to do so?

Because I wanted to print them, but I changed things so that on a verbose span, payloads are stringified into the log so they're already here (and in the right order relative to regular log messages).


pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 186 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Why ice? What does that stand for?

It abbreviated some earlier name for contentionEventHelper. Fixed.


pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 204 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

I think we're going to need to handle other waitKinds as well, not just waitFor and waitForDistinguished. While the other waitKinds might not create contention events, they'll certainly want to finalize the existing event.

This also makes it easier to extend this in the future to create contention events for other waitKinds. For instance, waitSelf may be an eventual candidate, though it should be super rare, so I wouldn't both doing anything special for it now.

So that indicates that we may just want to hand contentionEventHelper the entire waitingState (after moving this up two lines) and let it decide what to do for each waitKind.

Good idea - done. I stayed away from waitSelf but I think I want to fix it in a follow-up.

@tbg tbg force-pushed the kv-contention-info branch 2 times, most recently from cc22165 to 9f7b63d Compare January 12, 2021 12:56
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 12, 2021

@nvanbenschoten PTAL at the waitSelf commit. I tried writing the concurrency manager test that exercises waitSelf. I don't think I managed, but it is hopefully close. The test file has comments explaining where I'm stuck.

[2] sequence reqTxn1: acquiring latches
[2] sequence reqTxn1: blocked on select in spanlatch.(*Manager).waitForSignal

sequence req=reqTxnMiddle
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

move this to later to avoid any chance of racing with reqTxn1

@tbg tbg force-pushed the kv-contention-info branch from 9f7b63d to 756037b Compare January 12, 2021 16:48
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 13, 2021

This is ready for a look, @nvanbenschoten.

@irfansharif irfansharif self-requested a review January 18, 2021 16:57
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, I'll let Nathan sign off on waitSelf test.

var timerC <-chan time.Time
var timerWaitingState waitingState

ice := contentionEventHelper{
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why ice instead of ceh?


// Key is the key that this and the other transaction conflicted on.
bytes key = 1 [(gogoproto.casttype) = "Key"];
// Txn is the other transaction.
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

s/Txn/TxnMeta


// emitAndInit compares the waitingState's active txn (if any) against the open
// ContentionEvent (if any). If the they match, we are continuing to handle the
// same event and no action is taken. If they differ, the open event (if any) is
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

It would be a bit easier to read the implementation below if we had an explicit early return for this case.

h.ev = nil
}

// emitAndInit compares the waitingState's active txn (if any) against the open
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

s/open/{last, existing}?

#
# TODO(nvanbenschoten): would be nice to make this more explicit, but I'm not sure
# how to change `(*lockTableImpl).Format` to make it happen in debug-lock-table above.
# doesn't
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Stray "doesn't"

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 1 of 27 files at r9, 22 of 22 files at r10, 2 of 2 files at r11, 1 of 24 files at r13, 19 of 22 files at r14, 4 of 4 files at r15.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @tbg)


pkg/kv/kvserver/concurrency/concurrency_manager_test.go, line 213 at r15 (raw file):

				return c.waitAndCollect(t, mon)

			case "cancel":

We don't need this anymore, right? Were you intending to remove it?


pkg/kv/kvserver/concurrency/concurrency_manager_test.go, line 709 at r15 (raw file):

	defer c.mu.Unlock()
	if p, ok := c.txnPushes[pusher]; ok {
		// TODO(tbg): assert same pushee

Can we remove the TODO?


pkg/kv/kvserver/concurrency/lock_table.go, line 83 at r11 (raw file):

	// transaction may be a lock holder of a conflicting lock or a
	// conflicting request being sequenced through the same lockTable.
	txn  *enginepb.TxnMeta // always non-nil in waitFor{,Distinguished}

,Self now?


pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 186 at r8 (raw file):

Previously, tbg (Tobias Grieger) wrote…

It abbreviated some earlier name for contentionEventHelper. Fixed.

Did you want to update the variable name?


pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self, line 87 at r15 (raw file):

[4] sequence reqTxn2: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn

# TODO(tbg): show state

Did you mean to address this TODO?

@knz knz requested a review from angelapwen January 19, 2021 19:22
tbg added 2 commits January 20, 2021 10:23
When a payload is added, and the Span is verbose, also
stringify the payload into the verbose log.

Release note: None
This change attaches a protobuf payload to the current Span whenever a
request conflicts with another transaction. The payload contains the
contending txn (i.e. the pushee) at the time at which it was first
encountered, the key on which the conflict occurred (note that this is
not necessarily the key at which the pushee is anchored) and the time
spent waiting on the conflict (excluding intent resolution).

This enables cockroachdb#57114.

Touches cockroachdb#55583. I am not closing that issue yet because we also want
to potentially track the outcome of the conflict.

Release note: None
@tbg tbg requested review from irfansharif and nvb January 20, 2021 10:08
Copy link
Copy Markdown
Member Author

@tbg tbg left a comment

Choose a reason for hiding this comment

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

Dismissed @irfansharif from 2 discussions.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @angelapwen, @irfansharif, and @nvanbenschoten)


pkg/kv/kvserver/concurrency/lock_table_waiter.go, line 694 at r14 (raw file):

Previously, irfansharif (irfan sharif) wrote…

It would be a bit easier to read the implementation below if we had an explicit early return for this case.

Looking at this again, I agree. I also realized the code was buggy! It was resetting tBegin too frequently. The new implementation comes with a regression test for that, and the datadriven output didn't change (since the duration is mocked out there for determinism).


pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self, line 87 at r15 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Did you mean to address this TODO?

I did leave a TODO for you in the block below. It looked nontrivial to expose this properly.


pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self, line 105 at r15 (raw file):

Previously, irfansharif (irfan sharif) wrote…

Stray "doesn't"

Done.


pkg/roachpb/api.proto, line 2211 at r14 (raw file):

Previously, irfansharif (irfan sharif) wrote…

s/Txn/TxnMeta

Done.

@tbg tbg force-pushed the kv-contention-info branch from 756037b to dc689ae Compare January 20, 2021 10:08
@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 20, 2021

bors r=irfansharif,nvanbenschoten

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 20, 2021

Build failed:

@tbg
Copy link
Copy Markdown
Member Author

tbg commented Jan 21, 2021

bors r=irfansharif,nvanbenschoten

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jan 21, 2021

Build succeeded:

@craig craig bot merged commit 59ee86b into cockroachdb:master Jan 21, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

A-tracing Relating to tracing in CockroachDB.

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants