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/concurrency: implement lockTableWaiter #44885

Merged

Conversation

nvanbenschoten
Copy link
Member

This commit builds upon the concurrency control interfaces introduced in #44787 (and is rebased on those 3 commits). It implements the counter-part to the lockTable structure, the lockTableWaiter.

lockTableWaiter is concerned with waiting in lock wait-queues for locks held by conflicting transactions. It ensures that waiting requests continue to make forward progress even in the presence of faulty transaction coordinators and transaction deadlocks.

The waiter implements logic for a request to wait on conflicting locks in the lockTable until they are released. Similarly, it implements logic to wait on conflicting requests ahead of the caller's request in any lock wait-queues that it is a part of.

This waiting state responds to a set of state transitions in the lock table:

  • a conflicting lock is released
  • a conflicting lock is updated such that it no longer conflicts
  • a conflicting request in the lock wait-queue acquires the lock
  • a conflicting request in the lock wait-queue exits the lock wait-queue

These state transitions are typically reactive - the waiter can simply wait for locks to be released or lock wait-queues to be exited by other actors. Reacting to state transitions for conflicting locks is powered by the lockManager and reacting to state transitions for conflicting lock wait-queues is powered by the requestSequencer interface.

However, in the case of transaction coordinator failures or transaction deadlocks, a state transition may never occur without intervention from the waiter. To ensure forward-progress, the waiter may need to actively push either a lock holder of a conflicting lock or the head of a conflicting lock wait-queue. This active pushing requires an RPC to the leaseholder of the conflicting transaction's record, and will typically result in the RPC queuing in that leaseholder's txnWaitQueue. Because this can be expensive, the push is not immediately performed. Instead, it is only performed after a delay.

The semantics around how the lockTableWaiter interfaces with the intentresolver are guided by the current approach in Replica.handleWriteIntentError. Once we properly integrate the concurrency.Manager and remove calls into Replica.handleWriteIntentError, we can clean up the intentresolver's interface to be more easy to use. This will also let us delete the contentionQueue entirely.

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/lockTableWaiter branch 2 times, most recently from 2d4b93a to 44f453a Compare February 11, 2020 04:04
@nvanbenschoten
Copy link
Member Author

This is now rebased on master and ready for a review.

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

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


pkg/storage/concurrency/lock_table_waiter.go, line 34 at r2 (raw file):

