Skip to content

storage: push txn queues for high contention scenarios#13501

Merged
spencerkimball merged 1 commit intomasterfrom
spencerkimball/high-contention
Feb 23, 2017
Merged

storage: push txn queues for high contention scenarios#13501
spencerkimball merged 1 commit intomasterfrom
spencerkimball/high-contention

Conversation

@spencerkimball
Copy link
Copy Markdown
Member

@spencerkimball spencerkimball commented Feb 9, 2017

This commit replaces the abort-immediately-if-higher-priority txn semantics
with a more traditional locking approach. This yields significantly higher
performance on high-contention workloads.

Locking is done by disallowing intents to be pushed with two exceptions:

  1. the pusher has a maximum txn priority (always pushes), or 2) the
    pushee has a minimum txn priority (can always be pushed).

When a PushTxn request fails because the intent's txn is still live,
the pushee txn is added as a blocking txn to a new per-replica
pushTxnQueue. This queue maintains a list of waiting pushers per
blocking txn. Push failures are immediately retried and in the event
that the PushTxn request's PusheeTxn is in the queue, the pusher
request is enqueued behind it, waiting for it to be resolved.

While waiting on a txn to be resolved, a pusher periodically (with an
exponentially increasing backoff/retry) updates its own status. This
informs it whether it has had its priority ratcheted or been concurrently
aborted or committed. It also gathers information on which txns are in
turn waiting on the pusher in order to build a transitive closure of
txn dependencies. This is used to determine whether there is a
dependency cycle which would mean a deadlock. In the event of a deadlock,
txns with lower priorities are aborted.


This change is Reviewable

@tbg
Copy link
Copy Markdown
Member

tbg commented Feb 9, 2017

Push failures are immediately retried and in the event
that the PushTxn request's PusheeTxn is in the queue, the pusher
request is enqueued behind it, waiting for it to be resolved.

What do you mean by immediately retried? Also, is the Pusher notified of the failed push and goes into a backoff loop, or does the request block and retry its push locally? If the latter, do you also trigger the push when the txn commits? Could tell from the code, but can't really dig on my phone, and would be good to belabor this in the description anyway.

@petermattis
Copy link
Copy Markdown
Collaborator

I haven't looked at the implementation yet, but does the deadlock detection handle cycles that are larger than 2? That is, A -> B -> C -> D -> A.

@spencerkimball
Copy link
Copy Markdown
Member Author

spencerkimball commented Feb 9, 2017

I'm going to try creating a diagram of the old model and of the new model because there are just layers in layers here. Let me try to verbally answer the question:

When a PushTxn request first fails, its failure is noticed in replica_proposal during the local state application. This causes the pushTxnQueue to have an entry with the pushee txn's ID created (nothing is waiting on it yet, it's just a stub indicating that there have been push txn failures on that txn record). When the failed PushTxn request bubbles up to the Store.Send level, it immediately (no backoff) retries (nb: there is a testing knob that will cause it to return the error).

On the retry, the request hits a new conditional execution path in Replica.tryAddWriteCmd which checks the pushTxnQueue for an entry corresponding to the pushee txn's ID. If one exists (and a bunch of other criteria are met), the PushTxn request is enqueued and goes into a select loop with exit conditions including:

  • ctx.Done
  • pushee txn has been aborted, committed, or timestamp pushed forward
  • periodic check of pushee txn's last heartbeat indicates it has been abandoned
  • periodic update of pusher's txn record indicates change in status

As additional requests come in to push the txn record, they are immediately enqueued in same manner. So the pushTxnQueue is a quick check only for batches which contain a single PushTxn request, and is only activated once a failure to push has occurred. When the pushee txn record is updated, all waiters are signaled and if the txn status indicates the push is already complete, they exit without additional work being required.

Any call to PushTxn or EndTransaction which changes the state of a txn calls through to the pushTxnQueue and updates the txn, signaling all PushTxn waiters on the record. This is done from replica_proposal when handling the local evaluation path.

The only backoff loop is for checking the status of the pusher's txn. This is the mechanism by which we find out about which txn might in turn be depending on the pusher. This is called at 50ms, 100ms, 200ms, 400ms, 800ms, and then repeated at 1s intervals. A PUSH_QUERY PushTxn request now returns an additional WaitingTxns repeated uuid.UUID field which lists the transitive closure of dependencies. @petermattis, yes it noticies cycles larger than 2. It might simply require more than a single backoff to become aware of a larger cycle.

@spencerkimball spencerkimball force-pushed the spencerkimball/high-contention branch from 6889cab to 92d68c0 Compare February 9, 2017 14:22
@spencerkimball
Copy link
Copy Markdown
Member Author

../examples-go/bank2/bank2 -contention=high -num-transfers=500 -concurrency=X

Scenario	Concurrency	Execution time	Retries
New		1		4.005865074	2
New 		2		6.005129993	443
New		4		6.002048144	495
New		8		8.006018092	515
New		16		10.00227567	521
New		32		14.79346422	535

Old		1		4.002261427	1
Old		2		4.053355745	179
Old		4		6.006673351	580
Old		8		9.152501197	1,637
Old		16		22.39772261	5,025
Old		32		126.8099962	21,989

@knz
Copy link
Copy Markdown
Contributor

knz commented Feb 9, 2017

What concerns me with this approach is that we'll soon work on the ability to cancel ongoing transactions asynchronously (e.g. some interface to kill long-running queries from the UI) and introducing a queue makes it necessary to scan this queue and remove items in the middle when a txn is cancelled. Hence a couple of questions:

  • there's a Go channel in this patch for waiting transactions. What will be the plan when we need to cancel txns currently queued in there?
  • once we remove a txn in one of the queues, what's the complexity of removing the dependency from the dependent txns?

@spencerkimball
Copy link
Copy Markdown
Member Author

spencerkimball commented Feb 9, 2017

