Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
98261: sql: add crdb_internal views for system statistics tables r=ericharmeling a=ericharmeling

This commit adds two new crdb_internal views:

- crdb_internal.statement_statistics_persisted, which surfaces the system.statement_statistics table
- crdb_internal.transaction_statistics_persisted, which surfaces the system.transaction_statistics table

Example output from after flush:

```
root@127.0.0.1:26257/insights> select * from crdb_internal.statement_statistics_persisted limit 3;
-[ RECORD 1 ]
aggregated_ts              | 2023-03-08 23:00:00+00
fingerprint_id             | \x3ab7869b0bc4aa5a
transaction_fingerprint_id | \x95d43bd78dc51d85
plan_hash                  | \x9aec25074eb1e3a0
app_name                   | $ cockroach sql
node_id                    | 1
agg_interval               | 01:00:00
metadata                   | {"db": "insights", "distsql": true, "failed": false, "fullScan": true, "implicitTxn": true, "query": "SELECT * FROM crdb_internal.statement_statistics_persisted", "querySummary": "SELECT * FROM crdb_internal.statement_statis...", "stmtTyp": "TypeDML", "vec": true}
statistics                 | {"execution_statistics": {"cnt": 1, "contentionTime": {"mean": 0, "sqDiff": 0}, "cpuSQLNanos": {"mean": 24667, "sqDiff": 0}, "maxDiskUsage": {"mean": 0, "sqDiff": 0}, "maxMemUsage": {"mean": 2.048E+4, "sqDiff": 0}, "mvccIteratorStats": {"blockBytes": {"mean": 0, "sqDiff": 0}, "blockBytesInCache": {"mean": 0, "sqDiff": 0}, "keyBytes": {"mean": 0, "sqDiff": 0}, "pointCount": {"mean": 0, "sqDiff": 0}, "pointsCoveredByRangeTombstones": {"mean": 0, "sqDiff": 0}, "rangeKeyContainedPoints": {"mean": 0, "sqDiff": 0}, "rangeKeyCount": {"mean": 0, "sqDiff": 0}, "rangeKeySkippedPoints": {"mean": 0, "sqDiff": 0}, "seekCount": {"mean": 1, "sqDiff": 0}, "seekCountInternal": {"mean": 1, "sqDiff": 0}, "stepCount": {"mean": 0, "sqDiff": 0}, "stepCountInternal": {"mean": 0, "sqDiff": 0}, "valueBytes": {"mean": 0, "sqDiff": 0}}, "networkBytes": {"mean": 0, "sqDiff": 0}, "networkMsgs": {"mean": 0, "sqDiff": 0}}, "index_recommendations": [], "statistics": {"bytesRead": {"mean": 0, "sqDiff": 0}, "cnt": 1, "firstAttemptCnt": 1, "idleLat": {"mean": 0, "sqDiff": 0}, "indexes": ["42@1"], "lastErrorCode": "", "lastExecAt": "2023-03-08T23:14:04.614242Z", "latencyInfo": {"max": 0.001212208, "min": 0.001212208, "p50": 0, "p90": 0, "p99": 0}, "maxRetries": 0, "nodes": [1], "numRows": {"mean": 0, "sqDiff": 0}, "ovhLat": {"mean": 0.000007791999999999955, "sqDiff": 0}, "parseLat": {"mean": 0.000016666, "sqDiff": 0}, "planGists": ["AgFUAgD/FwAAAAcYBhg="], "planLat": {"mean": 0.000691666, "sqDiff": 0}, "regions": ["us-east1"], "rowsRead": {"mean": 0, "sqDiff": 0}, "rowsWritten": {"mean": 0, "sqDiff": 0}, "runLat": {"mean": 0.000496084, "sqDiff": 0}, "svcLat": {"mean": 0.001212208, "sqDiff": 0}}}
plan                       | {"Children": [], "Name": ""}
index_recommendations      | {}
indexes_usage              | ["42@1"]
-[ RECORD 2 ]
aggregated_ts              | 2023-03-08 23:00:00+00
fingerprint_id             | \x44c9fdb49be676cf
transaction_fingerprint_id | \xc1efcc0bba0909f8
plan_hash                  | \x780a1ba35976b15d
app_name                   | insights
node_id                    | 1
agg_interval               | 01:00:00
metadata                   | {"db": "insights", "distsql": false, "failed": false, "fullScan": false, "implicitTxn": false, "query": "UPDATE insights_workload_table_0 SET balance = balance + $1 WHERE id = $1", "querySummary": "UPDATE insights_workload_table_0 SET balance = balan... WHERE id = $1", "stmtTyp": "TypeDML", "vec": true}
statistics                 | {"execution_statistics": {"cnt": 28, "contentionTime": {"mean": 0, "sqDiff": 0}, "cpuSQLNanos": {"mean": 402538.75, "sqDiff": 1160598792985.25}, "maxDiskUsage": {"mean": 0, "sqDiff": 0}, "maxMemUsage": {"mean": 4.096E+4, "sqDiff": 0}, "mvccIteratorStats": {"blockBytes": {"mean": 31570.321428571428, "sqDiff": 20932497128.107143}, "blockBytesInCache": {"mean": 0, "sqDiff": 0}, "keyBytes": {"mean": 0, "sqDiff": 0}, "pointCount": {"mean": 6.857142857142857, "sqDiff": 435.42857142857133}, "pointsCoveredByRangeTombstones": {"mean": 0, "sqDiff": 0}, "rangeKeyContainedPoints": {"mean": 0, "sqDiff": 0}, "rangeKeyCount": {"mean": 0, "sqDiff": 0}, "rangeKeySkippedPoints": {"mean": 0, "sqDiff": 0}, "seekCount": {"mean": 2, "sqDiff": 0}, "seekCountInternal": {"mean": 2, "sqDiff": 0}, "stepCount": {"mean": 0, "sqDiff": 0}, "stepCountInternal": {"mean": 4.857142857142857, "sqDiff": 435.42857142857133}, "valueBytes": {"mean": 360.32142857142856, "sqDiff": 756476.107142857}}, "networkBytes": {"mean": 0, "sqDiff": 0}, "networkMsgs": {"mean": 0, "sqDiff": 0}}, "index_recommendations": [], "statistics": {"bytesRead": {"mean": 159.04887361588396, "sqDiff": 3909.7441771668127}, "cnt": 2619, "firstAttemptCnt": 2619, "idleLat": {"mean": 0.021495726165330273, "sqDiff": 36.39774900003032}, "indexes": ["106@1"], "lastErrorCode": "", "lastExecAt": "2023-03-08T23:31:03.079093Z", "latencyInfo": {"max": 1.724660916, "min": 0.0001765, "p50": 0.000757916, "p90": 0.00191375, "p99": 0.004730417}, "maxRetries": 0, "nodes": [1], "numRows": {"mean": 1, "sqDiff": 0}, "ovhLat": {"mean": 0.0000018584035891561339, "sqDiff": 3.132932109484058E-7}, "parseLat": {"mean": 0, "sqDiff": 0}, "planGists": ["AgHUAQIADwIAAAcKBQoh1AEAAA=="], "planLat": {"mean": 0.0002562748900343638, "sqDiff": 0.0002118085353898781}, "regions": ["us-east1"], "rowsRead": {"mean": 1, "sqDiff": 0}, "rowsWritten": {"mean": 1, "sqDiff": 0}, "runLat": {"mean": 0.0024048477613592997, "sqDiff": 4.850230671161608}, "svcLat": {"mean": 0.0026629810549828195, "sqDiff": 4.852464499918359}}}
plan                       | {"Children": [], "Name": ""}
index_recommendations      | {}
indexes_usage              | ["106@1"]
-[ RECORD 3 ]
aggregated_ts              | 2023-03-08 23:00:00+00
fingerprint_id             | \x54202c7b75a5ba87
transaction_fingerprint_id | \x0000000000000000
plan_hash                  | \xbee0e52ec8c08bdd
app_name                   | $$ $ cockroach demo
node_id                    | 1
agg_interval               | 01:00:00
metadata                   | {"db": "insights", "distsql": false, "failed": false, "fullScan": false, "implicitTxn": false, "query": "INSERT INTO system.jobs(id, created, status, payload, progress, claim_session_id, claim_instance_id, job_type) VALUES ($1, $1, __more1_10__)", "querySummary": "INSERT INTO system.jobs(id, created, st...)", "stmtTyp": "TypeDML", "vec": true}
statistics                 | {"execution_statistics": {"cnt": 1, "contentionTime": {"mean": 0, "sqDiff": 0}, "cpuSQLNanos": {"mean": 300625, "sqDiff": 0}, "maxDiskUsage": {"mean": 0, "sqDiff": 0}, "maxMemUsage": {"mean": 1.024E+4, "sqDiff": 0}, "mvccIteratorStats": {"blockBytes": {"mean": 0, "sqDiff": 0}, "blockBytesInCache": {"mean": 0, "sqDiff": 0}, "keyBytes": {"mean": 0, "sqDiff": 0}, "pointCount": {"mean": 0, "sqDiff": 0}, "pointsCoveredByRangeTombstones": {"mean": 0, "sqDiff": 0}, "rangeKeyContainedPoints": {"mean": 0, "sqDiff": 0}, "rangeKeyCount": {"mean": 0, "sqDiff": 0}, "rangeKeySkippedPoints": {"mean": 0, "sqDiff": 0}, "seekCount": {"mean": 0, "sqDiff": 0}, "seekCountInternal": {"mean": 0, "sqDiff": 0}, "stepCount": {"mean": 0, "sqDiff": 0}, "stepCountInternal": {"mean": 0, "sqDiff": 0}, "valueBytes": {"mean": 0, "sqDiff": 0}}, "networkBytes": {"mean": 0, "sqDiff": 0}, "networkMsgs": {"mean": 0, "sqDiff": 0}}, "index_recommendations": [], "statistics": {"bytesRead": {"mean": 0, "sqDiff": 0}, "cnt": 1, "firstAttemptCnt": 1, "idleLat": {"mean": 9223372036.854776, "sqDiff": 0}, "indexes": [], "lastErrorCode": "", "lastExecAt": "2023-03-08T23:13:25.132671Z", "latencyInfo": {"max": 0.000589375, "min": 0.000589375, "p50": 0, "p90": 0, "p99": 0}, "maxRetries": 0, "nodes": [1], "numRows": {"mean": 1, "sqDiff": 0}, "ovhLat": {"mean": 0.0000016249999999999988, "sqDiff": 0}, "parseLat": {"mean": 0, "sqDiff": 0}, "planGists": ["AiACHgA="], "planLat": {"mean": 0.000203792, "sqDiff": 0}, "regions": ["us-east1"], "rowsRead": {"mean": 0, "sqDiff": 0}, "rowsWritten": {"mean": 1, "sqDiff": 0}, "runLat": {"mean": 0.000383958, "sqDiff": 0}, "svcLat": {"mean": 0.000589375, "sqDiff": 0}}}
plan                       | {"Children": [], "Name": ""}
index_recommendations      | {}
indexes_usage              | []

Time: 4ms total (execution 3ms / network 1ms)

root@127.0.0.1:26257/insights> select * from crdb_internal.transaction_statistics_persisted limit 3;
-[ RECORD 1 ]
aggregated_ts  | 2023-03-08 23:00:00+00
fingerprint_id | \x17d80cf128571d63
app_name       | $ internal-migration-job-mark-job-succeeded
node_id        | 1
agg_interval   | 01:00:00
metadata       | {"stmtFingerprintIDs": ["b8bbb1bdae56aabc"]}
statistics     | {"execution_statistics": {"cnt": 1, "contentionTime": {"mean": 0, "sqDiff": 0}, "cpuSQLNanos": {"mean": 64709, "sqDiff": 0}, "maxDiskUsage": {"mean": 0, "sqDiff": 0}, "maxMemUsage": {"mean": 1.024E+4, "sqDiff": 0}, "mvccIteratorStats": {"blockBytes": {"mean": 0, "sqDiff": 0}, "blockBytesInCache": {"mean": 0, "sqDiff": 0}, "keyBytes": {"mean": 0, "sqDiff": 0}, "pointCount": {"mean": 0, "sqDiff": 0}, "pointsCoveredByRangeTombstones": {"mean": 0, "sqDiff": 0}, "rangeKeyContainedPoints": {"mean": 0, "sqDiff": 0}, "rangeKeyCount": {"mean": 0, "sqDiff": 0}, "rangeKeySkippedPoints": {"mean": 0, "sqDiff": 0}, "seekCount": {"mean": 0, "sqDiff": 0}, "seekCountInternal": {"mean": 0, "sqDiff": 0}, "stepCount": {"mean": 0, "sqDiff": 0}, "stepCountInternal": {"mean": 0, "sqDiff": 0}, "valueBytes": {"mean": 0, "sqDiff": 0}}, "networkBytes": {"mean": 0, "sqDiff": 0}, "networkMsgs": {"mean": 0, "sqDiff": 0}}, "statistics": {"bytesRead": {"mean": 0, "sqDiff": 0}, "cnt": 6, "commitLat": {"mean": 0, "sqDiff": 0}, "idleLat": {"mean": 0, "sqDiff": 0}, "maxRetries": 0, "numRows": {"mean": 1, "sqDiff": 0}, "retryLat": {"mean": 0, "sqDiff": 0}, "rowsRead": {"mean": 0, "sqDiff": 0}, "rowsWritten": {"mean": 1, "sqDiff": 0}, "svcLat": {"mean": 0.00026919450000000006, "sqDiff": 1.7615729685500012E-8}}}
-[ RECORD 2 ]
aggregated_ts  | 2023-03-08 23:00:00+00
fingerprint_id | \x2b024f7e2567a238
app_name       | $ internal-get-job-row
node_id        | 1
agg_interval   | 01:00:00
metadata       | {"stmtFingerprintIDs": ["8461f232a36615e7"]}
statistics     | {"execution_statistics": {"cnt": 1, "contentionTime": {"mean": 0, "sqDiff": 0}, "cpuSQLNanos": {"mean": 50835, "sqDiff": 0}, "maxDiskUsage": {"mean": 0, "sqDiff": 0}, "maxMemUsage": {"mean": 3.072E+4, "sqDiff": 0}, "mvccIteratorStats": {"blockBytes": {"mean": 0, "sqDiff": 0}, "blockBytesInCache": {"mean": 0, "sqDiff": 0}, "keyBytes": {"mean": 0, "sqDiff": 0}, "pointCount": {"mean": 3, "sqDiff": 0}, "pointsCoveredByRangeTombstones": {"mean": 0, "sqDiff": 0}, "rangeKeyContainedPoints": {"mean": 0, "sqDiff": 0}, "rangeKeyCount": {"mean": 0, "sqDiff": 0}, "rangeKeySkippedPoints": {"mean": 0, "sqDiff": 0}, "seekCount": {"mean": 1, "sqDiff": 0}, "seekCountInternal": {"mean": 1, "sqDiff": 0}, "stepCount": {"mean": 3, "sqDiff": 0}, "stepCountInternal": {"mean": 3, "sqDiff": 0}, "valueBytes": {"mean": 186, "sqDiff": 0}}, "networkBytes": {"mean": 0, "sqDiff": 0}, "networkMsgs": {"mean": 0, "sqDiff": 0}}, "statistics": {"bytesRead": {"mean": 284.81818181818176, "sqDiff": 3465.636363636355}, "cnt": 11, "commitLat": {"mean": 0.000003469727272727273, "sqDiff": 4.946789218181818E-11}, "idleLat": {"mean": 0, "sqDiff": 0}, "maxRetries": 0, "numRows": {"mean": 1, "sqDiff": 0}, "retryLat": {"mean": 0, "sqDiff": 0}, "rowsRead": {"mean": 1, "sqDiff": 0}, "rowsWritten": {"mean": 0, "sqDiff": 0}, "svcLat": {"mean": 0.0006771060909090909, "sqDiff": 8.91510436082909E-7}}}
-[ RECORD 3 ]
aggregated_ts  | 2023-03-08 23:00:00+00
fingerprint_id | \x37e130a1df20d299
app_name       | $ internal-create-stats
node_id        | 1
agg_interval   | 01:00:00
metadata       | {"stmtFingerprintIDs": ["98828ded59216546"]}
statistics     | {"execution_statistics": {"cnt": 1, "contentionTime": {"mean": 0, "sqDiff": 0}, "cpuSQLNanos": {"mean": 11875, "sqDiff": 0}, "maxDiskUsage": {"mean": 0, "sqDiff": 0}, "maxMemUsage": {"mean": 1.024E+4, "sqDiff": 0}, "mvccIteratorStats": {"blockBytes": {"mean": 0, "sqDiff": 0}, "blockBytesInCache": {"mean": 0, "sqDiff": 0}, "keyBytes": {"mean": 0, "sqDiff": 0}, "pointCount": {"mean": 0, "sqDiff": 0}, "pointsCoveredByRangeTombstones": {"mean": 0, "sqDiff": 0}, "rangeKeyContainedPoints": {"mean": 0, "sqDiff": 0}, "rangeKeyCount": {"mean": 0, "sqDiff": 0}, "rangeKeySkippedPoints": {"mean": 0, "sqDiff": 0}, "seekCount": {"mean": 0, "sqDiff": 0}, "seekCountInternal": {"mean": 0, "sqDiff": 0}, "stepCount": {"mean": 0, "sqDiff": 0}, "stepCountInternal": {"mean": 0, "sqDiff": 0}, "valueBytes": {"mean": 0, "sqDiff": 0}}, "networkBytes": {"mean": 0, "sqDiff": 0}, "networkMsgs": {"mean": 0, "sqDiff": 0}}, "statistics": {"bytesRead": {"mean": 0, "sqDiff": 0}, "cnt": 1, "commitLat": {"mean": 0.000002291, "sqDiff": 0}, "idleLat": {"mean": 0, "sqDiff": 0}, "maxRetries": 0, "numRows": {"mean": 0, "sqDiff": 0}, "retryLat": {"mean": 0, "sqDiff": 0}, "rowsRead": {"mean": 0, "sqDiff": 0}, "rowsWritten": {"mean": 0, "sqDiff": 0}, "svcLat": {"mean": 0.008680208, "sqDiff": 0}}}

Time: 3ms total (execution 2ms / network 1ms)
```

