Skip to content
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

storage: queue requests to push txn / resolve intents on single keys #25014

Merged
merged 1 commit into from
May 31, 2018

Conversation

spencerkimball
Copy link
Member

@spencerkimball spencerkimball commented Apr 23, 2018

Previously, high contention on a single key would cause every thread to
push the same conflicting transaction then resolve the same intent in
parallel. This is inefficient as only one pusher needs to succeed, and
only one resolver needs to resolve the intent, and then only one writer
should proceed while the other readers/writers should in turn wait on
the previous writer by pushing its transaction. This effectively
serializes the conflicting reader/writers.

One complication is that all pushers which may have a valid, writing
transaction (i.e., Transaction.Key != nil), must push either the
conflicting transaction or another transaction already pushing that
transaction. This allows dependency cycles to be discovered.

Fixes #20448

@spencerkimball spencerkimball requested review from nvanbenschoten and a team April 23, 2018 21:03
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@spencerkimball
Copy link
Member Author

chart

@bdarnell
Copy link
Contributor

Reviewed 5 of 5 files at r1.
Review status: all files reviewed at latest revision, all discussions resolved, some commit checks failed.


pkg/storage/intent_resolver.go, line 73 at r1 (raw file):

}

func (p *pusher) activeTxn() bool {

s/active/writing/g

On reading the code below I thought active referred to having a non-expired heartbeat or something like that.


pkg/storage/intent_resolver.go, line 92 at r1 (raw file):

		// directly after EndTransaction evaluation or during GC of txn spans.
		inFlightTxnCleanups map[uuid.UUID]struct{}
		// contendedKeys is a map from key to a slice of pusher instances.

The slice is ordered; document that here.


pkg/storage/intent_resolver.go, line 118 at r1 (raw file):

	h roachpb.Header,
	pushType roachpb.PushTxnType,
) (func(*roachpb.Transaction), *roachpb.Error) {

Document the new return value (on processWriteIntentError instead of queuePushIfContended since this is the entry point from outside this file)


pkg/storage/intent_resolver.go, line 162 at r1 (raw file):

}

// queuePushIfContended handles contention on an intent by redirecting

This is complicated, and I think it would benefit from a more verbose description. What are the preconditions and postconditions for queuePushIfContended? What are the preconditions for calling the returned function?

I have two conflicting gripes about the name. I think block might be a better verb than queue for this method name. queue sounds like it enqueues the push to be run later by some other task. But IfContended is kind of misleading too - this method still does something (but not blocking: populating mu.contendedKeys) whether there is contention or not.

I think the clearest thing might be to break this out into a separate type (contentionQueue?) with add() and finish() methods.


pkg/storage/intent_resolver.go, line 190 at r1 (raw file):

		// redirect current to await prior txn's completion. Note that
		// the i==0 clause handles the special case of the current pusher
		// with an active txn having to wait on the original pushee. We

How does this original pushee get here? Isn't i==0 the first pusher instead of the original pushee?


pkg/storage/intent_resolver.go, line 191 at r1 (raw file):

		// the i==0 clause handles the special case of the current pusher
		// with an active txn having to wait on the original pushee. We
		// want to divert waiters on the first prior pusher to wait on

You mean "divert future waiters"? If they're in the future, won't they find the current pusher anyway since we iterate in reverse arrival order?

I think you want to say something like "if i == 0, the current pusher is the first one with a writing transaction...". But I'm not following why this special case is needed instead of just letting the current pusher be appended to the end of contendedKeys in the if !inserted block below.