@knz, you don't need to look in the queue. Simply send a PushTxn request with PushType = roachpb.PUSH_ABORT and PushTxnRequest.Force = true and the txn's record will have status set to ABORTED. If that txn is busy running it will either discover it's been aborted when it heartbeats or attempts to commit or runs into an abort cache entry. If it's in the queue (see code in push_txn_queue.go, it will find out pretty quickly as it updates from its own txn record and will exit the queue and return an already-aborted error.

There is no need to explicitly remove from the queue (because pushers remove themselves).

It's safe to ignore removing a txn from other txn's dependency lists when aborted. In the worst case, we'd end up aborting another txn unnecessarily in the belief that there is a cycle that doesn't exist involving the already-aborted txn. This is also true in other circumstances, like when a txn commits just before we notice the dependency cycle and abort anyway. We still need clients to do txn retry loops for SERIALIZABLE restarts, so this is just another retry occasion, and likely an exceedingly rare one at that.

@knz
Copy link
Copy Markdown
Contributor

knz commented Feb 9, 2017

Thanks for clarifying!

@bdarnell
Copy link
Copy Markdown
Contributor

bdarnell commented Feb 9, 2017

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


.gitignore, line 20 at r1 (raw file):

/cockroach-darwin*
/certs
pkg/cmd/allocsim/allocsim

I think we prefer to put things like this in a per-directory pkg/cmd/allocsim/.gitignore file.


pkg/roachpb/data.go, line 750 at r1 (raw file):

//
// If userPriority is less than or equal to MinUserPriority, returns 0;
// if greater than or equal to MaxUserPriority, returns MaxInt32.

Refer to {Min,Max}UserPriority instead of 0 and MaxInt32.


pkg/sql/txn.go, line 73 at r1 (raw file):

		return p.txn.SetUserPriority(roachpb.NormalUserPriority)
	case parser.High:
		return p.txn.SetUserPriority(roachpb.MaxUserPriority)

I'm worried about removing the old randomized priority bands. Now, high priority transactions are special, and two contending high-priority transactions will interact badly with each other. With randomized bands, they'd still preempt most lower-priority transactions but interact with each other the same way two normal transactions would. This is going to make it risky to ever use a non-default priority.


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

	}

	resolveIntents, pushErr := ir.maybePushTransactions(ctx, wiErr.Intents, h, pushType, false)

