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

kv: introduce setting for maximum lock wait-queue depth #66146

Merged

Conversation

nvanbenschoten
Copy link
Member

@nvanbenschoten nvanbenschoten commented Jun 7, 2021

Fixes #66017.
Related to #67513.

This PR introduces a new kv.lock_table.maximum_lock_wait_queue_length cluster setting, which controls the maximum length of a lock wait-queue that requests are willing to enter and wait in. The setting can be used to ensure some level of quality-of-service under severe per-key contention. If set to a non-zero value and an existing lock wait-queue is already equal to or exceeding this length, requests will be rejected eagerly instead of entering the queue and waiting.

Before this change, the lock-table's wait-queues had no limit on the number of writers that could be queued on a given key. This could lead to unbounded queueing and diminishing quality of service for all writers as the queues built up. It could also leave to starvation (i.e. zero throughput) when requests had a timeout that fires before any single request can get to the head of the queue. This was all especially bad with high replication latency in multi-region clusters, as locks are held for the duration of a consensus replication round.

We can see this in the following experiment. Here, we run a multi-region (demo) cluster in three configurations: default, timeout, and queue_limit. Under the default configuration, we change nothing. Under the timeout configuration, we set sql.defaults.statement_timeout='250ms'. Under the queue_limit configuration, we set kv.lock_table.maximum_lock_wait_queue_length=3. We then run a single-row, update-only workload (workload "U", mocked out in this patch):

./cockroach demo --global --empty --nodes=9
./cockroach sql -e 'create database ycsb primary region "us-east1" regions "us-west1", "europe-west1" survive region failure'
./cockroach workload init ycsb --families=false --insert-count=1
./cockroach workload run  ycsb --families=false --insert-count=1 --workload=U --duration=30s --tolerate-errors --concurrency=?

This results in the following behavior:

setting concurrency qps errors p50 (ms) p95 (ms)
default 1 13.7 0 67.1 71.3
default 2 12.6 0 142.6 151.0
default 4 12.3 0 302.0 385.9
default 8 12.2 0 570.4 1610.6
default 16 12.0 0 1208.0 2550.1
default 32 8.0 0 4563.4 5637.1
timeout 1 14.7 0 67.1 67.1
timeout 2 12.8 17 142.6 142.6
timeout 4 0.2 464 71.3 352.3
timeout 8 0.2 913 67.1 335.5
timeout 16 0 - - -
timeout 32 0 - - -
queue_limit 1 14.5 0 67.1 71.3
queue_limit 2 14.2 0 134.2 176.2
queue_limit 4 13.3 0 285.2 369.1
queue_limit 8 13.0 1934 352.3 486.5
queue_limit 16 12.8 4290 352.3 486.5
queue_limit 32 11.6 9203 385.9 671.1

The first thing to note is that under the default config, throughput remains relatively steady as concurrency grows, but latency grows linearly with concurrency. Next, note that under the timeout config, throughput falls to 0 the moment the p50 latency exceeds the statement_timeout. Finally, note that under the queue_limit config, errors begin to build once the queue limit is exceeded. However, throughput and latency hold steady as concurrency grows, as desired. So some requests are rejected, but the ones that are not are provided a good quality-of-service.

/cc @cockroachdb/kv @sumeerbhola

@cockroach-teamcity
Copy link
Member

This change is Reviewable

Copy link
Contributor

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

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

Before this change, the lock-table's wait-queues had no limit on the number of writers that could be queued on a given key. This could lead to unbounded queueing and diminishing quality of service for all writers as the queues built up. It could also leave to starvation (i.e. zero throughput) when requests had a timeout that fires before any single request can get to the head of the queue.

Can you explain the 0-throughput a bit more to me? I'd hope that requests timing out in the queue don't affect requests queued up after them because, somehow, the timed out request's ctx gets canceled and so the requests either gets out of the queue or otherwise "runs" quickly when it's its turn. So, I'd expect that some requests still would manage to get lucky and go through the queue within their timeout, stepping on the corpses of the timed out waiters in front of them.

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

@nvanbenschoten
Copy link
Member Author

nvanbenschoten commented Jun 7, 2021