Epic: none

Release note: Added two views to the crdb_internal catalog: crdb_internal.statement_statistics_persisted, which surfaces data in the persisted system.statement_statistics table, and crdb_internal.transaction_statistics_persisted, which surfaces the system.transaction_statistics table.

98422: kvserver: disable {split,replicate,mvccGC} queues until... r=irfansharif a=irfansharif

...subscribed to span configs. Do the same for the store rebalancer. We applied this treatment for the merge queue back in #78122 since the fallback behavior, if not subscribed, is to use the statically defined span config for every operation.

- For the replicate queue this mean obtusely applying a replication factor of 3, regardless of configuration. This was possible typically post node restart before subscription was initially established. We saw this in #98385. It was possible then for us to ignore configured voter/non-voter/lease constraints.
- For the split queue, we wouldn't actually compute any split keys if unsubscribed, so the missing check was somewhat benign. But we would be obtusely applying the default range sizes [128MiB,512MiB], so for clusters configured with larger range sizes, this could lead to a burst of splitting post node-restart.
- For the MVCC GC queue, it would mean applying the the statically configured default GC TTL and ignoring any set protected timestamps. The latter is best-effort protection but could result in internal operations relying on protection (like backups, changefeeds) failing informatively. For clusters configured with GC TTL greater than the default, post node-restart it could lead to a burst of MVCC GC activity and AOST queries failing to find expected data.
- For the store rebalancer, ignoring span configs could result in violating lease preferences and voter constraints.

Fixes #98421.
Fixes #98385.

Release note (bug fix): It was previously possible for CockroachDB to not respect non-default zone configs. This only happened for a short window after nodes with existing replicas were restarted, and self-rectified within seconds. This manifested in a few ways:
- If num_replicas was set to something other than 3, we would still add or remove replicas to get to 3x replication.
  - If num_voters was set explicitly to get a mix of voting and non-voting replicas, it would be ignored. CockroachDB could possibly remove non-voting replicas.
- If range_min_bytes or range_max_bytes were changed from 128 MiB and 512 MiB respectively, we would instead try to size ranges to be within [128 MiB, 512MiB]. This could appear as an excess amount of range splits or merges, as visible in the Replication Dashboard under "Range Operations".
- If gc.ttlseconds was set to something other than 90000 seconds, we would still GC data only older than 90000s/25h. If the GC TTL was set to something larger than 25h, AOST queries going further back may now start failing. For GC TTLs less than the 25h default, clusters would observe increased disk usage due to more retained garbage.
- If constraints, lease_preferences or voter_constraints were set, they would be ignored. Range data and leases would possibly be moved outside where prescribed. This issues only lasted a few seconds post node-restarts, and any zone config violations were rectified shortly after.

98468: sql: add closest-instance physical planning r=dt a=dt