type lockTableWaiterImpl struct {
	nodeID  roachpb.NodeID
	stopper *stop.Stopper

what is being stopped by this Stopper? And who will call Stop() on it?


pkg/storage/concurrency/lock_table_waiter.go, line 51 at r2 (raw file):

	PushTransaction(
		context.Context, *enginepb.TxnMeta, roachpb.Header, roachpb.PushTxnType,
	) (roachpb.Transaction, *Error)

Does this return an error if the transaction is not pushed?


pkg/storage/concurrency/lock_table_waiter.go, line 66 at r2 (raw file):

	var timer *timeutil.Timer
	var timerC <-chan time.Time
	var timerState waitingState

nit: timerState suggests it is the state of the timer, but this is the state of the request in the lockTable.


pkg/storage/concurrency/lock_table_waiter.go, line 96 at r2 (raw file):

				if hasExtremePriority(req.Txn) {
					// However, transactions that have the maximum or minimum
					// priorities should push immediately.

Why should a min priority transaction push immediately? Wouldn't it be better to do something like: if hasMaxPriority(req.Txn) || hasMinPriority(state.txn) ...


pkg/storage/concurrency/lock_table_waiter.go, line 103 at r2 (raw file):

					defer timer.Stop()
				}
				timer.Reset(delay)

since the timer is only reset in waitFor state this admits the possibility of under-timing if there are a series of state transitions e.g. waitFor (starts timer) => waitForDistinguished => waitFor (timer fires). It may not be worth fixing, but is worthy of a code comment.


pkg/storage/concurrency/lock_table_waiter.go, line 122 at r2 (raw file):

				// this state.
				if err := w.pushTxn(ctx, req, state); err != nil {
					return err

Is the error from pushTxn() represent a failure to push? I am guessing not, since a failed push should result in continuing to wait instead of returning.


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

				// not safe to proceed with evaluation because there is a still a
				// transaction holding the lock (TODO or a request holding the
				// reservation?). It should push the transaction it is blocked on

Regarding this TODO, the waitElsewhere state currently does not change the "who it is waiting for", for requests that were previously in waitFor or waitForDistinguished state. So it does include waiting for reservation holders. If we only want to wait for lock holders, or even only replicated lock holders (since all other sequencing information is being discarded), it would be an easy change.


pkg/storage/concurrency/lock_table_waiter.go, line 134 at r2 (raw file):

				if err := w.pushTxn(ctx, req, state); err != nil {
					return err
				}

After waitElsewhere there are not going to be any more state changes, so it needs to call scanAndEnqueue() before any more waiting on the lockTableGuard.


pkg/storage/concurrency/lock_table_waiter.go, line 140 at r2 (raw file):

				// for. This can only happen when the request's transaction is
				// sending multiple requests concurrently. Proceed with looking for
				// new locks to wait for.

nit: the "Proceed with looking ..." is not quite accurate since it will continue waiting at this lock.


pkg/storage/concurrency/testdata/lock_table, line 533 at r2 (raw file):

# TODO DURING REVIEW: why didn't this switch over to held=true? It doesn't look
# like we're updating the waiters in acquireLock and informing them that the lock
# is now held. We'll need to do that.

Yes, the code does not update the state for waiters when the reservation holder acquires the lock.


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

		opts := ResolveOptions{Wait: true, Poison: true}
		if pErr := ir.ResolveIntents(ctx, resolveIntents, opts); pErr != nil {
			return 0, errors.Wrapf(pErr.GoError(), "failed to resolve intents")

what is the motivation for these small error handling changes in this file?

@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/lockTableWaiter branch from 6578b1f to 23203e8 Compare February 11, 2020 22:57
Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

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


pkg/storage/concurrency/lock_table_waiter.go, line 34 at r2 (raw file):

Previously, sumeerbhola wrote…

what is being stopped by this Stopper? And who will call Stop() on it?

This will be provided when the concurrency manager is created. The lifetime is set to the lifetime of the entire server process. Stop() is called in any of the shutdown code-paths. This is the standard approach to draining requests in the kv-layer upon server shutdown.


pkg/storage/concurrency/lock_table_waiter.go, line 51 at r2 (raw file):

Previously, sumeerbhola wrote…

Does this return an error if the transaction is not pushed?

It depends on what you mean by "not pushed". This will typically block until the push succeeds or until the pushee is finalized. In the case of context cancellation or some other failure condition, this will return an error.


pkg/storage/concurrency/lock_table_waiter.go, line 66 at r2 (raw file):

Previously, sumeerbhola wrote…

nit: timerState suggests it is the state of the timer, but this is the state of the request in the lockTable.

Renamed to timerWaitingState. I wanted it prefixed with timer so that it was clear that it was manipulated at the same time as the timer and timerC objects.


pkg/storage/concurrency/lock_table_waiter.go, line 96 at r2 (raw file):

Previously, sumeerbhola wrote…

Why should a min priority transaction push immediately? Wouldn't it be better to do something like: if hasMaxPriority(req.Txn) || hasMinPriority(state.txn) ...

Done.


pkg/storage/concurrency/lock_table_waiter.go, line 103 at r2 (raw file):

Previously, sumeerbhola wrote…

since the timer is only reset in waitFor state this admits the possibility of under-timing if there are a series of state transitions e.g. waitFor (starts timer) => waitForDistinguished => waitFor (timer fires). It may not be worth fixing, but is worthy of a code comment.

I'm not sure I understand the hazard you're referencing. We set timerC to nil on all state transitions, so the only way for the timer fire to be observed is if it fires immediately after a waitFor state, which resets the timer to the delay and drains timerC as a side-effect.


pkg/storage/concurrency/lock_table_waiter.go, line 122 at r2 (raw file):

Previously, sumeerbhola wrote…

Is the error from pushTxn() represent a failure to push? I am guessing not, since a failed push should result in continuing to wait instead of returning.

There really should never be a failure to push other than in error cases. All other cases will just block. I added to the comment on PushTransaction to indicate this.


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

If we only want to wait for lock holders, or even only replicated lock holders (since all other sequencing information is being discarded), it would be an easy change.

I think that would be a good change to make. It doesn't really make sense to push the old reservation holder if the reservation has been abandoned and no new requests will respect it. I think it's safer to just let everyone race to acquire latches and fall back to pushing.

That said, I'd like to double-check that this degraded state will actually result in durable lock holders being pushed. Right now we pass all WriteIntentErrors through the lockTable when we stumble upon them in the MVCC keyspace (HandleWriterIntentError). What will happen when the lockTable is full and has just dumped its locks. Will it start adding these discovered locks back in?


pkg/storage/concurrency/lock_table_waiter.go, line 134 at r2 (raw file):

Previously, sumeerbhola wrote…

After waitElsewhere there are not going to be any more state changes, so it needs to call scanAndEnqueue() before any more waiting on the lockTableGuard.

Good point, done. Should the current lockTableGuard still be passed in to scanAndEnqueue? Do we need to introduce some notion of a lockTable epoch to detect guards from old epochs?


pkg/storage/concurrency/lock_table_waiter.go, line 140 at r2 (raw file):

Previously, sumeerbhola wrote…

nit: the "Proceed with looking ..." is not quite accurate since it will continue waiting at this lock.

Really? Doesn't that create the possibility of deadlocks because a transaction will be blocked on itself without pushing anyone? I thought the idea was that once a txn acquires a lock, all of its requests are free to ignore the lock and proceed to evaluation.


pkg/storage/concurrency/testdata/lock_table, line 533 at r2 (raw file):

Previously, sumeerbhola wrote…

Yes, the code does not update the state for waiters when the reservation holder acquires the lock.

I was looking into how to address this and I realized I don't really understand why the state transitions in the lock table are written the way they are. I'm hoping you can help me to understand. Across each of the lockTable's methods (AcquireLock, UpdateLocks, AddDiscoveredLock, and Dequeue), we seem to break the effect of lockState state transitions into two parts.

First, we update the waitingState of waiters who must remain waiting after the state transition. This is done in two different places: informActiveWaiters (called by AcquireLock and AddDiscoveredLock) and lockIsFree (called by UpdateLocks and Dequeue). I don't understand why we need to reimplement this logic in lockIsFree instead of deferring toinformActiveWaiters in all four cases. If the idea that lockIsFree is able to enforce a larger number of preconditions on the lockState and is therefore able to be more specialized and efficient?

Second, we return a doneWaiting []*lockTableGuardImpl slice up to the top of each method and call processDoneWaiting with this slice. It's not clear to me why we don't inline the processDoneWaiting guard notifications to where we're constructing these slices and avoid the slice allocations entirely. I had thought this had to do with lock ordering, but now I'm not sure.


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

Previously, sumeerbhola wrote…

what is the motivation for these small error handling changes in this file?

So that we can expose a unified interface as intentResolver instead of having two different error types. In general, we should be keeping errors as *roachpb.Errors if they have come from an RPC.

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

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


pkg/storage/concurrency/lock_table_waiter.go, line 51 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

It depends on what you mean by "not pushed". This will typically block until the push succeeds or until the pushee is finalized. In the case of context cancellation or some other failure condition, this will return an error.

Ah yes, I remember now from an earlier conversation.


pkg/storage/concurrency/lock_table_waiter.go, line 103 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

I'm not sure I understand the hazard you're referencing. We set timerC to nil on all state transitions, so the only way for the timer fire to be observed is if it fires immediately after a waitFor state, which resets the timer to the delay and drains timerC as a side-effect.

I misread the code.


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

I think that would be a good change to make. It doesn't really make sense to push the old reservation holder if the reservation has been abandoned and no new requests will respect it. I think it's safer to just let everyone race to acquire latches and fall back to pushing.

I'll add it to the TODO list.

That said, I'd like to double-check that this degraded state will actually result in durable lock holders being pushed. Right now we pass all WriteIntentErrors through the lockTable when we stumble upon them in the MVCC keyspace (HandleWriterIntentError). What will happen when the lockTable is full and has just dumped its locks. Will it start adding these discovered locks back in?

Yes. And it does not dump locks that don't have a distinguished waiter. clearMostLocks() has the following comment (the comment is slightly incorrect -- it should say distinguished waiter since that is the correct thing to do and what the code does):

// Removes all locks that have active waiters and tells those waiters to wait
// elsewhere. A replicated lock which has been discovered by a request but the
// request is not yet actively waiting on it will be preserved since we need
// to tell that request who it is waiting for when it next calls
// ScanAndEnqueue(). If we aggressively removed even those requests, the next
// ScanAndEnqueue() would not find that lock, the request would evaluate
// again, again discover that lock and if clearMostLocks() keeps getting
// called would be stuck in this loop without pushing.

So the lockTable can exceed the maximum count and once the lock has a distinguished waiter it could be cleared. The idea was that by then the waiter would be calling pushTxn() which would take care of pushing and deadlock detection. But this may not do exactly what we want:

  • the scanAndEnqueue() which will cause the request to become a distinguished waiter, and the later call to CurState() to observe that state, are not atomic. So a clear could slip in in-between and cause the state to be changed to waitElsewhere. That is ok since waitElsewhere also causes a push.
  • when the push returns and the state that was previously observed was waitDistinguished, the current state may be waitElsewhere because the lock has been cleared. But the push has already succeeded, and the lockTable is providing stale information to the request (in the case where the lockTable has not cleared state, when the push resulted in updateLocks(), the next state observed by the request would not be stale), which would potentially cause the request to engage in an unnecessary push. To eliminate this possibility -- the lockTable would need to keep track of what state the request had observed and maybe introduce a state called retry in addition to the current waitElsewhere state. Do you think the current behavior is ok?

pkg/storage/concurrency/lock_table_waiter.go, line 134 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Good point, done. Should the current lockTableGuard still be passed in to scanAndEnqueue? Do we need to introduce some notion of a lockTable epoch to detect guards from old epochs?

Yes, the current lockTableGuard should continue to be used. It is possible that it still is in the queue for some lockStates as a non-active waiter and the original sequence number will still be used for fairness.


pkg/storage/concurrency/lock_table_waiter.go, line 140 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Really? Doesn't that create the possibility of deadlocks because a transaction will be blocked on itself without pushing anyone? I thought the idea was that once a txn acquires a lock, all of its requests are free to ignore the lock and proceed to evaluation.

waitSelf occurs only when the reservation holder is a different request from the same transaction. If the same transaction already holds the lock (or acquires the lock while this request is waiting) there is no waiting at this lock (or will stop waiting at this lock).
There could be other actively waiting requests in the same queue from different transactions that will see waitForDistinguished and waitFor and would take care of deadlock detection.


pkg/storage/concurrency/testdata/lock_table, line 533 at r2 (raw file):

I was looking into how to address this and I realized I don't really understand why the state transitions in the lock table are written the way they are. I'm hoping you can help me to understand. Across each of the lockTable's methods (AcquireLock, UpdateLocks, AddDiscoveredLock, and Dequeue), we seem to break the effect of lockState state transitions into two parts.
First, we update the waitingState of waiters who must remain waiting after the state transition. This is done in two different places: informActiveWaiters (called by AcquireLock and AddDiscoveredLock) and lockIsFree (called by UpdateLocks and Dequeue). I don't understand why we need to reimplement this logic in lockIsFree instead of deferring toinformActiveWaiters in all four cases. If the idea that lockIsFree is able to enforce a larger number of preconditions on the lockState and is therefore able to be more specialized and efficient?

That was the original motivation. As we've added more "features" the story has become more muddled. I was going to look at refactoring when adding the "non deduped spans" and "waitElsewhere should only wait for replicated lock holders" changes. If you prefer, you can add a TODO for me to also make this "held consistency" fix.

Second, we return a doneWaiting []*lockTableGuardImpl slice up to the top of each method and call processDoneWaiting with this slice. It's not clear to me why we don't inline the processDoneWaiting guard notifications to where we're constructing these slices and avoid the slice allocations entirely. I had thought this had to do with lock ordering, but now I'm not sure.

It was from before the deferred tryActiveWait() code. There is a TODO in the code, but I now inclined not to wait for the switch to the different btree and just do it now.

// TODO(sbhola): the deferred call to tryActiveWait() removes the need to
// return these doneWaiting slices -- the callee can notify the channel
// itself. Simplify this when switching to the copy-on-write btree since that
// will cause some related code restructuring.

@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/lockTableWaiter branch from 23203e8 to 1845867 Compare February 12, 2020 19:09
Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

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


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

I'll add it to the TODO list.

Thanks.

Yes...

Thanks for the explanation. I'm happy to see that we already thought through this.

Do you think the current behavior is ok?

Hitting the lockTable memory limits should be very rare, so I think this behavior is fine.


pkg/storage/concurrency/lock_table_waiter.go, line 134 at r2 (raw file):

Previously, sumeerbhola wrote…

Yes, the current lockTableGuard should continue to be used. It is possible that it still is in the queue for some lockStates as a non-active waiter and the original sequence number will still be used for fairness.

Got it, thanks.


pkg/storage/concurrency/lock_table_waiter.go, line 140 at r2 (raw file):

Previously, sumeerbhola wrote…

waitSelf occurs only when the reservation holder is a different request from the same transaction. If the same transaction already holds the lock (or acquires the lock while this request is waiting) there is no waiting at this lock (or will stop waiting at this lock).
There could be other actively waiting requests in the same queue from different transactions that will see waitForDistinguished and waitFor and would take care of deadlock detection.

Oh, you're right. I was misremembering what the state indicated. Done.


pkg/storage/concurrency/testdata/lock_table, line 533 at r2 (raw file):

If you prefer, you can add a TODO for me to also make this "held consistency" fix.

SGTM, fixing this at the same time as a refactor sounds like the right approach, as it might influence the way we structure things.

It was from before the deferred tryActiveWait() code. There is a TODO in the code, but I now inclined not to wait for the switch to the different btree and just do it now.

Ack. Should we add this TODO to #44976?

Copy link
Collaborator

@sumeerbhola sumeerbhola left a comment

Choose a reason for hiding this comment

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

:lgtm:

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


pkg/storage/concurrency/lock_table_waiter_test.go, line 43 at r4 (raw file):

	_ context.Context, intents []roachpb.Intent, _ intentresolver.ResolveOptions,
) *Error {
	return m.resolveIntents(intents[0])

the caller in lockTableWaiterImpl.pushTxn is always calling with 1 intent and this implementation also only looks at 1 intent. Should this be called ResolveIntent() and not use a slice parameter?


pkg/storage/concurrency/lock_table_waiter_test.go, line 251 at r4 (raw file):

	t.Run("state", func(t *testing.T) {
		// Used for waitFor, waitForDistinguished, and waitElsewhere.
		testWaitPush := func(t *testing.T, k stateKind) {

is this testWaitPush the same as the one defined in the previous test?


pkg/storage/concurrency/lock_table_waiter_test.go, line 323 at r4 (raw file):

			// Set up an observer channel to detect when the current
			// waiting state is observed.
			g.state = waitingState{stateKind: waitSelf}

is there a reason to use waitSelf here? That state won't actually occur for non-transactional requests, as you note below.


pkg/storage/concurrency/testdata/lock_table, line 533 at r2 (raw file):

Should we add this TODO to #44976?

Added

This will be useful to provide a consistent interface for the next commit.
This commit builds upon the concurrency control interfaces introduced in cockroachdb#44787
(and is rebased on those 3 commits). It implements the counter-part to the
`lockTable` structure, the `lockTableWaiter`.

lockTableWaiter is concerned with waiting in lock wait-queues for locks held
by conflicting transactions. It ensures that waiting requests continue to
make forward progress even in the presence of faulty transaction coordinators
and transaction deadlocks.

The waiter implements logic for a request to wait on conflicting locks in the
lockTable until they are released. Similarly, it implements logic to wait on
conflicting requests ahead of the caller's request in any lock wait-queues
that it is a part of.

This waiting state responds to a set of state transitions in the lock table:
 - a conflicting lock is released
 - a conflicting lock is updated such that it no longer conflicts
 - a conflicting request in the lock wait-queue acquires the lock
 - a conflicting request in the lock wait-queue exits the lock wait-queue

These state transitions are typically reactive - the waiter can simply wait
for locks to be released or lock wait-queues to be exited by other actors.
Reacting to state transitions for conflicting locks is powered by the
lockManager and reacting to state transitions for conflicting lock
wait-queues is powered by the requestSequencer interface.

However, in the case of transaction coordinator failures or transaction
deadlocks, a state transition may never occur without intervention from the
waiter. To ensure forward-progress, the waiter may need to actively push
either a lock holder of a conflicting lock or the head of a conflicting lock
wait-queue. This active pushing requires an RPC to the leaseholder of the
conflicting transaction's record, and will typically result in the RPC
queuing in that leaseholder's txnWaitQueue. Because this can be expensive,
the push is not immediately performed. Instead, it is only performed after a
delay.

The semantics around how the lockTableWaiter interfaces with the intentresolver
are guided by the current approach in `Replica.handleWriteIntentError`. Once
we properly integrate the `concurrency.Manager` and remove calls into
`Replica.handleWriteIntentError`, we can clean up the intentresolver's
interface to be more easy to use. This will also let us delete the
`contentionQueue` entirely.
@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/lockTableWaiter branch from 1845867 to ce8901d Compare February 12, 2020 21:52
Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

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

TFTR!

bors r+

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


pkg/storage/concurrency/lock_table_waiter_test.go, line 43 at r4 (raw file):

Previously, sumeerbhola wrote…

the caller in lockTableWaiterImpl.pushTxn is always calling with 1 intent and this implementation also only looks at 1 intent. Should this be called ResolveIntent() and not use a slice parameter?

The idea was to avoid modifying the existing *intentresolver.IntentResolver in this change, but this is easy enough. Done.


pkg/storage/concurrency/lock_table_waiter_test.go, line 251 at r4 (raw file):

Previously, sumeerbhola wrote…

is this testWaitPush the same as the one defined in the previous test?

Not exactly, but it was close enough to unify. Done.


pkg/storage/concurrency/lock_table_waiter_test.go, line 323 at r4 (raw file):

Previously, sumeerbhola wrote…

is there a reason to use waitSelf here? That state won't actually occur for non-transactional requests, as you note below.

No, this was meant to be waitFor. Thanks for catching.

craig bot pushed a commit that referenced this pull request Feb 12, 2020
44885: storage/concurrency: implement lockTableWaiter r=nvanbenschoten a=nvanbenschoten

This commit builds upon the concurrency control interfaces introduced in #44787 (and is rebased on those 3 commits). It implements the counter-part to the `lockTable` structure, the `lockTableWaiter`.

lockTableWaiter is concerned with waiting in lock wait-queues for locks held by conflicting transactions. It ensures that waiting requests continue to make forward progress even in the presence of faulty transaction coordinators and transaction deadlocks.

The waiter implements logic for a request to wait on conflicting locks in the lockTable until they are released. Similarly, it implements logic to wait on conflicting requests ahead of the caller's request in any lock wait-queues that it is a part of.

This waiting state responds to a set of state transitions in the lock table:
 - a conflicting lock is released
 - a conflicting lock is updated such that it no longer conflicts
 - a conflicting request in the lock wait-queue acquires the lock
 - a conflicting request in the lock wait-queue exits the lock wait-queue

These state transitions are typically reactive - the waiter can simply wait for locks to be released or lock wait-queues to be exited by other actors. Reacting to state transitions for conflicting locks is powered by the lockManager and reacting to state transitions for conflicting lock wait-queues is powered by the requestSequencer interface.

However, in the case of transaction coordinator failures or transaction deadlocks, a state transition may never occur without intervention from the waiter. To ensure forward-progress, the waiter may need to actively push either a lock holder of a conflicting lock or the head of a conflicting lock wait-queue. This active pushing requires an RPC to the leaseholder of the conflicting transaction's record, and will typically result in the RPC queuing in that leaseholder's txnWaitQueue. Because this can be expensive, the push is not immediately performed. Instead, it is only performed after a delay.

The semantics around how the lockTableWaiter interfaces with the intentresolver are guided by the current approach in `Replica.handleWriteIntentError`. Once we properly integrate the `concurrency.Manager` and remove calls into `Replica.handleWriteIntentError`, we can clean up the intentresolver's interface to be more easy to use. This will also let us delete the `contentionQueue` entirely.

Co-authored-by: Nathan VanBenschoten <nvanbenschoten@gmail.com>
sumeerbhola added a commit to sumeerbhola/cockroach that referenced this pull request Feb 12, 2020
when lock becomes free.

This reduces code duplication. Also added a TODO where the
!held => held transition happens, to change after cockroachdb#44885
merges.

Release note: None
@craig
Copy link
Contributor

craig bot commented Feb 12, 2020

Build succeeded

@craig craig bot merged commit ce8901d into cockroachdb:master Feb 12, 2020
@nvanbenschoten nvanbenschoten deleted the nvanbenschoten/lockTableWaiter branch February 12, 2020 22:45
sumeerbhola added a commit to sumeerbhola/cockroach that referenced this pull request Feb 13, 2020
when lock becomes free.

This reduces code duplication. Also added a TODO where the
!held => held transition happens, to change after cockroachdb#44885
merges.

Release note: None
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.

3 participants