Can you explain the 0-throughput a bit more to me? I'd hope that requests timing out in the queue don't affect requests queued up after them because, somehow, the timed out request's ctx gets canceled and so the requests either gets out of the queue or otherwise "runs" quickly when it's its turn. So, I'd expect that some requests still would manage to get lucky and go through the queue within their timeout, stepping on the corpses of the timed out waiters in front of them.

This is the hope, but it's not how these kinds of queuing problems with large processing times and equal timeouts behave in practice. Instead, they quickly reach an equilibrium where everyone reaches the head of the queue, but then gets canceled before they can finish processing. This is because the timeout doesn't just apply to queueing - it also applies to the processing itself.

We can derive this mathematically, assuming all requests have the same expected processing time (how long they need to process before completing successfully) and timeout.

queue_time = (concurrency - 1) * processing_time
queue_time + processing_time = timeout

queue_time = timeout - processing_time
timeout - processing_time = (concurrency - 1) * processing_time

processing_time = timeout/concurrency

Notice how the processing time becomes a function of the timeout and the concurrency. So if timeout/concurrency drops below the expected processing time, this means that everyone is getting canceled before they complete. That's why qps drops to 0 with a timeout of 250ms once concurrency hits 4 - 250ms/4 = 62.5 < 67.1.

Copy link
Contributor

@andreimatei andreimatei 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 @nvanbenschoten)


pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 514 at r1 (raw file):

		// The transaction cannot continue in this epoch whether this is
		// a retryable error or not.
		// WIP: we need some way to indicate that a request definitely did not leave

do you want a review before doing something here?

@nvanbenschoten
Copy link
Member Author

do you want a review before doing something here?

Not immediately. This is just a draft to prove out the idea and to solicit @sumeerbhola's feedback about how this relates to the hard rejections in the larger admission control project.

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 3 of 6 files at r1.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)


pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 516 at r1 (raw file):

		// WIP: we need some way to indicate that a request definitely did not leave
		// an intent. Otherwise, during hard rejects, we end up attempting to
		// resolve intents and disrupting traffic that was not rejected.

I didn't understand this comment. Is this talking about the some other request in the txn that had a rejected request, and that other request leaving an intent? If so, what is the issue with resolving that intent -- there is no latch contention given everyone is waiting on the intent after releasing latches. What am I missing?


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

		return nil
	},
)

This is fine, but would be worth having some commentary on more sophisticated alternatives like CoDel which take into account queueing time before rejecting https://queue.acm.org/detail.cfm?id=2209336, or the Facebook variant mentioned in https://queue.acm.org/detail.cfm?id=2839461
There is also the LIFO option, or adaptive-LIFO, but I think it does not work easily with the lock table with a request in multiple queues and reservations -- a newer request would potentially keep breaking some subset of locks.


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

// - The waitQueueMaxLengthExceeded state is used to indicate that the request
//   was rejected because it attempted to enter a lock wait-queue whose length
//   was already equal to or exceeding the configured maximum.

this is only for writers, yes?

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.

Thanks for the reviews on this PR so far! I've gone through, addressed comments, and polished off what was already here. I also added two new commits that ensure that we get the desired behavior under UPDATE contention.

The first commit here is pulled from #67514, so please ignore that. But otherwise, PTAL when you get a chance.

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


pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 514 at r1 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

do you want a review before doing something here?

No, I went back and reworked this. Please see the new first commit, titled "don't add intents from unambiguously failed requests to lock footprint".


pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 516 at r1 (raw file):

Previously, sumeerbhola wrote…

I didn't understand this comment. Is this talking about the some other request in the txn that had a rejected request, and that other request leaving an intent? If so, what is the issue with resolving that intent -- there is no latch contention given everyone is waiting on the intent after releasing latches. What am I missing?

The issue was that if we added the request to our lock spans, we would then try to resolve it in our EndTxn(abort) rollback batch. Typically, no-op intent resolution is fast because even though we grab latches, we bail before going through Raft because the Raft proposal is empty. However, in this case, the no-op intent resolution was still disruptive because we would hold latches, then delete our transaction record, then perform no-op intent resolution. Since we wrote to our txn record, the request still needed to go through Raft, so it held latches on the contended key across that entire duration.

I've fixed this in two different ways, either of which is enough to make the rest of this PR work correctly.

First, I've addressed this TODO by introducing a new commit here that builds upon #67514. This ensures that we don't count requests that hit lock wait-queue length limits in our lock tracking set, since we know we did not acquire a lock on that key.

I've also added a new commit that ensures that we only write tombstones to delete transaction records when the record previously existed. This can help make the EndTxn(abort) rollback batch in these cases much faster, as it allows them to skip Raft entirely.


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

Previously, sumeerbhola wrote…

This is fine, but would be worth having some commentary on more sophisticated alternatives like CoDel which take into account queueing time before rejecting https://queue.acm.org/detail.cfm?id=2209336, or the Facebook variant mentioned in https://queue.acm.org/detail.cfm?id=2839461
There is also the LIFO option, or adaptive-LIFO, but I think it does not work easily with the lock table with a request in multiple queues and reservations -- a newer request would potentially keep breaking some subset of locks.

Done. I also added a reference to #67513, which I think is a good medium-term project to improve these kinds of situations. The knob already exists in Postgres (so it can be viewed as a compatibility project) and it should address the pathological behavior described in #66146 (comment) by only setting a timeout on lock acquisition and not on an entire statement.


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

Previously, sumeerbhola wrote…

this is only for writers, yes?

Yes, commented.

@nvanbenschoten nvanbenschoten marked this pull request as ready for review July 13, 2021 06:17
Copy link
Contributor

@andreimatei andreimatei 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 @andreimatei, @nvanbenschoten, and @sumeerbhola)


pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 548 at r3 (raw file):

	// concurrent requests for extended periods of time. See #3346.
	if pErr != nil {
		// However, as an optimization, if the error indicates that a specific

nit: s/specific request/specific request (identified by pErr.Index)


pkg/kv/kvserver/concurrency/concurrency_control.go, line 375 at r5 (raw file):

	// to a non-zero value and an existing lock wait-queue is already equal
	// to or exceeding this length, the request will be rejected eagerly
	// instead of entering the queue and waiting.

consider specifying how the request is rejected - what error


pkg/kv/kvserver/concurrency/concurrency_manager.go, line 74 at r5 (raw file):

			return nil // disabled
		}
		// Don't let the setting be dropped below a reasonable value that we don't

why this min? What's wrong with setting it to 1? (i.e. a cluster-wide NOWAIT policy)
If there's no particular reason, my vote is to remove the min.


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

	// waitQueueMaxLengthExceeded indicates that the request attempted to enter a
	// lock wait-queue as a writer and found that the queue's length was already

where does the "as a writer" come from? Are we not rejecting readers just the same?

craig bot pushed a commit that referenced this pull request Jul 14, 2021
66795: sql: added missing tables from mysql information_schema r=rafiss a=mnovelodou

Previously, there was missing tables on information_schema present on
mysql
This was inadequate because it can cause compatibility problems
To address this, this patch adds missing information_schema tables

Release note (sql change): Added missing tables from mysql
information_schema. The tables are not populated and are
entirely empty.

- column_statistics
- columns_extensions
- engines
- events
- files
- keywords
- optimizer_trace
- partitions
- plugins
- processlist
- profiling
- resource_groups
- schemata_extensions
- st_geometry_columns
- st_spatial_reference_systems
- st_units_of_measure
- table_constraints_extensions
- tables_extensions
- tablespaces
- tablespaces_extensions
- user_attributes

67326: rowexec: remove unnecessary joiner member r=andreimatei a=andreimatei

The joiner had a field tracking a confusing lookup row index. Besides
being confusing, this member obscured the fact that it was equal to the
ordinal returned by inserting rows into a disk container - and thus that
the respective ordinal is used.

Release note: None

67514: sql,kv: permit txn rollbacks across LockNotAvailable errors r=nvanbenschoten a=nvanbenschoten

This commit adds support for rolling a transaction back across a `LockNotAvailable` (pgcode 55P03) error. `LockNotAvailable` errors are returned in two cases:
1. when a locking SELECT is run with a NOWAIT wait policy and conflicts with an active lock
2. when a statement is run with a `lock_timeout` and this timeout is exceeded (unsupported, see #67513)

The following test case from `pkg/sql/testdata/savepoints` demonstrates this new capability:

```
# txn1
BEGIN
INSERT INTO t VALUES (1)
----
1: BEGIN -- 0 rows
-- NoTxn       -> Open        #.  (none)
2: INSERT INTO t VALUES (1) -- 1 row
-- Open        -> Open        ##  (none)

# txn2
BEGIN
SAVEPOINT foo
SELECT * FROM t WHERE x = 1 FOR UPDATE NOWAIT
ROLLBACK TO SAVEPOINT foo
SELECT * FROM t WHERE x = 2 FOR UPDATE NOWAIT
COMMIT
----
1: BEGIN -- 0 rows
-- NoTxn       -> Open        #.....  (none)
2: SAVEPOINT foo -- 0 rows
-- Open        -> Open        ##....  foo
3: SELECT * FROM t WHERE x = 1 FOR UPDATE NOWAIT -- pq: could not obtain lock on row (x)=(1) in t@primary
-- Open        -> Aborted     XXXXXX  foo
4: ROLLBACK TO SAVEPOINT foo -- 0 rows
-- Aborted     -> Open        ##....  foo
5: SELECT * FROM t WHERE x = 2 FOR UPDATE NOWAIT -- 0 rows
-- Open        -> Open        ##..#.  foo
6: COMMIT -- 0 rows
-- Open        -> NoTxn       ##..##  (none)
```

This becomes the second error type that supports rollbacks, with the first being duplicate key errors, which was added in 65e8045.

The primary motivation for this PR was to be able to give `WriteIntentErrors` an `ErrorPriority` of `ErrorScoreUnambiguousError` for #66146. However, the added functionality fell out of making that change.

Release note (sql change): ROLLBACK TO SAVEPOINT can now be used to recover from LockNotAvailable errors (pgcode 55P03), which are returned when performing a FOR UPDATE SELECT with a NOWAIT wait policy.

67524: limit,storage: add more trace spans to backup path r=dt a=adityamaru

This change adds a trace recording to track how
many requests are waiting in the the concurrent limiter
queue.

The change also adds a child span to ExportMVCCToSst
to track how long the scan+SST creation is taking per
ExportRequest during backup.

Release note: None

67544: catalogkv: fix panic inside Txn swallowing retries r=postamar a=ajwerner

`(*kv.DB).Txn` may produce retry errors. While it's fine that this test
function can panic in the face of a real error, it's not okay to panic
the retry error.

Fixes #67213.

Release note: None

67550: authors: add xinhaoz to AUTHORS r=xinhaoz a=xinhaoz

Release note: None

67557: parser: reduce imports for scan.go r=otan a=rafiss

See individual commits:
- move keywords.go and tokens.go to lexbase 
- remove scan.go dependency on tree NewNumVal/NewStrVal

Touches #64710

Co-authored-by: MiguelNovelo <[email protected]>
Co-authored-by: Andrei Matei <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
Co-authored-by: Aditya Maru <[email protected]>
Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: Xin Hao Zhang <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/lockTableQueueDepth branch from 0786e26 to c14c9a3 Compare July 14, 2021 17:47
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 @andreimatei and @sumeerbhola)


pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go, line 548 at r3 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

nit: s/specific request/specific request (identified by pErr.Index)

Done.


pkg/kv/kvserver/concurrency/concurrency_control.go, line 375 at r5 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

consider specifying how the request is rejected - what error

Done.


pkg/kv/kvserver/concurrency/concurrency_manager.go, line 74 at r5 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

why this min? What's wrong with setting it to 1? (i.e. a cluster-wide NOWAIT policy)
If there's no particular reason, my vote is to remove the min.

It seems dangerous to me for this to be able to impact internal transaction processing, so I don't want users to risk shooting themselves in the foot. I think the long-term preferred approach is for this setting to be set to something high like 128 and then for specific operations to opt-in to an aggressive lock_timeout setting.


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

Previously, andreimatei (Andrei Matei) wrote…

where does the "as a writer" come from? Are we not rejecting readers just the same?

No, we are not. Readers don't wait in lock wait-queues in the same way that writers do. See the distinction between lockWaitQueue.queuedWriters and lockWaitQueue.waitingReaders.

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 2 files at r3, 4 of 24 files at r6, 2 of 14 files at r7, 9 of 21 files at r8.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @andreimatei, @nvanbenschoten, and @sumeerbhola)


pkg/kv/kvserver/concurrency/concurrency_manager.go, line 74 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

It seems dangerous to me for this to be able to impact internal transaction processing, so I don't want users to risk shooting themselves in the foot. I think the long-term preferred approach is for this setting to be set to something high like 128 and then for specific operations to opt-in to an aggressive lock_timeout setting.

Is there a plan to run some experiments (can be after this PR merges) with highly contended workloads and see the effect of adjusting this cluster setting?


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

			} else if g.maxWaitQueueLength > 0 && curLen >= g.maxWaitQueueLength {
				// The wait-queue is longer than the request is willing to wait for.
				// Instead of entering the queue, immediately reject the request.

can you add to this comment that for simplicity we are not finding the position of this writer in the queue and rejecting the tail of the queue above the max length. That would be more fair, but more complicated, and we expect that the common case is that this waiter will be at the end of the queue.


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

			}
			g.mu.locks[l] = struct{}{}
			waitForState.queuedWriters = l.queuedWriters.Len()

so we're not setting queuedReaders for a writer and not setting queuedWriters for a reader?
Can you update the comment where these fields are declared.

Copy link
Contributor

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

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

:LGTM

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


pkg/kv/kvserver/concurrency/concurrency_manager.go, line 74 at r5 (raw file):

It seems dangerous to me for this to be able to impact internal transaction processing

OK.


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

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

No, we are not. Readers don't wait in lock wait-queues in the same way that writers do. See the distinction between lockWaitQueue.queuedWriters and lockWaitQueue.waitingReaders.

consider hinting at the difference between reads and writes in the comment on the cluster setting.

…print

Typically, the `txnPipeliner` adds all lock acquisitions attempts
directly to the lock footprint when a batch request returns an error.

This commit adds an optimization where, if the error indicates that a
specific request unambiguously did not acquire any locks, we ignore that
request for the purposes of accounting for lock spans. This is important
for transactions that only perform a single request and hit an
unambiguous error like a ConditionFailedError, as it can allow them to
avoid sending a rollback. It it also important for transactions that
throw a WriteIntentError due to heavy contention on a certain key after
either passing a Error wait policy or hitting a lock timeout / queue
depth limit. In such cases, this optimization prevents these
transactions from adding even more load to the contended key by trying
to perform unnecessary intent resolution.
Before this commit, EndTxn requests that hit the auto-GC path, did not resolve
any intents, and also did not find an existing transaction record would still
write a tombstone over their transaction record. This would force them to go
through Raft and miss the fast-path to skip consensus added in 6fcb3db. This
commit ensures that a tombstone is not written over a non-existing transaction
record, which allows this case to avoid Raft altogether.
Fixes cockroachdb#66017.

This PR introduces a new `kv.lock_table.maximum_lock_wait_queue_length`
cluster setting, which controls the maximum length of a lock wait-queue
that requests are willing to enter and wait in. The setting can be used
to ensure some level of quality-of-service under severe per-key
contention. If set to a non-zero value and an existing lock wait-queue
is already equal to or exceeding this length, requests will be rejected
eagerly instead of entering the queue and waiting.

Before this change, the lock-table's wait-queues had no limit on the
number of writers that could be queued on a given key. This could lead
to unbounded queueing and diminishing quality of service for all writers
as the queues built up. It could also leave to starvation (i.e. zero
throughput) when requests had a timeout that fires before any single
request can get to the head of the queue. This was all especially bad
with high replication latency in multi-region clusters, as locks are
held for the duration of a consensus replication round.

We can see this in the following experiment. Here, we run a multi-region
(demo) cluster in three configurations: `default`, `timeout`, and
`queue_limit`. Under the `default` configuration, we change nothing.
Under the `timeout` configuration, we set `sql.defaults.statement_timeout='250ms'`.
Under the `queue_limit` configuration, we set `kv.lock_table.maximum_lock_wait_queue_length=3`.
We then run a single-row, update-only workload (workload "U", mocked
out in this patch):
```
./cockroach demo --global --empty --nodes=9
./cockroach sql -e 'create database ycsb primary region "us-east1" regions "us-west1", "europe-west1" survive region failure'
./cockroach workload init ycsb --families=false --insert-count=1
./cockroach workload run  ycsb --families=false --insert-count=1 --workload=U --duration=30s --tolerate-errors --concurrency=?
```

This results in the following behavior:

| setting     | concurrency | qps  | errors | p50 (ms) | p95 (ms) |
|-------------|-------------|------|--------|----------|----------|
| default     | 1           | 13.7 | 0      | 67.1     | 71.3     |
| default     | 2           | 12.6 | 0      | 142.6    | 151.0    |
| default     | 4           | 12.3 | 0      | 302.0    | 385.9    |
| default     | 8           | 12.2 | 0      | 570.4    | 1610.6   |
| default     | 16          | 12.0 | 0      | 1208.0   | 2550.1   |
| default     | 32          | 8.0  | 0      | 4563.4   | 5637.1   |
| timeout     | 1           | 14.7 | 0      | 67.1     | 67.1     |
| timeout     | 2           | 12.8 | 17     | 142.6    | 142.6    |
| timeout     | 4           | 0.2  | 464    | 71.3     | 352.3    |
| timeout     | 8           | 0.2  | 913    | 67.1     | 335.5    |
| timeout     | 16          | 0    | -      | -        | -        |
| timeout     | 32          | 0    | -      | -        | -        |
| queue_limit | 1           | 14.5 | 0      | 67.1     | 71.3     |
| queue_limit | 2           | 14.2 | 0      | 134.2    | 176.2    |
| queue_limit | 4           | 13.3 | 0      | 285.2    | 369.1    |
| queue_limit | 8           | 13.0 | 1934   | 352.3    | 486.5    |
| queue_limit | 16          | 12.8 | 4290   | 352.3    | 486.5    |
| queue_limit | 32          | 11.6 | 9203   | 385.9    | 671.1    |

The first thing to note is that under the `default` config,
throughput remains relatively steady as concurrency grows, but latency
grows linearly with concurrency. Next, note that under the `timeout`
config, throughput falls to 0 the moment the p50 latency exceeds the
`statement_timeout`. Finally, note that under the `queue_limit` config,
errors begin to build once the queue limit is exceeded. However,
throughput and latency hold steady as concurrency grows, as desired. So
some requests are rejected, but the ones that are not are provided a
good quality-of-service.
@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/lockTableQueueDepth branch from c14c9a3 to 368e589 Compare July 22, 2021 23:48
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.

TFTRs!

bors r+

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


pkg/kv/kvserver/concurrency/concurrency_manager.go, line 74 at r5 (raw file):

Is there a plan to run some experiments (can be after this PR merges) with highly contended workloads and see the effect of adjusting this cluster setting?

What kinds of experiments are you thinking? There are some experimental results included in the PR description, but I imagine you're thinking of more realistic benchmarks on real machines.


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

Previously, andreimatei (Andrei Matei) wrote…

consider hinting at the difference between reads and writes in the comment on the cluster setting.

Done.


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

Previously, sumeerbhola wrote…

can you add to this comment that for simplicity we are not finding the position of this writer in the queue and rejecting the tail of the queue above the max length. That would be more fair, but more complicated, and we expect that the common case is that this waiter will be at the end of the queue.

Done.


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

Previously, sumeerbhola wrote…

so we're not setting queuedReaders for a writer and not setting queuedWriters for a reader?
Can you update the comment where these fields are declared.

We are. We set the initial values of this up above. This is updating the value to reflect the new size. I added comments to make that clearer.

@craig
Copy link
Contributor

craig bot commented Jul 23, 2021

Build succeeded:

@craig craig bot merged commit 2ca4acb into cockroachdb:master Jul 23, 2021
@nvanbenschoten nvanbenschoten deleted the nvanbenschoten/lockTableQueueDepth branch July 27, 2021 20:49
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

kv: introduce cluster setting for maximum lock wait-queue depth
4 participants