This changes physical planning, specifically how the SQL instance for a
given KV node ID is resolved, to be more generalized w.r.t. different
locality tier taxonomies.

Previously this function had a special case that checked for, and only
for, a specific locality tier with the key "region" and if it was
found, picked a random instance from the subset of instances where their
value for that matched the value for the KV node.

Matching on and only on the "region" tier is both too specific and not
specific enough: it is "too specific" in that it requires a tier with
the key "region" to be used and to match, and is "not specific enough"
in that it simultaneously ignores more specific locality tiers that
would indicate closer matches (e.g. subregion, AZ, data-center or rack).

Instead, this change generalizes this function to identify the subset of
instances that have the "closest match" in localities to the KV node and
pick one of them, where closest match is defined as the longest matching
prefix of locality tiers. In a simple, single-tier locality taxonomy
using the key "region" this should yield the same behavior as the
previous implementation, as all instances with a matching "region" will
have the same longest matching prefix (at length 1), however this more
general approach should better handle other locality taxonomies that use
more tiers and/or tiers with names other than "region".

Currently this change only applies to physical planning for secondary
tenants until physical planning is unified for system and secondary
tenants.

Release note: none.
Epic: CRDB-16910


98471: changefeedccl: fix kafka messagetoolarge test failure r=samiskin a=samiskin

Fixes: #93847

This fixes the following bug in the TestChangefeedKafkaMessageTooLarge test setup:
1. The feed starts sending messages, randomly triggering a MessageTooLarge error causing a retry with a smaller batch size
2. Eventually, while the retrying process is still ongoing, all 2000 rows are successfully received by the mock kafka sink, causing assertPayloads to complete, causing the test to closeFeed and run CANCEL on the changefeed.
3. The retrying process gets stuck in sendMessage where it can't send the message to the feedCh which has been closed since the changefeed is trying to close, but it also can't exit on the mock sink's tg.done since that only closes after the feed fully closes, which requires the retrying process to end.

Release note: None

Co-authored-by: Eric Harmeling <eric.harmeling@cockroachlabs.com>
Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
Co-authored-by: David Taylor <tinystatemachine@gmail.com>
Co-authored-by: Shiranka Miskin <shiranka.miskin@gmail.com>
  • Loading branch information
5 people committed Mar 13, 2023
5 parents 36d39aa + 4557dd0 + 234bcd0 + e61d65a + f5902c0 commit c31c1ac
Show file tree
Hide file tree
Showing 40 changed files with 1,322 additions and 932 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/changefeedccl/testfeed_test.go
Expand Up @@ -1717,6 +1717,7 @@ func (s *fakeKafkaSink) Dial() error {
}
select {
case s.feedCh <- m:
case <-kafka.stopWorkerCh:
case <-s.tg.done:
}
return nil
Expand Down
168 changes: 85 additions & 83 deletions pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant
Expand Up @@ -22,89 +22,91 @@ DROP DATABASE crdb_internal
query TTTTIT
SHOW TABLES FROM crdb_internal
----
crdb_internal active_range_feeds table admin NULL NULL
crdb_internal backward_dependencies table admin NULL NULL
crdb_internal builtin_functions table admin NULL NULL
crdb_internal cluster_contended_indexes view admin NULL NULL
crdb_internal cluster_contended_keys view admin NULL NULL
crdb_internal cluster_contended_tables view admin NULL NULL
crdb_internal cluster_contention_events table admin NULL NULL
crdb_internal cluster_database_privileges table admin NULL NULL
crdb_internal cluster_distsql_flows table admin NULL NULL
crdb_internal cluster_execution_insights table admin NULL NULL
crdb_internal cluster_inflight_traces table admin NULL NULL
crdb_internal cluster_locks table admin NULL NULL
crdb_internal cluster_queries table admin NULL NULL
crdb_internal cluster_sessions table admin NULL NULL
crdb_internal cluster_settings table admin NULL NULL
crdb_internal cluster_statement_statistics table admin NULL NULL
crdb_internal cluster_transaction_statistics table admin NULL NULL
crdb_internal cluster_transactions table admin NULL NULL
crdb_internal cluster_txn_execution_insights table admin NULL NULL
crdb_internal create_function_statements table admin NULL NULL
crdb_internal create_schema_statements table admin NULL NULL
crdb_internal create_statements table admin NULL NULL
crdb_internal create_type_statements table admin NULL NULL
crdb_internal cross_db_references table admin NULL NULL
crdb_internal databases table admin NULL NULL
crdb_internal default_privileges table admin NULL NULL
crdb_internal feature_usage table admin NULL NULL
crdb_internal forward_dependencies table admin NULL NULL
crdb_internal gossip_alerts table admin NULL NULL
crdb_internal gossip_liveness table admin NULL NULL
crdb_internal gossip_network table admin NULL NULL
crdb_internal gossip_nodes table admin NULL NULL
crdb_internal index_columns table admin NULL NULL
crdb_internal index_spans table admin NULL NULL
crdb_internal index_usage_statistics table admin NULL NULL
crdb_internal invalid_objects table admin NULL NULL
crdb_internal jobs table admin NULL NULL
crdb_internal kv_catalog_comments table admin NULL NULL
crdb_internal kv_catalog_descriptor table admin NULL NULL
crdb_internal kv_catalog_namespace table admin NULL NULL
crdb_internal kv_catalog_zones table admin NULL NULL
crdb_internal kv_dropped_relations view admin NULL NULL
crdb_internal kv_node_liveness table admin NULL NULL
crdb_internal kv_node_status table admin NULL NULL
crdb_internal kv_store_status table admin NULL NULL
crdb_internal leases table admin NULL NULL
crdb_internal lost_descriptors_with_data table admin NULL NULL
crdb_internal node_build_info table admin NULL NULL
crdb_internal node_contention_events table admin NULL NULL
crdb_internal node_distsql_flows table admin NULL NULL
crdb_internal node_execution_insights table admin NULL NULL
crdb_internal node_inflight_trace_spans table admin NULL NULL
crdb_internal node_memory_monitors table admin NULL NULL
crdb_internal node_metrics table admin NULL NULL
crdb_internal node_queries table admin NULL NULL
crdb_internal node_runtime_info table admin NULL NULL
crdb_internal node_sessions table admin NULL NULL
crdb_internal node_statement_statistics table admin NULL NULL
crdb_internal node_tenant_capabilities_cache table admin NULL NULL
crdb_internal node_transaction_statistics table admin NULL NULL
crdb_internal node_transactions table admin NULL NULL
crdb_internal node_txn_execution_insights table admin NULL NULL
crdb_internal node_txn_stats table admin NULL NULL
crdb_internal partitions table admin NULL NULL
crdb_internal pg_catalog_table_is_implemented table admin NULL NULL
crdb_internal ranges view admin NULL NULL
crdb_internal ranges_no_leases table admin NULL NULL
crdb_internal regions table admin NULL NULL
crdb_internal schema_changes table admin NULL NULL
crdb_internal session_trace table admin NULL NULL
crdb_internal session_variables table admin NULL NULL
crdb_internal statement_statistics view admin NULL NULL
crdb_internal super_regions table admin NULL NULL
crdb_internal system_jobs table admin NULL NULL
crdb_internal table_columns table admin NULL NULL
crdb_internal table_indexes table admin NULL NULL
crdb_internal table_row_statistics table admin NULL NULL
crdb_internal table_spans table admin NULL NULL
crdb_internal tables table admin NULL NULL
crdb_internal tenant_usage_details view admin NULL NULL
crdb_internal transaction_contention_events table admin NULL NULL
crdb_internal transaction_statistics view admin NULL NULL
crdb_internal zones table admin NULL NULL
crdb_internal active_range_feeds table admin NULL NULL
crdb_internal backward_dependencies table admin NULL NULL
crdb_internal builtin_functions table admin NULL NULL
crdb_internal cluster_contended_indexes view admin NULL NULL
crdb_internal cluster_contended_keys view admin NULL NULL
crdb_internal cluster_contended_tables view admin NULL NULL
crdb_internal cluster_contention_events table admin NULL NULL
crdb_internal cluster_database_privileges table admin NULL NULL
crdb_internal cluster_distsql_flows table admin NULL NULL
crdb_internal cluster_execution_insights table admin NULL NULL
crdb_internal cluster_inflight_traces table admin NULL NULL
crdb_internal cluster_locks table admin NULL NULL
crdb_internal cluster_queries table admin NULL NULL
crdb_internal cluster_sessions table admin NULL NULL
crdb_internal cluster_settings table admin NULL NULL
crdb_internal cluster_statement_statistics table admin NULL NULL
crdb_internal cluster_transaction_statistics table admin NULL NULL
crdb_internal cluster_transactions table admin NULL NULL
crdb_internal cluster_txn_execution_insights table admin NULL NULL
crdb_internal create_function_statements table admin NULL NULL
crdb_internal create_schema_statements table admin NULL NULL
crdb_internal create_statements table admin NULL NULL
crdb_internal create_type_statements table admin NULL NULL
crdb_internal cross_db_references table admin NULL NULL
crdb_internal databases table admin NULL NULL
crdb_internal default_privileges table admin NULL NULL
crdb_internal feature_usage table admin NULL NULL
crdb_internal forward_dependencies table admin NULL NULL
crdb_internal gossip_alerts table admin NULL NULL
crdb_internal gossip_liveness table admin NULL NULL
crdb_internal gossip_network table admin NULL NULL
crdb_internal gossip_nodes table admin NULL NULL
crdb_internal index_columns table admin NULL NULL
crdb_internal index_spans table admin NULL NULL
crdb_internal index_usage_statistics table admin NULL NULL
crdb_internal invalid_objects table admin NULL NULL
crdb_internal jobs table admin NULL NULL
crdb_internal kv_catalog_comments table admin NULL NULL
crdb_internal kv_catalog_descriptor table admin NULL NULL
crdb_internal kv_catalog_namespace table admin NULL NULL
crdb_internal kv_catalog_zones table admin NULL NULL
crdb_internal kv_dropped_relations view admin NULL NULL
crdb_internal kv_node_liveness table admin NULL NULL
crdb_internal kv_node_status table admin NULL NULL
crdb_internal kv_store_status table admin NULL NULL
crdb_internal leases table admin NULL NULL
crdb_internal lost_descriptors_with_data table admin NULL NULL
crdb_internal node_build_info table admin NULL NULL
crdb_internal node_contention_events table admin NULL NULL
crdb_internal node_distsql_flows table admin NULL NULL
crdb_internal node_execution_insights table admin NULL NULL
crdb_internal node_inflight_trace_spans table admin NULL NULL
crdb_internal node_memory_monitors table admin NULL NULL
crdb_internal node_metrics table admin NULL NULL
crdb_internal node_queries table admin NULL NULL
crdb_internal node_runtime_info table admin NULL NULL
crdb_internal node_sessions table admin NULL NULL
crdb_internal node_statement_statistics table admin NULL NULL
crdb_internal node_tenant_capabilities_cache table admin NULL NULL
crdb_internal node_transaction_statistics table admin NULL NULL
crdb_internal node_transactions table admin NULL NULL
crdb_internal node_txn_execution_insights table admin NULL NULL
crdb_internal node_txn_stats table admin NULL NULL
crdb_internal partitions table admin NULL NULL
crdb_internal pg_catalog_table_is_implemented table admin NULL NULL
crdb_internal ranges view admin NULL NULL
crdb_internal ranges_no_leases table admin NULL NULL
crdb_internal regions table admin NULL NULL
crdb_internal schema_changes table admin NULL NULL
crdb_internal session_trace table admin NULL NULL
crdb_internal session_variables table admin NULL NULL
crdb_internal statement_statistics view admin NULL NULL
crdb_internal statement_statistics_persisted view admin NULL NULL
crdb_internal super_regions table admin NULL NULL
crdb_internal system_jobs table admin NULL NULL
crdb_internal table_columns table admin NULL NULL
crdb_internal table_indexes table admin NULL NULL
crdb_internal table_row_statistics table admin NULL NULL
crdb_internal table_spans table admin NULL NULL
crdb_internal tables table admin NULL NULL
crdb_internal tenant_usage_details view admin NULL NULL
crdb_internal transaction_contention_events table admin NULL NULL
crdb_internal transaction_statistics view admin NULL NULL
crdb_internal transaction_statistics_persisted view admin NULL NULL
crdb_internal zones table admin NULL NULL

statement ok
CREATE DATABASE testdb; CREATE TABLE testdb.foo(x INT)
Expand Down
2 changes: 2 additions & 0 deletions pkg/cli/zip_test.go
Expand Up @@ -97,8 +97,10 @@ table_name NOT IN (
'table_spans',
'tables',
'cluster_statement_statistics',
'statement_statistics_persisted',
'cluster_transaction_statistics',
'statement_statistics',
'transaction_statistics_persisted',
'transaction_statistics',
'tenant_usage_details',
'pg_catalog_table_is_implemented'
Expand Down
21 changes: 15 additions & 6 deletions pkg/kv/kvserver/client_spanconfigs_test.go
Expand Up @@ -13,6 +13,7 @@ package kvserver_test
import (
"context"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/keys"
Expand Down Expand Up @@ -42,7 +43,8 @@ func TestSpanConfigUpdateAppliedToReplica(t *testing.T) {
cluster.MakeTestingClusterSettings(),
nil,
)
mockSubscriber := newMockSpanConfigSubscriber(spanConfigStore)
var t0 = time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC)
mockSubscriber := newMockSpanConfigSubscriber(t0, spanConfigStore)

ctx := context.Background()

Expand Down Expand Up @@ -106,7 +108,8 @@ func TestFallbackSpanConfigOverride(t *testing.T) {

st := cluster.MakeTestingClusterSettings()
spanConfigStore := spanconfigstore.New(roachpb.TestingDefaultSpanConfig(), st, nil)
mockSubscriber := newMockSpanConfigSubscriber(spanConfigStore)
var t0 = time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC)
mockSubscriber := newMockSpanConfigSubscriber(t0, spanConfigStore)

ctx := context.Background()
args := base.TestServerArgs{
Expand Down Expand Up @@ -152,14 +155,20 @@ func TestFallbackSpanConfigOverride(t *testing.T) {
}

type mockSpanConfigSubscriber struct {
callback func(ctx context.Context, config roachpb.Span)
callback func(ctx context.Context, config roachpb.Span)
lastUpdated time.Time
spanconfig.Store
}

var _ spanconfig.KVSubscriber = &mockSpanConfigSubscriber{}

func newMockSpanConfigSubscriber(store spanconfig.Store) *mockSpanConfigSubscriber {
return &mockSpanConfigSubscriber{Store: store}
func newMockSpanConfigSubscriber(
lastUpdated time.Time, store spanconfig.Store,
) *mockSpanConfigSubscriber {
return &mockSpanConfigSubscriber{
lastUpdated: lastUpdated,
Store: store,
}
}

func (m *mockSpanConfigSubscriber) NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool {
Expand All @@ -185,7 +194,7 @@ func (m *mockSpanConfigSubscriber) GetProtectionTimestamps(
}

func (m *mockSpanConfigSubscriber) LastUpdated() hlc.Timestamp {
panic("unimplemented")
return hlc.Timestamp{WallTime: m.lastUpdated.UnixNano()}
}

func (m *mockSpanConfigSubscriber) Subscribe(callback func(context.Context, roachpb.Span)) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/consistency_queue.go
Expand Up @@ -93,7 +93,7 @@ func newConsistencyQueue(store *Store) *consistencyQueue {
queueConfig{
maxSize: defaultQueueMaxSize,
needsLease: true,
needsSystemConfig: false,
needsSpanConfigs: false,
acceptsUnsplitRanges: true,
successes: store.metrics.ConsistencyQueueSuccesses,
failures: store.metrics.ConsistencyQueueFailures,
Expand Down
27 changes: 27 additions & 0 deletions pkg/kv/kvserver/kvserverbase/base.go
Expand Up @@ -36,6 +36,33 @@ var MergeQueueEnabled = settings.RegisterBoolSetting(
true,
)

// ReplicateQueueEnabled is a setting that controls whether the replicate queue
// is enabled.
var ReplicateQueueEnabled = settings.RegisterBoolSetting(
settings.SystemOnly,
"kv.replicate_queue.enabled",
"whether the replicate queue is enabled",
true,
)

// SplitQueueEnabled is a setting that controls whether the split queue is
// enabled.
var SplitQueueEnabled = settings.RegisterBoolSetting(
settings.SystemOnly,
"kv.split_queue.enabled",
"whether the split queue is enabled",
true,
)

// MVCCGCQueueEnabled is a setting that controls whether the MVCC GC queue is
// enabled.
var MVCCGCQueueEnabled = settings.RegisterBoolSetting(
settings.SystemOnly,
"kv.mvcc_gc_queue.enabled",
"whether the MVCC GC queue is enabled",
true,
)

// CmdIDKey is a Raft command id. This will be logged unredacted - keep it random.
type CmdIDKey string

Expand Down
11 changes: 1 addition & 10 deletions pkg/kv/kvserver/merge_queue.go
Expand Up @@ -116,7 +116,7 @@ func newMergeQueue(store *Store, db *kv.DB) *mergeQueue {
// factor.
processTimeoutFunc: makeRateLimitedTimeoutFunc(rebalanceSnapshotRate, recoverySnapshotRate),
needsLease: true,
needsSystemConfig: true,
needsSpanConfigs: true,
acceptsUnsplitRanges: false,
successes: store.metrics.MergeQueueSuccesses,
failures: store.metrics.MergeQueueFailures,
Expand All @@ -129,15 +129,6 @@ func newMergeQueue(store *Store, db *kv.DB) *mergeQueue {
}

func (mq *mergeQueue) enabled() bool {
if !mq.store.cfg.SpanConfigsDisabled {
if mq.store.cfg.SpanConfigSubscriber.LastUpdated().IsEmpty() {
// If we don't have any span configs available, enabling range merges would
// be extremely dangerous -- we could collapse everything into a single
// range.
return false
}
}

st := mq.store.ClusterSettings()
return kvserverbase.MergeQueueEnabled.Get(&st.SV)
}
Expand Down
17 changes: 16 additions & 1 deletion pkg/kv/kvserver/mvcc_gc_queue.go
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand Down Expand Up @@ -178,7 +179,7 @@ func newMVCCGCQueue(store *Store) *mvccGCQueue {
queueConfig{
maxSize: defaultQueueMaxSize,
needsLease: true,
needsSystemConfig: true,
needsSpanConfigs: true,
acceptsUnsplitRanges: false,
processTimeoutFunc: func(st *cluster.Settings, _ replicaInQueue) time.Duration {
timeout := mvccGCQueueTimeout
Expand Down Expand Up @@ -232,13 +233,22 @@ func (r mvccGCQueueScore) String() string {
humanizeutil.IBytes(r.GCByteAge), humanizeutil.IBytes(r.ExpMinGCByteAgeReduction))
}

func (mgcq *mvccGCQueue) enabled() bool {
st := mgcq.store.ClusterSettings()
return kvserverbase.MVCCGCQueueEnabled.Get(&st.SV)
}

// shouldQueue determines whether a replica should be queued for garbage
// collection, and if so, at what priority. Returns true for shouldQ
// in the event that the cumulative ages of GC'able bytes or extant
// intents exceed thresholds.
func (mgcq *mvccGCQueue) shouldQueue(
ctx context.Context, _ hlc.ClockTimestamp, repl *Replica, _ spanconfig.StoreReader,
) (bool, float64) {
if !mgcq.enabled() {
return false, 0
}

// Consult the protected timestamp state to determine whether we can GC and
// the timestamp which can be used to calculate the score.
conf := repl.SpanConfig()
Expand Down Expand Up @@ -672,6 +682,11 @@ func (r *replicaGCer) GC(
func (mgcq *mvccGCQueue) process(
ctx context.Context, repl *Replica, _ spanconfig.StoreReader,
) (processed bool, err error) {
if !mgcq.enabled() {
log.VEventf(ctx, 2, "skipping mvcc gc: queue has been disabled")
return false, nil
}

// Record the CPU time processing the request for this replica. This is
// recorded regardless of errors that are encountered.
defer repl.MeasureReqCPUNanos(grunning.Time())
Expand Down

0 comments on commit c31c1ac

Please sign in to comment.