pkg/storage/intent_resolver.go, line 196 at r1 (raw file):

			// The redirect case.
			if p.activeTxn() {
				wiErr.Intents[0] = roachpb.Intent{

Don't mutate wiErr; make a copy (and return it) if necessary.


pkg/storage/intent_resolver.go, line 204 at r1 (raw file):

			// If applicable, insert this pusher earlier into the queue of
			// waiting pushers, so that pushers with non-active txns will
			// wait on the outcome of this pusher.

Add "on their next iteration" (right?)


pkg/storage/intent_resolver.go, line 242 at r1 (raw file):

	return func(txn *roachpb.Transaction) {
		ir.mu.Lock()
		for i, p := range ir.mu.contendedKeys[key] {

Are contended keys always resolved in FIFO order? If not, what does the dependency graph look like that results in out-of-order resolutions?


pkg/storage/intent_resolver.go, line 252 at r1 (raw file):

			delete(ir.mu.contendedKeys, key)
		}
		pusher.waitCh <- txn

Only one waiter will ever get a response on waitCh. This seems prone to deadlocks if we ever mess up and let two pushers wait on the same txn. We should be defensive about this and close the channel after we write to it, and check for closed status when reading from it.


pkg/storage/intent_resolver.go, line 400 at r1 (raw file):

			resolve = true
		} else if pushee.Status == roachpb.COMMITTED && len(pushee.Intents) > 0 {
			// Only resolve intents which are not already resolved by the

Was this necessary for the rest of the change or is it an unrelated optimization.


pkg/storage/batcheval/cmd_begin_transaction.go, line 111 at r1 (raw file):

		}
	} else {
		reply.Txn.LastHeartbeat.Forward(cArgs.EvalCtx.Clock().Now())

This came from @nvanbenschoten 's change, right? Probably deserves its own commit.


Comments from Reviewable

@nvanbenschoten
Copy link
Member

Reviewed 2 of 5 files at r1.
Review status: all files reviewed at latest revision, 12 unresolved discussions, some commit checks failed.


pkg/storage/intent_resolver.go, line 74 at r1 (raw file):

func (p *pusher) activeTxn() bool {
	return p.txn != nil && p.txn.Key != nil && p.txn.Writing

Is p.txn.Key == nil && p.txn.Writing possible? I would expect that p.txn.Writing always implies p.txn.Key != nil. If so, I'd avoid the p.txn.Key != nil condition to eliminate redundancy and room for confusion.


pkg/storage/intent_resolver.go, line 162 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

This is complicated, and I think it would benefit from a more verbose description. What are the preconditions and postconditions for queuePushIfContended? What are the preconditions for calling the returned function?

I have two conflicting gripes about the name. I think block might be a better verb than queue for this method name. queue sounds like it enqueues the push to be run later by some other task. But IfContended is kind of misleading too - this method still does something (but not blocking: populating mu.contendedKeys) whether there is contention or not.

I think the clearest thing might be to break this out into a separate type (contentionQueue?) with add() and finish() methods.

a new contentionQueue sounds like a good idea to me. It will also be easier to test in isolation.


pkg/storage/intent_resolver.go, line 185 at r1 (raw file):

	var waitCh chan *roachpb.Transaction
	var inserted bool
	for i := len(ir.mu.contendedKeys[key]) - 1; i >= 0; i-- {

We index into this map a number of times in this loop. Pull the ir.mu.contendedKeys[key] slice into a variable.


pkg/storage/intent_resolver.go, line 204 at r1 (raw file):

			// If applicable, insert this pusher earlier into the queue of
			// waiting pushers, so that pushers with non-active txns will
			// wait on the outcome of this pusher.

Could this result in starvation of non-writing txns?


pkg/storage/intent_resolver.go, line 205 at r1 (raw file):

			// waiting pushers, so that pushers with non-active txns will
			// wait on the outcome of this pusher.
			if i < len(ir.mu.contendedKeys[key])-1 {

Doesn't i == 0 at this point? So this condition is equivalent to len(ir.mu.contendedKeys[key]) > 1?


pkg/storage/intent_resolver.go, line 219 at r1 (raw file):

		}
	}
	if !inserted {

Add a comment here and rename inserted to insertedInMiddle.


pkg/storage/intent_resolver.go, line 225 at r1 (raw file):

	var done bool
	if waitCh != nil {

We only want to wait here if the pusher is not writing? That seems to be the only case where we assign waitCh. I must have this wrong, so maybe just add a comment.


pkg/storage/intent_resolver.go, line 243 at r1 (raw file):

		ir.mu.Lock()
		for i, p := range ir.mu.contendedKeys[key] {
			if p == pusher {

This indicates to me that a linked-list would be a better data structure for this queue.


pkg/storage/intent_resolver.go, line 400 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Was this necessary for the rest of the change or is it an unrelated optimization.

I think Spencer mentioned that it was an unrelated optimization. I like it though. We should split it off into its own change.


pkg/storage/store.go, line 2786 at r1 (raw file):

	}

	var cleanup func(*roachpb.Transaction)

cleanup is a pretty vague name for a variable that lives in Store.Send.


pkg/storage/store.go, line 2908 at r1 (raw file):

				}
				if cleanup != nil {
					cleanup(nil)

Explain how we get in this case.


pkg/storage/batcheval/cmd_begin_transaction.go, line 111 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

This came from @nvanbenschoten 's change, right? Probably deserves its own commit.

Yeah, this is from a change I proposed a while ago. I'll pull it into its own PR and give it a nice comment.


Comments from Reviewable

@spencerkimball
Copy link
Member Author

Review status: all files reviewed at latest revision, 21 unresolved discussions, some commit checks failed.


pkg/storage/intent_resolver.go, line 73 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

s/active/writing/g

On reading the code below I thought active referred to having a non-expired heartbeat or something like that.

Done.


pkg/storage/intent_resolver.go, line 74 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Is p.txn.Key == nil && p.txn.Writing possible? I would expect that p.txn.Writing always implies p.txn.Key != nil. If so, I'd avoid the p.txn.Key != nil condition to eliminate redundancy and room for confusion.

Done.


pkg/storage/intent_resolver.go, line 92 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

The slice is ordered; document that here.

Done.


pkg/storage/intent_resolver.go, line 118 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Document the new return value (on processWriteIntentError instead of queuePushIfContended since this is the entry point from outside this file)

Done.


pkg/storage/intent_resolver.go, line 162 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

a new contentionQueue sounds like a good idea to me. It will also be easier to test in isolation.

OK, took a stab at this. PTAL.


pkg/storage/intent_resolver.go, line 185 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

We index into this map a number of times in this loop. Pull the ir.mu.contendedKeys[key] slice into a variable.

Done.


pkg/storage/intent_resolver.go, line 190 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

How does this original pushee get here? Isn't i==0 the first pusher instead of the original pushee?

The original pushee is the txn which owns the intent. Before this change, all pushers of course waited on the "original pushee".

This comment is explaining that even if there are already pushers queued up here, if those pushers are all non-writing, the first writing pusher to come along will still have to wait on the original pushee. This is necessary because non-writing pushers don't have actual transaction records which can be pushed instead of the original pushee. In other words, we can only daisy chain actual PushTxn requests when the pushers are Writing=true txns.

I updated the comment.


pkg/storage/intent_resolver.go, line 191 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

You mean "divert future waiters"? If they're in the future, won't they find the current pusher anyway since we iterate in reverse arrival order?

I think you want to say something like "if i == 0, the current pusher is the first one with a writing transaction...". But I'm not following why this special case is needed instead of just letting the current pusher be appended to the end of contendedKeys in the if !inserted block below.

That's a good way to start the explanation (incorporated). However, we must have writing transactions actually push something so we can detect dependency cycles, so we can't just have a writing transaction wait on the done channel of a non-writing transaction.


pkg/storage/intent_resolver.go, line 196 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Don't mutate wiErr; make a copy (and return it) if necessary.

Done.


pkg/storage/intent_resolver.go, line 204 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Add "on their next iteration" (right?)

No, they actually wait for it synchronously (see below where the non-writing txns wait on the done channel of an earlier pusher).


pkg/storage/intent_resolver.go, line 204 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Could this result in starvation of non-writing txns?

Yes, but nothing guaranteed it before either (although in theory this could be worse except there's an relief valve that we do release one of the non-writing txns each time). It's a really thorny problem and I can't think of a good solution that doesn't result in the entire queue getting released (the workload A performance drops by 20% if we don't order the writers first). I'll put a note in the comments.


pkg/storage/intent_resolver.go, line 205 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Doesn't i == 0 at this point? So this condition is equivalent to len(ir.mu.contendedKeys[key]) > 1?

Not necessarily. i can be > 0 if we encounter a pusher which is writing.


pkg/storage/intent_resolver.go, line 219 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Add a comment here and rename inserted to insertedInMiddle.

Done, but called it alreadyInserted because it's not necessarily in the middle.


pkg/storage/intent_resolver.go, line 225 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

We only want to wait here if the pusher is not writing? That seems to be the only case where we assign waitCh. I must have this wrong, so maybe just add a comment.

That's correct. Any writing txn must push something immediately so we can detect dependency cycles. Added a comment.


pkg/storage/intent_resolver.go, line 242 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Are contended keys always resolved in FIFO order? If not, what does the dependency graph look like that results in out-of-order resolutions?

You can have a non-writing txn and a writing txn both push the txn which owns the contended key. In that case, the non-writing txn was the original pusher, then the writing txn pusher comes along and inserts ahead of it; they resolve simultaneously in principle. So you have something like:

Orig <-- TxnA
   ^-- TxnB <-- TxnC <-- TxnD

Also, because we can insert into the middle of the dependency graph at arbitrary times, we unfortunately need to search through the slice. Doesn't seem to be a performance issue with queue sizes in the 100s.

I agree with @nvanbenschoten's suggestion to use a linked list instead.


pkg/storage/intent_resolver.go, line 243 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

This indicates to me that a linked-list would be a better data structure for this queue.

Done.


pkg/storage/intent_resolver.go, line 252 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Only one waiter will ever get a response on waitCh. This seems prone to deadlocks if we ever mess up and let two pushers wait on the same txn. We should be defensive about this and close the channel after we write to it, and check for closed status when reading from it.

How do you check for closed status when reading from a channel?


pkg/storage/intent_resolver.go, line 400 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

I think Spencer mentioned that it was an unrelated optimization. I like it though. We should split it off into its own change.

It's meaningless for the perf improvement on ycsb workload A, but does seem pretty reasonable.


pkg/storage/store.go, line 2786 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

cleanup is a pretty vague name for a variable that lives in Store.Send.

Renamed.


pkg/storage/store.go, line 2908 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Explain how we get in this case.

Some other actor could come along and write the key, but not be part of the intent resolver queue yet. We could end up getting a new write intent error, and we definitely need to make sure we cleanup before becoming part of another queue.

Added a comment.


pkg/storage/batcheval/cmd_begin_transaction.go, line 111 at r1 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Yeah, this is from a change I proposed a while ago. I'll pull it into its own PR and give it a nice comment.

Removed; it's already been fixed.


Comments from Reviewable

@bdarnell
Copy link
Contributor

bdarnell commented May 6, 2018

Reviewed 4 of 4 files at r2.
Review status: all files reviewed at latest revision, 5 unresolved discussions, some commit checks failed.


pkg/storage/intent_resolver.go, line 252 at r1 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

How do you check for closed status when reading from a channel?

x, ok := <-ch. ok will be false if the channel is closed.


pkg/storage/intent_resolver.go, line 83 at r2 (raw file):

// pushers. The physical implementation of queues is complicated by
// the difference between pushers with writing transactions (i.e. they
// have a transaction record which can by pushed) and non-writing

s/by/be/


pkg/storage/intent_resolver.go, line 100 at r2 (raw file):

// fashion, all writing transactions are daisy chained in such a way
// that they run to completion in a serialized fashion, and also still
// allow the system to detect dependency cycles and abort on deadlock.

Nice explanation.


pkg/storage/intent_resolver.go, line 121 at r2 (raw file):

// (i.e. read-only or hasn't successfully executed BeginTxn).
//
// Otherwise, returns a cleanup function to be invoked by the caller

Remove "Otherwise"; it returns these things whether it blocks or not.


pkg/storage/intent_resolver.go, line 130 at r2 (raw file):

	ctx context.Context,
	wiErr *roachpb.WriteIntentError,
	h roachpb.Header,

h.Txn is the only use of the header, so should this just take a Transaction instead?


pkg/storage/intent_resolver.go, line 166 at r2 (raw file):

		// redirect current to await prior txn's completion.
		//
		// However, if i==0, the current pusher is the first pusher with a

There is no longer an i.


pkg/storage/intent_resolver.go, line 192 at r2 (raw file):

			// results in 20% better performance on YCSB workload A.
			if e != contended.Back() {
				if p.writingTxn() {

Why this difference? Why do we swap the wait channels whether we insert before or after p?


pkg/storage/intent_resolver.go, line 282 at r2 (raw file):

// cleanup function should be invoked by the caller after the request
// which experienced the conflict has completed with a parameter
// specifying a transaction in the event that the request left its own

It feels error-prone to use the nil-ness of the transaction point to mean "did you leave an intent on the key of this WriteIntentError". Maybe this function should take an extra bool to make this more explicit.


pkg/storage/intent_resolver.go, line 305 at r2 (raw file):

	if len(wiErr.Intents) == 1 && len(wiErr.Intents[0].Span.EndKey) == 0 {
		var done bool
		if cleanup, wiErr, done = ir.contentionQ.add(ctx, wiErr, h); done {

Add a comment that we're mutating wiErr here so we may push a txn other than the one that was originally requested.


pkg/storage/store.go, line 2709 at r2 (raw file):

	defer func() {
		if cleanupAfterWriteIntentError != nil {
			// This request wrote an intent only if there was no error, the request

Are false positives here a problem? I think this is currently always true, but we've talked about introducing conditional operations that may or may not write an intent on success.

Or what about races? If we write an intent and it quickly gets resolved, do we just go ahead and push the finished transaction or could it get gummed up somewhere?


pkg/storage/store.go, line 2831 at r2 (raw file):

				// any other pusher queued up behind this RPC to proceed.
				if cleanupAfterWriteIntentError != nil {
					cleanupAfterWriteIntentError(nil)

This gives up our place in line. Would it be better to accumulate a list of cleanup functions to run when the request ultimately finishes, or is it important to clean up now and let other requests proceed instead of blocking them longer?


Comments from Reviewable

@spencerkimball
Copy link
Member Author

Review status: 2 of 5 files reviewed at latest revision, 15 unresolved discussions.


pkg/storage/intent_resolver.go, line 252 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

x, ok := <-ch. ok will be false if the channel is closed.

Done.


pkg/storage/intent_resolver.go, line 83 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

s/by/be/

Done.


pkg/storage/intent_resolver.go, line 100 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Nice explanation.

Done.


pkg/storage/intent_resolver.go, line 121 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Remove "Otherwise"; it returns these things whether it blocks or not.

Done.


pkg/storage/intent_resolver.go, line 130 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

h.Txn is the only use of the header, so should this just take a Transaction instead?

Might as well. Changed.


pkg/storage/intent_resolver.go, line 166 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

There is no longer an i.

Done.


pkg/storage/intent_resolver.go, line 192 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Why this difference? Why do we swap the wait channels whether we insert before or after p?

That was an error. We should only swap the channels if we insert before.


pkg/storage/intent_resolver.go, line 282 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

It feels error-prone to use the nil-ness of the transaction point to mean "did you leave an intent on the key of this WriteIntentError". Maybe this function should take an extra bool to make this more explicit.

I hear the complaint, but the bool then looks really redundant in that light. I tried something slightly different: added a parameter name to all the signatures. I think it's less likely to be misinterpreted now.


pkg/storage/intent_resolver.go, line 305 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Add a comment that we're mutating wiErr here so we may push a txn other than the one that was originally requested.

Done.


pkg/storage/store.go, line 2709 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Are false positives here a problem? I think this is currently always true, but we've talked about introducing conditional operations that may or may not write an intent on success.

Or what about races? If we write an intent and it quickly gets resolved, do we just go ahead and push the finished transaction or could it get gummed up somewhere?

False positives are always OK by design with the pushing machinery. If for any reason the intent is not there, we'll still wait on the transaction which is not optimal, but that is guaranteed to conclude.


pkg/storage/store.go, line 2831 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

This gives up our place in line. Would it be better to accumulate a list of cleanup functions to run when the request ultimately finishes, or is it important to clean up now and let other requests proceed instead of blocking them longer?

I think simpler is better here. I think this is a low probability event. Accumulating a list of cleanup functions seems


Comments from Reviewable

@bdarnell
Copy link
Contributor

bdarnell commented May 8, 2018

:lgtm:


Reviewed 3 of 3 files at r3.
Review status: all files reviewed at latest revision, 4 unresolved discussions, some commit checks failed.


Comments from Reviewable

@nvanbenschoten
Copy link
Member

Reviewed 2 of 4 files at r2, 2 of 3 files at r3.
Review status: all files reviewed at latest revision, all discussions resolved, some commit checks failed.


pkg/storage/intent_resolver.go, line 156 at r2 (raw file):

	var alreadyInserted bool
	var curElement *list.Element
	contended, ok := cq.mu.keys[key]

We don't need to hold the contentionQueue lock until here. Since we're performing a few allocations above, we should move cq.mu.Lock() down.


pkg/storage/intent_resolver.go, line 116 at r3 (raw file):

}

// add adds the intent specified in the supplied wiErr to the

Add a note to this method comment that len(wiErr.Intents) == 1. Maybe also assert this internally.


pkg/storage/intent_resolver.go, line 139 at r3 (raw file):

	// updateErr creates a copy of the write intent error with the
	// intent updated to point to a new transaction.
	updateErr := func(txn *roachpb.Transaction) *roachpb.WriteIntentError {

This doesn't need to capture anything, and I think having it here and called "update" will just create confusion because it's throwing away eveything from the old error except the key, which we know is shared with the overlapping txn. I think it would be more clear to create a stand-alone function newWriteIntentError(txn *roachpb.Transaction, key roachpb.Key) *roachpb.WriteIntentError.


pkg/storage/intent_resolver.go, line 161 at r3 (raw file):

		cq.mu.keys[key] = contended
	}
	for e := contended.Back(); e != nil; e = e.Prev() {

Make a note somewhere that Back is the front of the queue. That threw me off a bit.

list.List is doubly linked, so you could also just reverse the order of the linked list.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the sequence-txns branch 2 times, most recently from 02f496a to 8a30bff Compare May 14, 2018 17:27
@spencerkimball
Copy link
Member Author

PTAL. I updated this after realizing that even if a transaction's Writing field is false, it may still have a transaction record and may have succeeded in writing an intent and be involved in a dependency cycle. This is true because the first batch may have split between ranges where one range contained writes and the txn record while the other range(s) may have experienced a conflict and ended up in the contention queue.

This means all transactions which have a non-nil key must push in order to detect deadlocks. In order to make this work and still maintain the performance gains, I had to rethink things a bit. Now we have a strictly ordered FIFO queue (nice side benefit is this eliminates the starvation case which @nvanbenschoten pointed out in his review). I also avoid pushing in daisy-chained fashion, though that might still be worthwhile as an optimization.

@bdarnell
Copy link
Contributor

LGTM, although it's concerning that there's not a lot of test changes to go with this reworking.

Can you post an updated version of the performance chart above?


Reviewed 2 of 3 files at r4.
Review status: 4 of 5 files reviewed at latest revision, 4 unresolved discussions, some commit checks failed.


pkg/storage/intent_resolver.go, line 84 at r4 (raw file):

type contendedKey struct {
	ll          *list.List
	lastTxnMeta *enginepb.TxnMeta

This field needs documentation.


pkg/storage/intent_resolver.go, line 216 at r4 (raw file):

				cq.mu.Unlock()
				if pushTxnMeta == nil {
					time.Sleep(10 * time.Millisecond)

You'll hit this sleep an unpredictable number of times (if pushIfActiveCh if both pushIfActiveCh and waitCh are readable, it will choose randomly between them), which I don't think is what you want. I think what you want to do here is replace pushIfActiveCh with a time.After instead of sleeping. This way you'll check contended.lastTxnMeta every 10ms but these waits will be interruptible if one of the other channels fire.


Comments from Reviewable

@tbg tbg mentioned this pull request May 28, 2018
@spencerkimball
Copy link
Member Author

Will post updated graph.


Review status: 4 of 5 files reviewed at latest revision, 6 unresolved discussions, some commit checks failed.


pkg/storage/intent_resolver.go, line 156 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

We don't need to hold the contentionQueue lock until here. Since we're performing a few allocations above, we should move cq.mu.Lock() down.

Done.


pkg/storage/intent_resolver.go, line 116 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Add a note to this method comment that len(wiErr.Intents) == 1. Maybe also assert this internally.

Done.


pkg/storage/intent_resolver.go, line 139 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

This doesn't need to capture anything, and I think having it here and called "update" will just create confusion because it's throwing away eveything from the old error except the key, which we know is shared with the overlapping txn. I think it would be more clear to create a stand-alone function newWriteIntentError(txn *roachpb.Transaction, key roachpb.Key) *roachpb.WriteIntentError.

This comment has been obviated by recent changes.


pkg/storage/intent_resolver.go, line 161 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Make a note somewhere that Back is the front of the queue. That threw me off a bit.

list.List is doubly linked, so you could also just reverse the order of the linked list.

Not a concern any longer, as the new code keeps a strictly fifo queue and does not search backwards through the list.


pkg/storage/intent_resolver.go, line 84 at r4 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

This field needs documentation.

Done.


pkg/storage/intent_resolver.go, line 216 at r4 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

You'll hit this sleep an unpredictable number of times (if pushIfActiveCh if both pushIfActiveCh and waitCh are readable, it will choose randomly between them), which I don't think is what you want. I think what you want to do here is replace pushIfActiveCh with a time.After instead of sleeping. This way you'll check contended.lastTxnMeta every 10ms but these waits will be interruptible if one of the other channels fire.

I've updated the code not to randomly wait between these things. There's now an updateCh created in each pusher object if the pusher has an active transaction.


Comments from Reviewable

@spencerkimball
Copy link
Member Author

Here's an updated graph: notice that the additional changes which were required to handle cases where deadlock could be introduced have caused our throughput with very high contention to degrade. On a positive note, I added a mechanism to delay checking for dependency cycles, which has resulted in better performance for small to high contention.

chart 1

@spencerkimball spencerkimball force-pushed the sequence-txns branch 2 times, most recently from a4604b1 to 6b1b429 Compare May 30, 2018 22:14
@bdarnell
Copy link
Contributor

Review status: all files reviewed at latest revision, 4 unresolved discussions, some commit checks failed.


Comments from Reviewable

@tbg
Copy link
Member

tbg commented May 31, 2018

cc @andreimatei -- I saw some mention of the Writing field in the discussion, so I assume it plays a role here and you should check that your recent changes in this area are compatible with what this PR wants.

@andreimatei
Copy link
Contributor

nit: please update the PR description when the commit message changes. The commit now says Transaction.Key != nil, the PR description talks about Txn.Writing.

The PR Tobi is talking about is #25541, which changes when/how the .Writing field is set. The PR wants to set it exclusively on the client, before the BeginTxn request is sent. I think this only serves to help this PR - I believe you can go back to the implementation relying on the Writing field if that was easier/better (it would seem cleaner to me than to rely on the Key but I am curious why changing from one to the other triggered many changes in the PR, as you suggest?).

I have a question, though - if a txn doesn't have a txn record (yet), can it push/be pushed? Won't both pushing and being pushed attempt to check its txn record and do something when it's not found? In fact, I believe I've recently made some check of a either the pusher's or the pushee's txn record run sooner than it used to in order to avoid some race.

And a 2nd question. The commit says "I also avoid pushing in daisy-chained fashion, though that might still be worthwhile as an optimization". I'm confused about what this means - isn't the daisy chaining the whole point of this PR? Or are you saying that this PR introduces a queue of pushers, but they're all waiting on the original intent?


Review status: all files reviewed at latest revision, 4 unresolved discussions, some commit checks failed.


Comments from Reviewable

@spencerkimball
Copy link
Member Author

@andreimatei: updated PR description. Relying on the Writing field was unworkable for other reasons, so no need to go back to it. If the new behavior of Writing is that it is set whenever there is any possibility of an intent existing somewhere, then we could use it instead of Key != nil. Doesn't seem like a big deal, but feel free to make the change to this code when #25541 goes in.

In the latest changes in the PR, no txn will ever be pushed if it hasn't laid down an intent. Still, you can encounter an intent and go to push the txn record but discover it hasn't been written yet – that results in an automatic abort. A pusher need not have a txn record, either because it's a non-transactional operation, a read-only txn without a Key set, or a write txn with a Key set, but not txn record written yet. The pusher, in the cases where Key is set, will attempt to query its own txn record periodically in case it begins to accrue dependencies which indicate a dependency cycle / deadlock.

Daisy-chaining was the original direction of this PR, but it didn't end up being very useful. I'm actually doubtful now that it would amount to any kind of worthwhile optimization, because we now delay the check for dependency cycles. This PR does introduce a FIFO queue of pushers, lined up behind a contended intent. The front of the queue will push the owner of the intent, and the others in the queue ( the ones which may have their own txn records) will also push the owner after a delay. When the pusher at the front of the queue succeeds, it re-executes its request and either leaves its own intent (the next queued pusher will then push its txn), or simply reads the intent (the next queued pusher will immediately re-execute instead of pushing).


Review status: 4 of 6 files reviewed at latest revision, 4 unresolved discussions, some commit checks pending.


Comments from Reviewable

Previously, high contention on a single key would cause every thread to
push the same conflicting transaction then resolve the same intent in
parallel. This is inefficient as only one pusher needs to succeed, and
only one resolver needs to resolve the intent, and then only one writer
should proceed while the other readers/writers should in turn wait on
the previous writer by pushing its transaction. This effectively
serializes the conflicting reader/writers.

One complication is that all pushers which may have a valid, writing
transaction (i.e., `Transaction.Key != nil`), must push either the
conflicting transaction or another transaction already pushing that
transaction. This allows dependency cycles to be discovered.
@spencerkimball
Copy link
Member Author

bors r+

craig bot pushed a commit that referenced this pull request May 31, 2018
25014: storage: queue requests to push txn / resolve intents on single keys r=spencerkimball a=spencerkimball

Previously, high contention on a single key would cause every thread to
push the same conflicting transaction then resolve the same intent in
parallel. This is inefficient as only one pusher needs to succeed, and
only one resolver needs to resolve the intent, and then only one writer
should proceed while the other readers/writers should in turn wait on
the previous writer by pushing its transaction. This effectively
serializes the conflicting reader/writers.
    
One complication is that all pushers which may have a valid, writing
transaction (i.e., `Transaction.Key != nil`), must push either the
conflicting transaction or another transaction already pushing that
transaction. This allows dependency cycles to be discovered.

Fixes #20448 

25791: jobs: bump default progress log time to 30s r=mjibson a=mjibson

The previous code allowed updates to be performed every 1s, which could
cause the MVCC row to be very large causing problems with splits. We
can update much more slowly by default. In the case of a small backup
job, the 5% fraction threshold will allow a speedier update rate.

Remove a note that's not useful anymore since the referred function
can now only be used in the described safe way.

See #25770. Although this change didn't fix that bug, we still think
it's a good idea.

Release note: None

26293: opt: enable a few distsql logictests r=RaduBerinde a=RaduBerinde

 - `distsql_indexjoin`: this is only a planning test. Modifying the
   split points and queries a bit to make the condition more
   restrictive and make the optimizer choose index joins. There was a
   single plan that was different, and the difference was minor (the
   old planner is emitting an unnecessary column).

 - `distsql_expr`: logic-only test, enabling for opt.

 - `distsql_scrub`: planning test; opt version commented out for now.

Release note: None

Co-authored-by: Spencer Kimball <[email protected]>
Co-authored-by: Matt Jibson <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
@craig
Copy link
Contributor

craig bot commented May 31, 2018

Build succeeded

@craig craig bot merged commit 9a54256 into cockroachdb:master May 31, 2018
@jseldess
Copy link
Contributor

@spencerkimball, @nvanbenschoten, can either of you help me write a brief release note for this change? I somehow got left out of the July 2 release notes.

@nvanbenschoten
Copy link
Member

How's this?

Release note (performance improvement): Improve throughput of highly contended writes with new contentionQueue.

@jseldess
Copy link
Contributor

jseldess commented Aug 2, 2018

Yes. Thanks, @nvanbenschoten!

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.

YCSB workload A kills cockroach
7 participants