Rename pushErr (and resErr below) to err now that we're no longer handling errors out-of-order.


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

	resolveIntents, pushErr := ir.maybePushTransactions(ctx, wiErr.Intents, h, pushType, false)
	if pushErr != nil {
		return pushErr

The old code here would hide certain errors from the caller (primarily AmbiguousResultError - an ambiguous result on a push shouldn't make the entire operation ambiguous)


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

		// when they may be (especially when a test cluster is in
		// the process of shutting down).
		log.Warningf(ctx, "asynchronous resolveIntents failed: %s", resErr)

I think we probably want to return errors here instead of swallowing them now (with the same AmbiguousResultError caveat as above). If we hide errors from either push or resolve, we should hide from both.


pkg/storage/push_txn_queue.go, line 66 at r1 (raw file):

type waitingPush struct {
	req     *roachpb.PushTxnRequest
	pending chan *roachpb.Transaction // closed when complete

This comment appears to be out of date; we write (possibly nil) values to the channel.


pkg/storage/push_txn_queue.go, line 89 at r1 (raw file):

	store *Store
	mu    struct {
		sync.Locker                           // Protects all variables in the mu struct

Why is this sync.Locker instead of syncutil.Mutex?


pkg/storage/push_txn_queue.go, line 115 at r1 (raw file):

	}
	ptq.mu.txns = map[uuid.UUID]*pendingTxn{}
	ptq.mu.Unlock()

What's stopping another goroutine from adding a new waiter right after we've cleared the map here?


pkg/storage/push_txn_queue.go, line 158 at r1 (raw file):

	// Send on pending waiter channels outside of the mutex lock.
	for _, w := range waiters {
		w.pending <- txn

We've been bitten before by passing pointers to the same Transaction around to multiple callers. Do we need to be worried about that here?


pkg/storage/push_txn_queue.go, line 187 at r1 (raw file):

		return dependents
	}
	return []uuid.UUID(nil)

This can be return nil (And I don't think you need to call this out in the doc comment)


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

// MaybeWait checks whether there is a queue already established for
// pushing the transaction. If not, of if the PushTxn request isn't

s/of/or/


pkg/storage/push_txn_queue.go, line 245 at r1 (raw file):

		// pushee (updating is non-zero), then this defaults to current
		// time plus twice the heartbeat interval. Otherwise, based on last
		// active timestamp plus twice the heartbeat interval.

Why is this conditional? (instead of always basing it on LastActive()?


pkg/storage/push_txn_queue.go, line 275 at r1 (raw file):

		case <-txnExpiresCh:
			// Periodically check whether the txn has been abandoned.

s/the txn/the pushee txn/


pkg/storage/push_txn_queue.go, line 279 at r1 (raw file):

				updatedPushee, _, pErr := ptq.queryTxnStatus(ctx, req.PusheeTxn, ptq.store.Clock().Now())
				if pErr != nil {
					return nil, pErr

If we hit this early return pending.updating is never reset.


pkg/storage/push_txn_queue.go, line 287 at r1 (raw file):

		case <-queryTxnCh:
			// Query the transaction periodically to get updated status.

s/the transaction/the pusher transaction/

Rename queryTxnCh and txnExpiresCh to match.


pkg/storage/push_txn_queue.go, line 329 at r1 (raw file):

// to push a transaction, as two txns can have circular dependencies
// where both are unable to push because they have different
// information about their own txns.

Is this scenario covered by existing tests?


pkg/storage/replica.go, line 2070 at r1 (raw file):

		// may cause this request to wait and either return a successful push
		// txn response or else allow this request to proceed.
		if ba.IsSinglePushTxnRequest() {

Why does this happen in tryAddWriteCmd instead of in the loop in Store?


pkg/storage/replica_command.go, line 1374 at r1 (raw file):

	if args.PushType == roachpb.PUSH_QUERY {
		// Get the list of txns waiting on this txn.
		reply.WaitingTxns = r.pushTxnQueue.GetDependents(*args.PusheeTxn.ID)

This is sketchy - it runs on each replica and produces different results. We don't currently use the replies from different replicas in ways that could cause them to diverge, but I think it's possible for this to return incorrect results if e.g. the lease changed hands while this PushTxn command was in flight. It might be better to introduce a new read-only command that returns waiting transactions instead of adding to the overloaded PushTxn.


pkg/storage/replica_command.go, line 1399 at r1 (raw file):

		reason = "pushee is SNAPSHOT"
		pusherWins = true
	case priority > roachpb.MinTxnPriority && reply.PusheeTxn.Priority == roachpb.MinTxnPriority:

For the record, this is going to require a stop-the-world migration.


pkg/storage/replica_proposal.go, line 157 at r1 (raw file):

	maybeGossipNodeLiveness *roachpb.Span

	// Set what a transaction record is updated, after a call to

That's not a sentence.


pkg/util/retry/retry.go, line 135 at r1 (raw file):

// whether the retry loop should terminate, in the event that the
// current attempt exceeds the maximum number of retries.
func (r *Retry) ManualNext() bool {

Add a simple test of this method in retry_test.go


Comments from Reviewable

@spencerkimball
Copy link
Copy Markdown
Member Author

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


.gitignore, line 20 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

I think we prefer to put things like this in a per-directory pkg/cmd/allocsim/.gitignore file.

Done.


pkg/roachpb/data.go, line 750 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Refer to {Min,Max}UserPriority instead of 0 and MaxInt32.

Done.


pkg/sql/txn.go, line 73 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

I'm worried about removing the old randomized priority bands. Now, high priority transactions are special, and two contending high-priority transactions will interact badly with each other. With randomized bands, they'd still preempt most lower-priority transactions but interact with each other the same way two normal transactions would. This is going to make it risky to ever use a non-default priority.

Two contending high priority transactions don't react badly IMO. They simply cannot abort each other. In this new model, normal transactions react the same way as two high priority transactions. The priorities have been reduced to a random variable for deciding which txns in a dependency cycle to abort to avoid deadlock.

We will need to update the docs to clarify this new txn model. Our behavior is now much closer to that of traditional locking. For priorities, we currently only allow {LOW, NORMAL, HIGH} in the SQL API. What we need to highlight here is that LOW is for low-priority background work and will always be aborted by contending non-LOW transactions. HIGH is for high-priority low-latency work that will always abort non-HIGH transactions.


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

Previously, bdarnell (Ben Darnell) wrote…

Rename pushErr (and resErr below) to err now that we're no longer handling errors out-of-order.

Done.


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

Previously, bdarnell (Ben Darnell) wrote…

The old code here would hide certain errors from the caller (primarily AmbiguousResultError - an ambiguous result on a push shouldn't make the entire operation ambiguous)

OK, I'm now swallowing ambiguous result errors.


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

Previously, bdarnell (Ben Darnell) wrote…

I think we probably want to return errors here instead of swallowing them now (with the same AmbiguousResultError caveat as above). If we hide errors from either push or resolve, we should hide from both.

Done.


pkg/storage/push_txn_queue.go, line 66 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

This comment appears to be out of date; we write (possibly nil) values to the channel.

Done.


pkg/storage/push_txn_queue.go, line 89 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Why is this sync.Locker instead of syncutil.Mutex?

Done.


pkg/storage/push_txn_queue.go, line 115 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

What's stopping another goroutine from adding a new waiter right after we've cleared the map here?

I think we might want to add Stop() and Start() methods instead of just this Clear(). Will make that change.


pkg/storage/push_txn_queue.go, line 158 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

We've been bitten before by passing pointers to the same Transaction around to multiple callers. Do we need to be worried about that here?

Done.


pkg/storage/push_txn_queue.go, line 187 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

This can be return nil (And I don't think you need to call this out in the doc comment)

Done.


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

Previously, bdarnell (Ben Darnell) wrote…

s/of/or/

Done.


pkg/storage/push_txn_queue.go, line 245 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Why is this conditional? (instead of always basing it on LastActive()?

Because there are multiple waiters, and if you're a waiter which isn't handling the active update, you don't want to go into a busy loop on the expired heartbeat timeout. You also don't want to just not wait with any timeout or you might never wake up to discover the txn has expired.


pkg/storage/push_txn_queue.go, line 275 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

s/the txn/the pushee txn/

Done.


pkg/storage/push_txn_queue.go, line 279 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

If we hit this early return pending.updating is never reset.

Done.


pkg/storage/push_txn_queue.go, line 287 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

s/the transaction/the pusher transaction/

Rename queryTxnCh and txnExpiresCh to match.

Done.


pkg/storage/push_txn_queue.go, line 329 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Is this scenario covered by existing tests?

No, I'm adding tests still.


pkg/storage/replica.go, line 2070 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Why does this happen in tryAddWriteCmd instead of in the loop in Store?

No real reason. This seemed more appropriate to me. Do you think it should be in Store?


pkg/storage/replica_command.go, line 1374 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

This is sketchy - it runs on each replica and produces different results. We don't currently use the replies from different replicas in ways that could cause them to diverge, but I think it's possible for this to return incorrect results if e.g. the lease changed hands while this PushTxn command was in flight. It might be better to introduce a new read-only command that returns waiting transactions instead of adding to the overloaded PushTxn.

Yeah, I think we want a different push query type operation. This would benefit from being a read-only op. I'll make that change.


pkg/storage/replica_command.go, line 1399 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

For the record, this is going to require a stop-the-world migration.

Yes. Will add that to the PR description.


pkg/storage/replica_proposal.go, line 157 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

That's not a sentence.

Done.


pkg/util/retry/retry.go, line 135 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Add a simple test of this method in retry_test.go

Will do; was just getting this out the door, as @petermattis suggested. Will get tests done this weekend.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the spencerkimball/high-contention branch 3 times, most recently from d6d79fc to 8448919 Compare February 11, 2017 18:25
@spencerkimball
Copy link
Copy Markdown
Member Author

@bdarnell I've addressed the pushTxnQueue.Clear() comments by adding pushTxnQueue.Enable and pushTxnQueue.ClearAndDisable methods. If the queue is not enabled, then no waiters can be added.

Also, removed the PUSH_QUERY behavior in PushTxn command and added a new read-only QueryTxn command.

Tests still to come.

@petermattis
Copy link
Copy Markdown
Collaborator

This is a Big Scary Change. Can it be protected by an env var? That seems a bit difficult, but I worry about the beta release we enable this if we don't have prior testing on our test clusters.


Review status: 19 of 40 files reviewed at latest revision, 25 unresolved discussions, some commit checks failed.


pkg/roachpb/batch_generated.go, line 1 at r2 (raw file):

// Code generated by gen_batch.go; DO NOT EDIT

You're supposed to edit gen_batch.go to make edits to this file.


pkg/storage/push_txn_queue.go, line 37 at r2 (raw file):

// shouldPushImmediately returns whether the PushTxn request should
// proceed without queueing. This is true for TOUCH and QUERY pushes,

I thought the QUERY pushes were removed in this PR.


pkg/storage/push_txn_queue.go, line 67 at r2 (raw file):

}

// A waitingPush represents a waiting PushTxn command. It also

I find the "push" terminology confusing. Not for this PR, but we should think about revisiting it now that we have something that is essentially a lock.

In the short term, waitingTxn might be a better name for this structure and meshes with pendingTxn.


.gitignore, line 20 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

I think we prefer to put things like this in a per-directory pkg/cmd/allocsim/.gitignore file.

Or use go install which will avoid rebuilds by caching intermediaries.


Comments from Reviewable

@bdarnell
Copy link
Copy Markdown
Contributor

I don't think putting this behind an env var is going to be feasible. I think we should put it on at least one test cluster before merging it to master, then merge it to master just after a beta release so we'll have a full week to run it on all the test clusters.


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


pkg/roachpb/data.go, line 751 at r2 (raw file):

// If userPriority is less than or equal to MinUserPriority, returns
// MinUserPriority; if greater than or equal to MaxUserPriority,
// returns MaxUserPriority.

These should be {Min,Max}TxnPriority, not {Min,Max}UserPriority.


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

	defer func() {
		if _, ok := pErr.GetDetail().(*roachpb.AmbiguousResultError); ok {
			pErr = wiPErr

The code that handled WriteIntentErrors specially at the call site for this method (in store.go) is gone; this will cause the entire operation to fail. Instead, we want to assume that the resolution succeeded and allow the retry in store.go to proceed. We could either transform AmbiguousResultError into nil here or catch the error in the caller; I think the latter would be cleaner.


pkg/storage/push_txn_queue.go, line 287 at r1 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

Done.

To clarify, I meant s/queryTxnCh/queryPusherTxnCh/ and s/txnExpiresCh/pusheeTxnExpiresCh/. "Txn" is ambiguous here.


pkg/storage/replica.go, line 2070 at r1 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

No real reason. This seemed more appropriate to me. Do you think it should be in Store?

I think so. It's related to the intentResolver stuff done in that loop, and I think it's a better fit there than here (since tryAddWriteCmd is mainly about the things that need to be repeated after a raft-related retry).


pkg/storage/replica_command.go, line 1399 at r1 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

Yes. Will add that to the PR description.

Actually, I don't think stop-the-world is enough without freeze-cluster. It would be enough for the new Force flag, which is what I was thinking of when I made this comment, but the change to the interpretation of the priority fields is unsafe. The simplest way to make this change in a safe way (assuming you don't want to wait for propEvalKV) would be to introduce some new field on the request to indicate that it is using the new semantics.


pkg/storage/replica_command.go, line 1329 at r2 (raw file):

		// vulnerable, but the alternative is having more intents never cleaned
		// up eagerly.
		if args.PushType == roachpb.PUSH_QUERY {

We need to keep the implementation of PUSH_QUERY mode intact to avoid replica divergence (even with stop-the-world. We need freeze-cluster or propEvalKV to make changes like this safely)


Comments from Reviewable

@spencerkimball
Copy link
Copy Markdown
Member Author

Agreed. Would be very difficult and we'd have to leave a lot of stuff in place which is complex (the backoff loops). I'll target getting this in by Thursday to meet the suggested schedule.


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


pkg/roachpb/batch_generated.go, line 1 at r2 (raw file):

Previously, petermattis (Peter Mattis) wrote…

You're supposed to edit gen_batch.go to make edits to this file.

I didn't edit this file. I edited gen_batch.go and ran go generate.


pkg/roachpb/data.go, line 751 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

These should be {Min,Max}TxnPriority, not {Min,Max}UserPriority.

Done.


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

Previously, bdarnell (Ben Darnell) wrote…

The code that handled WriteIntentErrors specially at the call site for this method (in store.go) is gone; this will cause the entire operation to fail. Instead, we want to assume that the resolution succeeded and allow the retry in store.go to proceed. We could either transform AmbiguousResultError into nil here or catch the error in the caller; I think the latter would be cleaner.

Ah, good point. I'm moving this error handling to store.go.


pkg/storage/push_txn_queue.go, line 287 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

To clarify, I meant s/queryTxnCh/queryPusherTxnCh/ and s/txnExpiresCh/pusheeTxnExpiresCh/. "Txn" is ambiguous here.

Done.


pkg/storage/push_txn_queue.go, line 37 at r2 (raw file):

Previously, petermattis (Peter Mattis) wrote…

I thought the QUERY pushes were removed in this PR.

Done.


pkg/storage/push_txn_queue.go, line 67 at r2 (raw file):

Previously, petermattis (Peter Mattis) wrote…

I find the "push" terminology confusing. Not for this PR, but we should think about revisiting it now that we have something that is essentially a lock.

In the short term, waitingTxn might be a better name for this structure and meshes with pendingTxn.

The waiter isn't necessarily a transaction, so I'm not in favor of waitingTxn. I'll wait for a rename to change this terminology.


pkg/storage/replica.go, line 2070 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

I think so. It's related to the intentResolver stuff done in that loop, and I think it's a better fit there than here (since tryAddWriteCmd is mainly about the things that need to be repeated after a raft-related retry).

Done.


pkg/storage/replica_command.go, line 1399 at r1 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Actually, I don't think stop-the-world is enough without freeze-cluster. It would be enough for the new Force flag, which is what I was thinking of when I made this comment, but the change to the interpretation of the priority fields is unsafe. The simplest way to make this change in a safe way (assuming you don't want to wait for propEvalKV) would be to introduce some new field on the request to indicate that it is using the new semantics.

Done.


pkg/storage/replica_command.go, line 1329 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

We need to keep the implementation of PUSH_QUERY mode intact to avoid replica divergence (even with stop-the-world. We need freeze-cluster or propEvalKV to make changes like this safely)

PUSH_QUERY does not mutate any state, so it should be safe to remove this code.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the spencerkimball/high-contention branch 2 times, most recently from 4d650cd to 45ec826 Compare February 14, 2017 00:35
@bdarnell
Copy link
Copy Markdown
Contributor

Looks good aside from the tests that still need to be added.


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


pkg/storage/replica_command.go, line 1329 at r2 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

PUSH_QUERY does not mutate any state, so it should be safe to remove this code.

OK


Comments from Reviewable

@tamird
Copy link
Copy Markdown
Contributor

tamird commented Feb 14, 2017

Reviewed 22 of 35 files at r1, 11 of 22 files at r2, 11 of 12 files at r3.
Review status: all files reviewed at latest revision, 28 unresolved discussions, some commit checks failed.


pkg/kv/txn_correctness_test.go, line 380 at r3 (raw file):

// enumeratePriorities returns a slice enumerating all combinations of
// priorities across the transactions. The inner slice describes the

what transactions? this function does not take any transactions as arguments.


pkg/roachpb/api.proto, line 472 at r3 (raw file):

  // Query and return the latest transaction record if available. If no record
  // is persisted, returns a zero Pushee.
  PUSH_QUERY = 3;

should we also reserve this one?


pkg/roachpb/api.proto, line 520 at r3 (raw file):

  // pushed and only pushers with priority MaxTxnPriority may always
  // push a txn.
  optional bool new_priorities = 8 [(gogoproto.nullable) = false];

is this transitional? how is it intended to be used?


pkg/roachpb/batch_generated.go, line 1 at r2 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

I didn't edit this file. I edited gen_batch.go and ran go generate.

@spencerkimball gen_batch.go does not appear to have been modified.


pkg/roachpb/data_test.go, line 490 at r3 (raw file):

		10.0,
		100.0,
		999.0,

how come this changed?


pkg/roachpb/data_test.go, line 493 at r3 (raw file):

	}

	// Verify min & max.

nit: move this above the "table" above since it isn't related.


pkg/roachpb/errors.go, line 33 at r3 (raw file):

	TxnID   *uuid.UUID
	// TODO(spencer): Get rid of BACKOFF retries. Note that we don't propagate
	// the backoff hint to the client anyway. See #5249

Can you add "Fixes #5249." to the commit message? That issue was mistakenly left closed in May of last year, but I've now reopened it and assigned to you so that things can be linked properly.


pkg/roachpb/errors.proto, line 267 at r3 (raw file):

  // BACKOFF is for errors that can retried by restarting the transaction
  // after an exponential backoff.
  BACKOFF = 1;

don't we want to keep this to avoid reusing it?


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

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

why the named return?


pkg/storage/replica.go, line 1969 at r3 (raw file):

		pErr = r.checkIfTxnAborted(ctx, r.store.Engine(), *ba.Txn)
	}

nit: random newline in otherwise untouched area


pkg/storage/replica_command.go, line 1401 at r3 (raw file):

		pusherWins = false
	default:
		reason = "equal priorities; greater ID wins"

looks like we lost some fidelity here; perhaps canPushWithPriority should return the string as well.


pkg/storage/replica_command.go, line 1432 at r3 (raw file):

	if newPriorities {
		if (pusher.Priority > roachpb.MinTxnPriority && pushee.Priority == roachpb.MinTxnPriority) ||
			(pusher.Priority == roachpb.MaxTxnPriority && pushee.Priority < pusher.Priority) {

the comment in the proto is confusing in light of this code - you say that if true (newPriorities) then only minpriority can always be pushed, and only maxpriority can always push.

Why do we need to bother checking the other txn's priority?


pkg/storage/replica_command.go, line 1439 at r3 (raw file):

			return pusher.Priority > pushee.Priority
		} else if pusher.ID != nil &&
			bytes.Compare(pushee.ID.GetBytes(), pusher.ID.GetBytes()) < 0 {
if pusher.ID != nil {
  return bytes.Compare(...) < 0
}

pkg/storage/replica_command.go, line 1463 at r3 (raw file):

	}
	if !bytes.Equal(args.Key, args.Txn.Key) {
		return EvalResult{}, errors.Errorf("request key %s should match txn key %s", args.Key, args.Txn.Key)

s/should/does not/?


pkg/storage/replica_test.go, line 3058 at r3 (raw file):

		pusher := newTransaction("pusher", key, 1, enginepb.SERIALIZABLE, tc.Clock())
		pushee.Priority = 1
		pusher.Priority = 2 // pusher will win

restore this comment for consistency, you kept it everywhere else.


pkg/storage/store.go, line 2436 at r3 (raw file):

	// Add the command to the range for execution; exit retry loop on success.
	for {
		// Exit loop if context has been canceled or timed out.

Rather than selecting, I think you can just if err := ctx.Err(); err != nil { //exit the loop }

also, this code currently does not exit the loop if the context is cancelled.


pkg/storage/store.go, line 2500 at r3 (raw file):

			// updates to the unpushed txn's status.
			if s.cfg.DontRetryPushTxnFailures {
				// If we're not to retry on push txn failures (unittesting

can you explain in the comment why this doesn't propagate the original error?


pkg/storage/store.go, line 2548 at r3 (raw file):

					}
				}
				// We've resolved the write intent; retry command.

don't you need to set pErr = nil here?


pkg/storage/store.go, line 2557 at r3 (raw file):

		if pErr != nil {
			log.Eventf(ctx, "error: %T", pErr.GetDetail())

debugging code? you're only logging the type.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the spencerkimball/high-contention branch from 45ec826 to f61f143 Compare February 16, 2017 00:12
@spencerkimball
Copy link
Copy Markdown
Member Author

Tests added. PTAL


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


pkg/kv/txn_correctness_test.go, line 380 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

what transactions? this function does not take any transactions as arguments.

The comment is referring to the transactions involved in the history. You don't need to know anything specific about the transactions themselves, only how many there are (numTxns) in order to enumerate the possible combinations of priorities.


pkg/roachpb/api.proto, line 472 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

should we also reserve this one?

OK, will restore.


pkg/roachpb/api.proto, line 520 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

is this transitional? how is it intended to be used?

Not sure we can get rid of it if we always want to be able to upgrade from an older version, no matter how much older. We should probably sunset it at some point regardless. Not sure we have a policy for things like this.


pkg/roachpb/batch_generated.go, line 1 at r2 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

@spencerkimball gen_batch.go does not appear to have been modified.

Sorry, yes I didn't edit gen_batch.go I edited the api.proto which created a new roachpb.RequestUnion, which gen_batch.go uses to create batch_generated.go. My point is that I didn't edit this file. It was re-generated due to changes to api.proto.


pkg/roachpb/data_test.go, line 490 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

how come this changed?

Because specifying 1000 doesn't generate a random priority any longer. It just returns a deterministic max.


pkg/roachpb/data_test.go, line 493 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

nit: move this above the "table" above since it isn't related.

Done.


pkg/roachpb/errors.go, line 33 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

Can you add "Fixes #5249." to the commit message? That issue was mistakenly left closed in May of last year, but I've now reopened it and assigned to you so that things can be linked properly.

Done.


pkg/roachpb/errors.proto, line 267 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

don't we want to keep this to avoid reusing it?

OK, added a deprecated warning.


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

Previously, tamird (Tamir Duberstein) wrote…

why the named return?

Oops vestigial when we had a defer to intercept.


pkg/storage/replica.go, line 1969 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

nit: random newline in otherwise untouched area

Done.


pkg/storage/replica_command.go, line 1329 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

OK

Done.


pkg/storage/replica_command.go, line 1401 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

looks like we lost some fidelity here; perhaps canPushWithPriority should return the string as well.

That reason hasn't been useful yet. We still have the right info here. Easier this way.


pkg/storage/replica_command.go, line 1432 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

the comment in the proto is confusing in light of this code - you say that if true (newPriorities) then only minpriority can always be pushed, and only maxpriority can always push.

Why do we need to bother checking the other txn's priority?

minpriority can't push minpriority and maxpriority can't push maxpriority. I clarified api.proto a bit.


pkg/storage/replica_command.go, line 1439 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…
if pusher.ID != nil {
  return bytes.Compare(...) < 0
}

Done.


pkg/storage/replica_command.go, line 1463 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

s/should/does not/?

Done.


pkg/storage/replica_test.go, line 3058 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

restore this comment for consistency, you kept it everywhere else.

Done.


pkg/storage/store.go, line 2436 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

Rather than selecting, I think you can just if err := ctx.Err(); err != nil { //exit the loop }

also, this code currently does not exit the loop if the context is cancelled.

Oops on not exiting the loop. Done.


pkg/storage/store.go, line 2500 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

can you explain in the comment why this doesn't propagate the original error?

Done.


pkg/storage/store.go, line 2548 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

don't you need to set pErr = nil here?

Done.


pkg/storage/store.go, line 2557 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

debugging code? you're only logging the type.

Done.


Comments from Reviewable

@tamird
Copy link
Copy Markdown
Contributor

tamird commented Feb 16, 2017

Haven't reviewed tests yet; will do so tomorrow.


Reviewed 14 of 16 files at r4.
Review status: 41 of 43 files reviewed at latest revision, 16 unresolved discussions, some commit checks failed.


pkg/kv/txn_correctness_test.go, line 380 at r3 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

The comment is referring to the transactions involved in the history. You don't need to know anything specific about the transactions themselves, only how many there are (numTxns) in order to enumerate the possible combinations of priorities.

I see. I think it'd be clearer not to refer to "the transactions" here, but I won't insist.


pkg/roachpb/api.proto, line 520 at r3 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

Not sure we can get rid of it if we always want to be able to upgrade from an older version, no matter how much older. We should probably sunset it at some point regardless. Not sure we have a policy for things like this.

OK. Can you documented that nuance in the comment on this field (or wherever you see fit)?


pkg/storage/replica_command.go, line 1389 at r4 (raw file):

		pusherWins = true
	}
	log.Infof(ctx, "pusher wins: %t, reason: %s", pusherWins, reason)

debugging code?


pkg/util/retry/retry.go, line 122 at r4 (raw file):

		return true
	}

nit: another random whitespace change


pkg/util/retry/retry_test.go, line 118 at r4 (raw file):

	var attempts int

	// Create a retry loop which will never stop without stopper.

this comment doesn't make sense?


pkg/util/retry/retry_test.go, line 119 at r4 (raw file):

	// Create a retry loop which will never stop without stopper.
	for r := Start(opts); attempts < 2; attempts++ {

why can't this be for r := Start(opts); r.Next(); {? You've already set MaxRetries above.

You could eliminate attempts altogether, I think, and just use r.currentAttempt in the loop body.


Comments from Reviewable

@spencerkimball
Copy link
Copy Markdown
Member Author

Review status: 41 of 43 files reviewed at latest revision, 16 unresolved discussions, some commit checks failed.


pkg/roachpb/api.proto, line 520 at r3 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

OK. Can you documented that nuance in the comment on this field (or wherever you see fit)?

Done.


pkg/storage/replica_command.go, line 1389 at r4 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

debugging code?

Done.


pkg/util/retry/retry.go, line 122 at r4 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

nit: another random whitespace change

Done.


pkg/util/retry/retry_test.go, line 118 at r4 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

this comment doesn't make sense?

Removed.


pkg/util/retry/retry_test.go, line 119 at r4 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

why can't this be for r := Start(opts); r.Next(); {? You've already set MaxRetries above.

You could eliminate attempts altogether, I think, and just use r.currentAttempt in the loop body.

Because I'm checking the use of NextCh(), not Next(). I need attempts in order to verify that calling NextCh() advances r.currentAttempt by the correct amount.


Comments from Reviewable

@bdarnell
Copy link
Copy Markdown
Contributor

:lgtm:


Reviewed 16 of 16 files at r4.
Review status: all files reviewed at latest revision, 16 unresolved discussions, some commit checks failed.


pkg/storage/push_txn_queue_test.go, line 164 at r4 (raw file):

	ptq := tc.repl.pushTxnQueue
	ptq.mu.Lock()
	if ptq.mu.txns == nil {

Maybe add an isEnabled method so the test doesn't have to know this implementation detail.


pkg/storage/push_txn_queue_test.go, line 214 at r4 (raw file):

	if deps := ptq.GetDependents(*txn.ID); deps != nil {
		t.Errorf("expected GetDependents to return nil as queue is disabled")

Include the returned value in the error message.


pkg/storage/push_txn_queue_test.go, line 220 at r4 (raw file):

	ptq.mu.Lock()
	if ptq.mu.txns != nil {
		t.Errorf("expected enqueue to silently fail since queue is disabled")

Maybe Enqueue and UpdateTxn should have return values instead?


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the spencerkimball/high-contention branch from f61f143 to 15a57f8 Compare February 16, 2017 18:57
@tamird
Copy link
Copy Markdown
Contributor

tamird commented Feb 16, 2017

:lgtm:


Reviewed 2 of 16 files at r4, 6 of 6 files at r5.
Review status: all files reviewed at latest revision, 37 unresolved discussions, some commit checks pending.


pkg/roachpb/api.proto, line 520 at r3 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

Done.

heh, not quite the nuance i was hoping for.


pkg/storage/push_txn_queue.go, line 41 at r4 (raw file):

// pusher has max priority.
func shouldPushImmediately(req *roachpb.PushTxnRequest) bool {
	if !(req.PushType == roachpb.PUSH_ABORT || req.PushType == roachpb.PUSH_TIMESTAMP) {

shouldn't this check for TOUCH rather than checking for not ABORT and not TIMETSTAMP?


pkg/storage/push_txn_queue.go, line 55 at r4 (raw file):

// for transactions with pushed timestamps.
func isPushed(req *roachpb.PushTxnRequest, txn *roachpb.Transaction) bool {
	return txn != nil &&

is this nil check needed? isn't that undefined behaviour for this function?


pkg/storage/push_txn_queue.go, line 77 at r4 (raw file):

type waitingPush struct {
	req     *roachpb.PushTxnRequest
	pending chan *roachpb.Transaction // receives updated, pushed txn or nil

what does nil mean?


pkg/storage/push_txn_queue.go, line 87 at r4 (raw file):

// or more PushTxn requests.
type pendingTxn struct {
	txn     atomic.Value   // the most recent txn record, atomically updated

(optional) s/, atomically updated//


pkg/storage/push_txn_queue.go, line 88 at r4 (raw file):

type pendingTxn struct {
	txn     atomic.Value   // the most recent txn record, atomically updated
	waiters []*waitingPush // slice of waiting push txn requests

(optional) i'd remove this comment, it's exactly repeating the code


pkg/storage/push_txn_queue.go, line 91 at r4 (raw file):

}

// A pushTxnQueue manages a map of queues (keyed by txn ID) for txns

"map of queues" seems wrong? i'm not sure what this comment is saying.


pkg/storage/push_txn_queue.go, line 99 at r4 (raw file):

	store *Store
	mu    struct {
		syncutil.Mutex                           // Protects all variables in the mu struct

(optional) consider removing this comment


pkg/storage/push_txn_queue.go, line 128 at r4 (raw file):

func (ptq *pushTxnQueue) ClearAndDisable() {
	ptq.mu.Lock()
	if ptq.mu.txns == nil {

you can remove this special case without changing what this function does.


pkg/storage/push_txn_queue.go, line 252 at r4 (raw file):

	// already pushed, return push success.
	pending, ok := ptq.mu.txns[*req.PusheeTxn.ID]
	if !ok || (ok && isPushed(req, pending.txn.Load().(*roachpb.Transaction))) {

this checks ok 3 times. it might be clearer as:

if !ok {
  ptq.mu.Unlock()
  return nil, nil
}
if isPushed(....) {
  defer ptq.mu.Unlock()
  return createPush...
}

pkg/storage/push_txn_queue.go, line 269 at r4 (raw file):

	// Periodically refresh the pusher txn (with an exponential backoff),
	// in order to determine whether it's status or priority have changed,

nit: its


pkg/storage/push_txn_queue.go, line 275 at r4 (raw file):

	if req.PusherTxn.ID != nil {
		ptq.store.mu.Lock()
		retryOpts := ptq.store.cfg.RangeRetryOptions

does this need to be accessed under lock? my undersatnding is that the cfg struct is not modified


pkg/storage/push_txn_queue.go, line 285 at r4 (raw file):

		now := ptq.store.Clock().Now().WallTime
		// Get a channel to check for the pushee txn's expiration.
		expires := pending.txn.Load().(*roachpb.Transaction).LastActive().WallTime +

this appears to very nearly duplicate isExpired


pkg/storage/push_txn_queue.go, line 287 at r4 (raw file):

		expires := pending.txn.Load().(*roachpb.Transaction).LastActive().WallTime +
			2*base.DefaultHeartbeatInterval.Nanoseconds()
		pusheeTxnExpiresCh = time.After(time.Duration(expires - now))

does this need to be a util.Timer thing? might be leaking some timers here.


pkg/storage/push_txn_queue.go, line 401 at r4 (raw file):

	}
	br := b.RawResponse()
	// ID can be nil if no BeginTransaction has been sent yet.

this comment should be one line lower


pkg/storage/push_txn_queue_test.go, line 140 at r4 (raw file):

	if err := engine.MVCCPutProto(context.Background(), tc.store.Engine(),
		nil, key, hlc.ZeroTimestamp, nil, txn); err != nil {
		t.Fatal(err)

consider returning an error (it is otherwise difficult to locate failures).


pkg/storage/push_txn_queue_test.go, line 148 at r4 (raw file):

func createTxnForPushQueue(t *testing.T, tc *testContext) *roachpb.Transaction {
	txn := newTransaction("txn", roachpb.Key("a"), 1, enginepb.SERIALIZABLE, tc.Clock())
	writeTxnRecord(t, tc, txn)

ditto (don't pass t)


pkg/storage/push_txn_queue_test.go, line 182 at r4 (raw file):

		PusheeTxn: txn.TxnMeta,
	}
	respCh := make(chan *roachpb.PushTxnResponse, 1)

optional golf (applies throughout):

type respWithErr struct {
  resp *roachpb.PushTxnResponse
  pErr *roachpb.Error
}
retCh := make(chan respWithErr, 1)

and

retCh <- respWithErr{resp, pErr}

below


pkg/storage/push_txn_queue_test.go, line 187 at r4 (raw file):

		resp, pErr := ptq.MaybeWait(context.Background(), &req)
		respCh <- resp
		errCh <- pErr

consider closing after sending (throughout)


pkg/storage/push_txn_queue_test.go, line 273 at r4 (raw file):

	cancel()

	if pErr := <-errCh; !testutils.IsPError(pErr, "context canceled") {

nit: context. Canceled.Error()


pkg/storage/push_txn_queue_test.go, line 401 at r4 (raw file):

	var queryTxnCount int32

	clock := hlc.NewClock(hlc.UnixNano, 0)

prefer to avoid starting clocks at zero, we've had some fragile tests as a result in the past. I've been using 123 instead.


pkg/storage/push_txn_queue_test.go, line 565 at r4 (raw file):

	errCh := make(chan *roachpb.Error, 3)

	ctx, cancel := context.WithCancel(context.Background())

does this context need to be cancellable?


pkg/storage/replica_command.go, line 1367 at r4 (raw file):

	switch {
	case reply.PusheeTxn.LastActive().Less(args.Now.Add(-2*base.DefaultHeartbeatInterval.Nanoseconds(), 0)):

this now duplicates the logic in the function isExpired.


pkg/util/retry/retry_test.go, line 110 at r4 (raw file):

func TestRetryNextCh(t *testing.T) {
	opts := Options{
		InitialBackoff: time.Second,

seems like you could empty out this entire structure, all these values are red herrings


pkg/util/retry/retry_test.go, line 119 at r4 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

Because I'm checking the use of NextCh(), not Next(). I need attempts in order to verify that calling NextCh() advances r.currentAttempt by the correct amount.

Ah, I misunderstood this test. Thanks.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the spencerkimball/high-contention branch from f26830b to dd14808 Compare February 21, 2017 16:54
@tamird
Copy link
Copy Markdown
Contributor

tamird commented Feb 21, 2017

Reviewed 25 of 25 files at r7.
Review status: all files reviewed at latest revision, 20 unresolved discussions, some commit checks failed.


pkg/storage/push_txn_queue.go, line 91 at r4 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

Well they all queue up and then they are released in the same order that they were enqueued. The fact that they're all scheduled in their original goroutines is an implementation detail.

I don't understand your reply; I wasn't even thinking about that implementation detail, I'm just confused about the double use of "queue" in the first line of this comment - it's both in the name of this structure (which doesn't appear to be a queue itself) and in the description of its members (which are not true queues either). Am I crazy? Does this not seem confusing to you?


pkg/storage/push_txn_queue.go, line 99 at r4 (raw file):

Previously, spencerkimball (Spencer Kimball) wrote…

Done.

not done


pkg/storage/push_txn_queue.go, line 294 at r7 (raw file):

	for {
		// Set the timer to check for the pushee txn's expiration.
		pusheeTxnTimer.Reset(time.Duration(

instead of this cast, you can write

pusheeTxnTimer.Reset(
  txnExpiration(pending.txn.Load().(*roachpb.Transaction)).GoTime().Sub(
    ptq.store.Clock().Now().GoTime(),
  ),
)

...or whatever your preferred wrapped style.


Comments from Reviewable

@spencerkimball
Copy link
Copy Markdown
Member Author

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


pkg/storage/push_txn_queue.go, line 91 at r4 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

I don't understand your reply; I wasn't even thinking about that implementation detail, I'm just confused about the double use of "queue" in the first line of this comment - it's both in the name of this structure (which doesn't appear to be a queue itself) and in the description of its members (which are not true queues either). Am I crazy? Does this not seem confusing to you?

This struct enqueues PushTxn requests which are waiting on extant txns with conflicting intents to abort or commit. I didn't find it too confusing, but I've changed the comment.


pkg/storage/push_txn_queue.go, line 99 at r4 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

not done

Done.


pkg/storage/push_txn_queue.go, line 294 at r7 (raw file):

Previously, tamird (Tamir Duberstein) wrote…

instead of this cast, you can write

pusheeTxnTimer.Reset(
  txnExpiration(pending.txn.Load().(*roachpb.Transaction)).GoTime().Sub(
    ptq.store.Clock().Now().GoTime(),
  ),
)

...or whatever your preferred wrapped style.

I'm not a fan of this substitution.


Comments from Reviewable

@spencerkimball spencerkimball force-pushed the spencerkimball/high-contention branch from dd14808 to a71065b Compare February 22, 2017 20:45
@tamird
Copy link
Copy Markdown
Contributor

tamird commented Feb 22, 2017

Reviewed 2 of 2 files at r8.
Review status: all files reviewed at latest revision, 18 unresolved discussions, some commit checks pending.


Comments from Reviewable

@spencerkimball
Copy link
Copy Markdown
Member Author

Looks like this PR is ready to merge. @bdarnell, will it be safe to merge this tomorrow, and if so, at what time?

@bdarnell
Copy link
Copy Markdown
Contributor

We can merge it now - the commit for the beta has already been chosen. But we shouldn't deploy it to clusters other than cyan and cobalt until the beta is released.

@tamird
Copy link
Copy Markdown
Contributor

tamird commented Feb 22, 2017

@spencerkimball sorry, you'll need to rebase because of the Go 1.8 bump (and regenerate the protos with 1.8).

This commit replaces the abort-immediately-if-higher-priority txn semantics
with a more traditional locking approach. This yields significantly higher
performance on high-contention workloads.

Locking is done by disallowing intents to be pushed with two exceptions:
1) the pusher has a maximum txn priority (always pushes), or 2) the
pushee has a minimum txn priority (can always be pushed).

When a `PushTxn` request fails because the intent's txn is still live,
the pushee txn is added as a blocking txn to a new per-replica
`pushTxnQueue`. This queue maintains a list of waiting pushers per
blocking txn. Push failures are immediately retried and in the event
that the `PushTxn` request's `PusheeTxn` is in the queue, the pusher
request is enqueued behind it, waiting for it to be resolved.

While waiting on a txn to be resolved, a pusher periodically (with an
exponentially increasing backoff/retry) updates its own status. This
informs it whether it has had its priority ratcheted or been concurrently
aborted or committed. It also gathers information on which txns are in
turn waiting on the pusher in order to build a transitive closure of
txn dependencies. This is used to determine whether there is a
dependency cycle which would mean a deadlock. In the event of a deadlock,
txns with lower priorities are aborted.

Removed support for `PushType=PUSH_QUERY` from the `PushTxn` request
and added a new `QueryTxn` read-only command in its place which returns
both the transaction record as well as the list of txn IDs for txns
which are waiting on the queried txn.

Note that this PR will require a stop-the-world migration.

Fixes #5249
@spencerkimball spencerkimball force-pushed the spencerkimball/high-contention branch from a71065b to 1222d5d Compare February 23, 2017 16:07
@spencerkimball spencerkimball merged commit d553562 into master Feb 23, 2017
@spencerkimball spencerkimball deleted the spencerkimball/high-contention branch February 23, 2017 16:31
petermattis added a commit to petermattis/cockroach that referenced this pull request Feb 23, 2017
A bug was introduced by cockroachdb#13501. Need to track that down.
@dianasaur323 dianasaur323 mentioned this pull request Feb 24, 2017
9 tasks
tamird added a commit that referenced this pull request Feb 28, 2017
nvb added a commit to nvb/cockroach that referenced this pull request Jan 5, 2019
This was deprecated in cockroachdb#13501,
almost 2 years ago.

Release note: None
craig bot pushed a commit that referenced this pull request Jan 5, 2019
33480: roachpb: remove deprecated PUSH_QUERY PushTxnType r=nvanbenschoten a=nvanbenschoten

This was deprecated in #13501, almost 2 years ago.

Release note: None

Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants