From b3713c785ea39338dccaf38f82de41a5e0a9b272 Mon Sep 17 00:00:00 2001 From: Marius Posta Date: Wed, 25 Jan 2023 10:05:23 -0500 Subject: [PATCH 1/5] validate: use immutable descriptors only The descriptor validation logic will accept any implementation of catalog.Descriptor be it mutable or immutable, it doesn't care. However, using mutable implementations can have a significant performance impact especially in the case of tables, where every column or index or constraint lookup will lead to the cache being regenerated for the whole descriptor. This commit fixes this by having validate.Validate replace any mutable descriptor instances it encounters with immutable copies. This doesn't change anything except performance. Fixes #95827. Release note: None --- pkg/sql/catalog/internal/validate/validate.go | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/pkg/sql/catalog/internal/validate/validate.go b/pkg/sql/catalog/internal/validate/validate.go index 616dd23ed605..3d1d834bccbe 100644 --- a/pkg/sql/catalog/internal/validate/validate.go +++ b/pkg/sql/catalog/internal/validate/validate.go @@ -58,6 +58,15 @@ func Validate( targetLevel catalog.ValidationLevel, descriptors ...catalog.Descriptor, ) catalog.ValidationErrors { + for i, d := range descriptors { + // Replace mutable descriptors with immutable copies. Validation is + // read-only in any case, and using immutables can have a significant + // impact on performance when validating tables due to columns, indexes, + // and so forth being cached. + if mut, ok := d.(catalog.MutableDescriptor); ok { + descriptors[i] = mut.ImmutableCopy() + } + } vea := validationErrorAccumulator{ ValidationTelemetry: telemetry, targetLevel: targetLevel, @@ -409,9 +418,17 @@ func (cs *collectorState) getMissingDescs( return nil, err } for _, desc := range resps { - if desc != nil { - cs.vdg.descriptors[desc.GetID()] = desc + if desc == nil { + continue + } + if mut, ok := desc.(catalog.MutableDescriptor); ok { + // Replace mutable descriptors with immutable copies. Validation is + // read-only in any case, and using immutables can have a significant + // impact on performance when validating tables due to columns, indexes, + // and so forth being cached. + desc = mut.ImmutableCopy() } + cs.vdg.descriptors[desc.GetID()] = desc } return resps, nil } From 662e19c291607cc7668de24bd20906d30c571713 Mon Sep 17 00:00:00 2001 From: Evan Wall Date: Tue, 24 Jan 2023 18:00:08 -0500 Subject: [PATCH 2/5] sql: improve stack trace for get-user-timeout timeouts Fixes #95794 The cause of the `get-user-timeout errors` is unknown. Part of the problem is that the stack trace gets cut off at ``` | | github.com/cockroachdb/cockroach/pkg/sql.retrieveSessionInitInfoWithCache | | github.com/cockroachdb/cockroach/pkg/sql/user.go:238 ``` which does not explain what is actually being blocked. The reason that the stack trace is cut off is that the timeout is initiated by `contextutil.RunWithTimeout` which results in a "simple" (no stack trace) `context.DeadlineExceeded` error. `retrieveSessionInitInfoWithCache` is the first line in the stack trace because it calls `errors.Wrap` on `context.DeadlineExceeded`. To get a fuller stack trace, `context.DeadlineExceeded` must be wrapped immediately (`errors.Wrap` or `errors.WithStack`) before it bubbles up. Release note: None --- pkg/util/contextutil/context.go | 13 +++++++++++++ pkg/util/contextutil/context_test.go | 19 +++++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/pkg/util/contextutil/context.go b/pkg/util/contextutil/context.go index db7dbb03597c..87a139437153 100644 --- a/pkg/util/contextutil/context.go +++ b/pkg/util/contextutil/context.go @@ -79,6 +79,18 @@ func wrap(ctx context.Context, cancel context.CancelFunc) (context.Context, cont } } +// ctxWithStacktrace overrides Err to annotate context.DeadlineExceeded and +// context.Canceled errors with a stacktrace. +// See: https://github.com/cockroachdb/cockroach/issues/95794 +type ctxWithStacktrace struct { + context.Context +} + +// Err implements the context.Context interface. +func (ctx *ctxWithStacktrace) Err() error { + return errors.WithStack(ctx.Context.Err()) +} + // RunWithTimeout runs a function with a timeout, the same way you'd do with // context.WithTimeout. It improves the opaque error messages returned by // WithTimeout by augmenting them with the op string that is passed in. @@ -86,6 +98,7 @@ func RunWithTimeout( ctx context.Context, op string, timeout time.Duration, fn func(ctx context.Context) error, ) error { ctx, cancel := context.WithTimeout(ctx, timeout) + ctx = &ctxWithStacktrace{Context: ctx} defer cancel() start := timeutil.Now() err := fn(ctx) diff --git a/pkg/util/contextutil/context_test.go b/pkg/util/contextutil/context_test.go index f313f8f540df..f84748f5a535 100644 --- a/pkg/util/contextutil/context_test.go +++ b/pkg/util/contextutil/context_test.go @@ -12,6 +12,7 @@ package contextutil import ( "context" + "fmt" "net" "testing" "time" @@ -70,6 +71,24 @@ func TestRunWithTimeout(t *testing.T) { } } +func testFuncA(ctx context.Context) error { + return testFuncB(ctx) +} + +func testFuncB(ctx context.Context) error { + <-ctx.Done() + return ctx.Err() +} + +func TestRunWithTimeoutCtxWithStacktrace(t *testing.T) { + ctx := context.Background() + err := RunWithTimeout(ctx, "foo", 1, testFuncA) + require.Error(t, err) + stacktrace := fmt.Sprintf("%+v", err) + require.Contains(t, stacktrace, "testFuncB") + require.Contains(t, stacktrace, "testFuncA") +} + // TestRunWithTimeoutWithoutDeadlineExceeded ensures that when a timeout on the // context occurs but the underlying error does not have // context.DeadlineExceeded as its Cause (perhaps due to serialization) the From 62d314d26619be1f7841f76c68c582e2aa8e38bc Mon Sep 17 00:00:00 2001 From: maryliag Date: Wed, 25 Jan 2023 13:57:41 -0500 Subject: [PATCH 3/5] ui: cache sqlroles results Previously, the call to get sql roles was constantly being requested. This commits adds a cache limit, so it will only get request after the expiration time. Epic: None Release note: None --- .../workspaces/cluster-ui/src/api/userApi.ts | 4 +- .../workspaces/cluster-ui/src/store/sagas.ts | 2 +- .../src/store/uiConfig/uiConfig.reducer.ts | 16 ++++--- .../src/store/uiConfig/uiConfig.sagas.ts | 46 +++++++++++++++---- 4 files changed, 47 insertions(+), 21 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/api/userApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/userApi.ts index c7c2c4681e5d..29362b287943 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/userApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/userApi.ts @@ -16,9 +16,7 @@ export type UserSQLRolesRequestMessage = export type UserSQLRolesResponseMessage = cockroach.server.serverpb.UserSQLRolesResponse; -export function getUserSQLRoles( - req: UserSQLRolesRequestMessage, -): Promise { +export function getUserSQLRoles(): Promise { return fetchData( cockroach.server.serverpb.UserSQLRolesResponse, `/_status/sqlroles`, diff --git a/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts index b5067189d60f..98dbdbd8ac22 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/sagas.ts @@ -49,6 +49,6 @@ export function* sagas(cacheInvalidationPeriod?: number): SagaIterator { fork(indexStatsSaga), fork(clusterLocksSaga), fork(schemaInsightsSaga), - fork(uiConfigSaga), + fork(uiConfigSaga, cacheInvalidationPeriod), ]); } diff --git a/pkg/ui/workspaces/cluster-ui/src/store/uiConfig/uiConfig.reducer.ts b/pkg/ui/workspaces/cluster-ui/src/store/uiConfig/uiConfig.reducer.ts index 22ac70ecf430..ca51a9ca3830 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/uiConfig/uiConfig.reducer.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/uiConfig/uiConfig.reducer.ts @@ -10,7 +10,7 @@ import { createSlice, PayloadAction } from "@reduxjs/toolkit"; import { merge } from "lodash"; -import { DOMAIN_NAME } from "../utils"; +import { DOMAIN_NAME, noopReducer } from "../utils"; import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; export type UserSQLRolesRequest = cockroach.server.serverpb.UserSQLRolesRequest; @@ -57,15 +57,17 @@ const uiConfigSlice = createSlice({ update: (state, action: PayloadAction>) => { merge(state, action.payload); }, - refreshUserSQLRoles: ( - state, - action?: PayloadAction, - ) => { + receivedUserSQLRoles: (state, action: PayloadAction) => { if (action?.payload) { - const resp = action.payload.toJSON(); - state.userSQLRoles = resp["roles"]; + state.userSQLRoles = action.payload; } }, + invalidatedUserSQLRoles: state => { + state.userSQLRoles = []; + }, + // Define actions that don't change state + refreshUserSQLRoles: noopReducer, + requestUserSQLRoles: noopReducer, }, }); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/uiConfig/uiConfig.sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/uiConfig/uiConfig.sagas.ts index 529669c61f2a..0255dcdabb78 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/uiConfig/uiConfig.sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/uiConfig/uiConfig.sagas.ts @@ -8,22 +8,48 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -import { all, call, put, takeLatest } from "redux-saga/effects"; -import { actions, UserSQLRolesRequest } from "./uiConfig.reducer"; -import { PayloadAction } from "@reduxjs/toolkit"; +import { all, call, delay, put, takeLatest } from "redux-saga/effects"; +import { actions } from "./uiConfig.reducer"; import { getUserSQLRoles } from "../../api/userApi"; +import { CACHE_INVALIDATION_PERIOD, throttleWithReset } from "../utils"; +import { rootActions } from "../reducers"; +import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; -export function* refreshUserSQLRoles( - action: PayloadAction, -): any { +export function* refreshUserSQLRolesSaga(): any { + yield put(actions.requestUserSQLRoles()); +} + +export function* requestUserSQLRolesSaga(): any { try { - const result = yield call(getUserSQLRoles, action?.payload); - yield put(actions.refreshUserSQLRoles(result)); + const result: cockroach.server.serverpb.UserSQLRolesResponse = yield call( + getUserSQLRoles, + ); + yield put(actions.receivedUserSQLRoles(result.roles)); } catch (e) { console.warn(e.message); } } -export function* uiConfigSaga() { - yield all([takeLatest(actions.refreshUserSQLRoles, refreshUserSQLRoles)]); +export function* receivedUserSQLRolesSaga(delayMs: number): any { + yield delay(delayMs); + yield put(actions.invalidatedUserSQLRoles()); +} + +export function* uiConfigSaga( + cacheInvalidationPeriod: number = CACHE_INVALIDATION_PERIOD, +): any { + yield all([ + throttleWithReset( + cacheInvalidationPeriod, + actions.refreshUserSQLRoles, + [actions.invalidatedUserSQLRoles, rootActions.resetState], + refreshUserSQLRolesSaga, + ), + takeLatest(actions.requestUserSQLRoles, requestUserSQLRolesSaga), + takeLatest( + actions.receivedUserSQLRoles, + receivedUserSQLRolesSaga, + cacheInvalidationPeriod, + ), + ]); } From e40bd52902e72aa5762655c63b106870ab862d94 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Wed, 25 Jan 2023 15:29:37 -0500 Subject: [PATCH 4/5] storage: reorder EventListeners To be defensive, sequence the EventListener responsible for crashing the process during a disk stall first, before the Pebble logging event listener. Informs #94373. Epic: None Release note: None --- pkg/storage/pebble.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index afb7b3f8c26f..31c3282a4db0 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -981,12 +981,18 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { return int(atomic.LoadUint64(&p.atomic.compactionConcurrency)) } + // NB: The ordering of the event listeners passed to TeeEventListener is + // deliberate. The listener returned by makeMetricEtcEventListener is + // responsible for crashing the process if a DiskSlow event indicates the + // disk is stalled. While the logging subsystem should also be robust to + // stalls and crash the process if unable to write logs, there's less risk + // to sequencing the crashing listener first. el := pebble.TeeEventListener( + p.makeMetricEtcEventListener(ctx), pebble.MakeLoggingEventListener(pebbleLogger{ ctx: logCtx, depth: 2, // skip over the EventListener stack frame }), - p.makeMetricEtcEventListener(ctx), ) p.eventListener = &el From 0b29509521329ac7a1c995f45397e0ba212d9ec1 Mon Sep 17 00:00:00 2001 From: Evan Wall Date: Tue, 10 Jan 2023 14:27:08 -0500 Subject: [PATCH 5/5] sql: add ability set, edit, read tenant capabilities Fixes #87851 Add new SQL syntax for 1) Setting tenant capabilities: `ALTER TENANT t GRANT CAPABILITY capabilitiy_name=capability_value;` 2) Resetting tenant capabilities: `ALTER TENANT t REVOKE CAPABILITIY capability_name;` 3) Reading tenant capabilities: `SHOW TENANT t WITH CAPABILITIES;` Release note: None --- docs/generated/sql/bnf/BUILD.bazel | 3 + .../sql/bnf/opt_show_tenant_options.bnf | 2 + docs/generated/sql/bnf/show_tenant_stmt.bnf | 9 +- docs/generated/sql/bnf/stmt_block.bnf | 17 +- docs/generated/sql/bnf/tenant_capability.bnf | 3 + .../sql/bnf/tenant_capability_list.bnf | 2 + .../testdata/logic_test/tenant_capability | 43 +++++ .../tests/local/generated_test.go | 7 + pkg/gen/bnf.bzl | 3 + pkg/gen/diagrams.bzl | 3 + pkg/gen/docs.bzl | 3 + pkg/sql/BUILD.bazel | 1 + pkg/sql/catalog/colinfo/result_columns.go | 7 + pkg/sql/opaque.go | 3 + pkg/sql/parser/sql.y | 141 +++++++++++++--- pkg/sql/parser/testdata/alter_tenant | 48 ++++++ pkg/sql/parser/testdata/show | 8 + pkg/sql/sem/tree/alter_tenant.go | 53 ++++++ pkg/sql/sem/tree/show.go | 14 +- pkg/sql/sem/tree/stmt.go | 10 ++ pkg/sql/sem/tree/walk.go | 27 +++ pkg/sql/show_tenant.go | 159 ++++++++++++------ pkg/sql/tenant_capability.go | 84 +++++++++ pkg/sql/walk.go | 2 + 24 files changed, 562 insertions(+), 90 deletions(-) create mode 100644 docs/generated/sql/bnf/opt_show_tenant_options.bnf create mode 100644 docs/generated/sql/bnf/tenant_capability.bnf create mode 100644 docs/generated/sql/bnf/tenant_capability_list.bnf create mode 100644 pkg/ccl/logictestccl/testdata/logic_test/tenant_capability create mode 100644 pkg/sql/tenant_capability.go diff --git a/docs/generated/sql/bnf/BUILD.bazel b/docs/generated/sql/bnf/BUILD.bazel index 644cef523a04..82999088d812 100644 --- a/docs/generated/sql/bnf/BUILD.bazel +++ b/docs/generated/sql/bnf/BUILD.bazel @@ -165,6 +165,7 @@ FILES = [ "opt_frame_clause", "opt_locality", "opt_persistence_temp_table", + "opt_show_tenant_options", "opt_with_storage_parameter_list", "pause_all_jobs_stmt", "pause_job", @@ -266,6 +267,8 @@ FILES = [ "table_clause", "table_constraint", "table_ref", + "tenant_capability", + "tenant_capability_list", "transaction_stmt", "truncate_stmt", "unique_column_level", diff --git a/docs/generated/sql/bnf/opt_show_tenant_options.bnf b/docs/generated/sql/bnf/opt_show_tenant_options.bnf new file mode 100644 index 000000000000..24f7e7bdba00 --- /dev/null +++ b/docs/generated/sql/bnf/opt_show_tenant_options.bnf @@ -0,0 +1,2 @@ +opt_show_tenant_options ::= + 'WITH' show_tenant_options diff --git a/docs/generated/sql/bnf/show_tenant_stmt.bnf b/docs/generated/sql/bnf/show_tenant_stmt.bnf index ec8fc653487f..36496113c396 100644 --- a/docs/generated/sql/bnf/show_tenant_stmt.bnf +++ b/docs/generated/sql/bnf/show_tenant_stmt.bnf @@ -1,7 +1,4 @@ show_tenant_stmt ::= - 'SHOW' 'TENANTS' - | 'SHOW' 'TENANT_ALL' 'ALL' - | 'SHOW' 'TENANTS' 'WITH' 'REPLICATION' 'STATUS' - | 'SHOW' 'TENANT_ALL' 'ALL' 'WITH' 'REPLICATION' 'STATUS' - | 'SHOW' 'TENANT' tenant_spec - | 'SHOW' 'TENANT' tenant_spec 'WITH' 'REPLICATION' 'STATUS' + 'SHOW' 'TENANTS' opt_show_tenant_options + | 'SHOW' 'TENANT_ALL' 'ALL' opt_show_tenant_options + | 'SHOW' 'TENANT' tenant_spec opt_show_tenant_options diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 3ccf1474840e..5d347c1cf792 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -917,12 +917,9 @@ show_tables_stmt ::= | 'SHOW' 'TABLES' with_comment show_tenant_stmt ::= - 'SHOW' 'TENANTS' - | 'SHOW' 'TENANT_ALL' 'ALL' - | 'SHOW' 'TENANTS' 'WITH' 'REPLICATION' 'STATUS' - | 'SHOW' 'TENANT_ALL' 'ALL' 'WITH' 'REPLICATION' 'STATUS' - | 'SHOW' 'TENANT' tenant_spec - | 'SHOW' 'TENANT' tenant_spec 'WITH' 'REPLICATION' 'STATUS' + 'SHOW' 'TENANTS' opt_show_tenant_options + | 'SHOW' 'TENANT_ALL' 'ALL' opt_show_tenant_options + | 'SHOW' 'TENANT' tenant_spec opt_show_tenant_options show_trace_stmt ::= 'SHOW' opt_compact 'TRACE' 'FOR' 'SESSION' @@ -1031,6 +1028,8 @@ unreserved_keyword ::= | 'CALLED' | 'CANCEL' | 'CANCELQUERY' + | 'CAPABILITIES' + | 'CAPABILITY' | 'CASCADE' | 'CHANGEFEED' | 'CLOSE' @@ -1938,6 +1937,9 @@ statements_or_queries ::= opt_show_ranges_options ::= 'WITH' show_ranges_options +opt_show_tenant_options ::= + 'WITH' show_tenant_options + opt_compact ::= 'COMPACT' | @@ -2647,6 +2649,9 @@ targets_roles ::= show_ranges_options ::= ( 'TABLES' | 'INDEXES' | 'DETAILS' | 'KEYS' | 'EXPLAIN' ) ( ( ',' 'TABLES' | ',' 'INDEXES' | ',' 'DETAILS' | ',' 'EXPLAIN' | ',' 'KEYS' ) )* +show_tenant_options ::= + ( 'REPLICATION' 'STATUS' | 'CAPABILITIES' ) ( ( ',' 'REPLICATION' 'STATUS' | ',' 'CAPABILITIES' ) )* + partition ::= 'PARTITION' partition_name diff --git a/docs/generated/sql/bnf/tenant_capability.bnf b/docs/generated/sql/bnf/tenant_capability.bnf new file mode 100644 index 000000000000..20fcc373191c --- /dev/null +++ b/docs/generated/sql/bnf/tenant_capability.bnf @@ -0,0 +1,3 @@ +tenant_capability ::= + var_name + | var_name to_or_eq var_value diff --git a/docs/generated/sql/bnf/tenant_capability_list.bnf b/docs/generated/sql/bnf/tenant_capability_list.bnf new file mode 100644 index 000000000000..960a9d14efaa --- /dev/null +++ b/docs/generated/sql/bnf/tenant_capability_list.bnf @@ -0,0 +1,2 @@ +tenant_capability_list ::= + tenant_capability ( ( ',' tenant_capability ) )* diff --git a/pkg/ccl/logictestccl/testdata/logic_test/tenant_capability b/pkg/ccl/logictestccl/testdata/logic_test/tenant_capability new file mode 100644 index 000000000000..8bfb3abfeb99 --- /dev/null +++ b/pkg/ccl/logictestccl/testdata/logic_test/tenant_capability @@ -0,0 +1,43 @@ +# LogicTest: local + +statement ok +SELECT crdb_internal.create_tenant(5, 'five') + +statement error error parsing capability "not_a_capability": invalid capability +ALTER TENANT [5] GRANT CAPABILITY not_a_capability=true + +statement error error parsing capability "can_admin_split": value must be bool +ALTER TENANT [5] GRANT CAPABILITY can_admin_split=1 + +statement error error parsing capability "not_a_capability": invalid capability +ALTER TENANT [5] REVOKE CAPABILITY not_a_capability + +statement error error parsing capability "can_admin_split": revoke must not specify value +ALTER TENANT [5] REVOKE CAPABILITY can_admin_split=false + +query ITTTT colnames +SHOW TENANT 'five' WITH CAPABILITIES +---- +id name status capability_name capability_value +5 five ACTIVE can_admin_split false +5 five ACTIVE can_admin_unsplit false + +statement ok +ALTER TENANT [5] GRANT CAPABILITY can_admin_split=true + +query ITTTT colnames +SHOW TENANT 'five' WITH CAPABILITIES +---- +id name status capability_name capability_value +5 five ACTIVE can_admin_split true +5 five ACTIVE can_admin_unsplit false + +statement ok +ALTER TENANT [5] REVOKE CAPABILITY can_admin_split + +query ITTTT colnames +SHOW TENANT 'five' WITH CAPABILITIES +---- +id name status capability_name capability_value +5 five ACTIVE can_admin_split false +5 five ACTIVE can_admin_unsplit false diff --git a/pkg/ccl/logictestccl/tests/local/generated_test.go b/pkg/ccl/logictestccl/tests/local/generated_test.go index 458d132af8f5..0e6afe61ed4e 100644 --- a/pkg/ccl/logictestccl/tests/local/generated_test.go +++ b/pkg/ccl/logictestccl/tests/local/generated_test.go @@ -184,6 +184,13 @@ func TestCCLLogic_show_create( runCCLLogicTest(t, "show_create") } +func TestCCLLogic_tenant_capability( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runCCLLogicTest(t, "tenant_capability") +} + func TestCCLLogic_tenant_usage( t *testing.T, ) { diff --git a/pkg/gen/bnf.bzl b/pkg/gen/bnf.bzl index b22742da50e5..f54c79118a32 100644 --- a/pkg/gen/bnf.bzl +++ b/pkg/gen/bnf.bzl @@ -165,6 +165,7 @@ BNF_SRCS = [ "//docs/generated/sql/bnf:opt_frame_clause.bnf", "//docs/generated/sql/bnf:opt_locality.bnf", "//docs/generated/sql/bnf:opt_persistence_temp_table.bnf", + "//docs/generated/sql/bnf:opt_show_tenant_options.bnf", "//docs/generated/sql/bnf:opt_with_storage_parameter_list.bnf", "//docs/generated/sql/bnf:pause_all_jobs_stmt.bnf", "//docs/generated/sql/bnf:pause_job.bnf", @@ -266,6 +267,8 @@ BNF_SRCS = [ "//docs/generated/sql/bnf:table_clause.bnf", "//docs/generated/sql/bnf:table_constraint.bnf", "//docs/generated/sql/bnf:table_ref.bnf", + "//docs/generated/sql/bnf:tenant_capability.bnf", + "//docs/generated/sql/bnf:tenant_capability_list.bnf", "//docs/generated/sql/bnf:transaction_stmt.bnf", "//docs/generated/sql/bnf:truncate_stmt.bnf", "//docs/generated/sql/bnf:unique_column_level.bnf", diff --git a/pkg/gen/diagrams.bzl b/pkg/gen/diagrams.bzl index 654330faec50..2fb5135ee36e 100644 --- a/pkg/gen/diagrams.bzl +++ b/pkg/gen/diagrams.bzl @@ -161,6 +161,7 @@ DIAGRAMS_SRCS = [ "//docs/generated/sql/bnf:opt_frame_clause.html", "//docs/generated/sql/bnf:opt_locality.html", "//docs/generated/sql/bnf:opt_persistence_temp_table.html", + "//docs/generated/sql/bnf:opt_show_tenant_options.html", "//docs/generated/sql/bnf:opt_with_storage_parameter_list.html", "//docs/generated/sql/bnf:pause.html", "//docs/generated/sql/bnf:pause_all_jobs.html", @@ -260,6 +261,8 @@ DIAGRAMS_SRCS = [ "//docs/generated/sql/bnf:table_clause.html", "//docs/generated/sql/bnf:table_constraint.html", "//docs/generated/sql/bnf:table_ref.html", + "//docs/generated/sql/bnf:tenant_capability.html", + "//docs/generated/sql/bnf:tenant_capability_list.html", "//docs/generated/sql/bnf:transaction.html", "//docs/generated/sql/bnf:truncate.html", "//docs/generated/sql/bnf:unique_column_level.html", diff --git a/pkg/gen/docs.bzl b/pkg/gen/docs.bzl index bad5180e4298..e98b7429fcbe 100644 --- a/pkg/gen/docs.bzl +++ b/pkg/gen/docs.bzl @@ -177,6 +177,7 @@ DOCS_SRCS = [ "//docs/generated/sql/bnf:opt_frame_clause.bnf", "//docs/generated/sql/bnf:opt_locality.bnf", "//docs/generated/sql/bnf:opt_persistence_temp_table.bnf", + "//docs/generated/sql/bnf:opt_show_tenant_options.bnf", "//docs/generated/sql/bnf:opt_with_storage_parameter_list.bnf", "//docs/generated/sql/bnf:pause_all_jobs_stmt.bnf", "//docs/generated/sql/bnf:pause_job.bnf", @@ -278,6 +279,8 @@ DOCS_SRCS = [ "//docs/generated/sql/bnf:table_clause.bnf", "//docs/generated/sql/bnf:table_constraint.bnf", "//docs/generated/sql/bnf:table_ref.bnf", + "//docs/generated/sql/bnf:tenant_capability.bnf", + "//docs/generated/sql/bnf:tenant_capability_list.bnf", "//docs/generated/sql/bnf:transaction_stmt.bnf", "//docs/generated/sql/bnf:truncate_stmt.bnf", "//docs/generated/sql/bnf:unique_column_level.bnf", diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index e1efb737b04a..62c2986901fa 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -241,6 +241,7 @@ go_library( "telemetry_logging.go", "temporary_schema.go", "tenant_accessors.go", + "tenant_capability.go", "tenant_creation.go", "tenant_deletion.go", "tenant_gc.go", diff --git a/pkg/sql/catalog/colinfo/result_columns.go b/pkg/sql/catalog/colinfo/result_columns.go index 3f75744c281f..6ac64174622d 100644 --- a/pkg/sql/catalog/colinfo/result_columns.go +++ b/pkg/sql/catalog/colinfo/result_columns.go @@ -296,6 +296,13 @@ var TenantColumnsWithReplication = ResultColumns{ {Name: "cutover_time", Typ: types.Decimal}, } +// TenantColumnsWithCapabilities is appended to TenantColumns for +// SHOW TENANT ... WITH CAPABILITIES queries. +var TenantColumnsWithCapabilities = ResultColumns{ + {Name: "capability_name", Typ: types.String}, + {Name: "capability_value", Typ: types.String}, +} + // RangesNoLeases is the schema for crdb_internal.ranges_no_leases. var RangesNoLeases = ResultColumns{ {Name: "range_id", Typ: types.Int}, diff --git a/pkg/sql/opaque.go b/pkg/sql/opaque.go index 62077f4644dd..25d98a9b203c 100644 --- a/pkg/sql/opaque.go +++ b/pkg/sql/opaque.go @@ -128,6 +128,8 @@ func planOpaque(ctx context.Context, p *planner, stmt tree.Statement) (planNode, return p.AlterTableOwner(ctx, n) case *tree.AlterTableSetSchema: return p.AlterTableSetSchema(ctx, n) + case *tree.AlterTenantCapability: + return p.AlterTenantCapability(ctx, n) case *tree.AlterTenantSetClusterSetting: return p.AlterTenantSetClusterSetting(ctx, n) case *tree.AlterTenantRename: @@ -312,6 +314,7 @@ func init() { &tree.AlterTableLocality{}, &tree.AlterTableOwner{}, &tree.AlterTableSetSchema{}, + &tree.AlterTenantCapability{}, &tree.AlterTenantRename{}, &tree.AlterTenantSetClusterSetting{}, &tree.AlterType{}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 9b9b044b3545..21b0fd23f00a 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -410,6 +410,15 @@ func (u *sqlSymUnion) storageParamKeys() []tree.Name { } return nil } +func (u *sqlSymUnion) tenantCapability() tree.TenantCapability { + return u.val.(tree.TenantCapability) +} +func (u *sqlSymUnion) tenantCapabilities() []tree.TenantCapability { + if capabilities, ok := u.val.([]tree.TenantCapability); ok { + return capabilities + } + return nil +} func (u *sqlSymUnion) persistence() tree.Persistence { return u.val.(tree.Persistence) } @@ -843,6 +852,9 @@ func (u *sqlSymUnion) tenantSpec() *tree.TenantSpec { func (u *sqlSymUnion) cteMaterializeClause() tree.CTEMaterializeClause { return u.val.(tree.CTEMaterializeClause) } +func (u *sqlSymUnion) showTenantOpts() tree.ShowTenantOptions { + return u.val.(tree.ShowTenantOptions) +} %} // NB: the %token definitions must come before the %type definitions in this @@ -870,7 +882,7 @@ func (u *sqlSymUnion) cteMaterializeClause() tree.CTEMaterializeClause { %token BUCKET_COUNT %token BOOLEAN BOTH BOX2D BUNDLE BY -%token CACHE CALLED CANCEL CANCELQUERY CASCADE CASE CAST CBRT CHANGEFEED CHAR +%token CACHE CALLED CANCEL CANCELQUERY CAPABILITIES CAPABILITY CASCADE CASE CAST CBRT CHANGEFEED CHAR %token CHARACTER CHARACTERISTICS CHECK CLOSE %token CLUSTER COALESCE COLLATE COLLATION COLUMN COLUMNS COMMENT COMMENTS COMMIT %token COMMITTED COMPACT COMPLETE COMPLETIONS CONCAT CONCURRENTLY CONFIGURATION CONFIGURATIONS CONFIGURE @@ -1042,10 +1054,18 @@ func (u *sqlSymUnion) cteMaterializeClause() tree.CTEMaterializeClause { %type alter_table_locality_stmt %type alter_table_owner_stmt -// ALTER TENANT CLUSTER SETTINGS +// ALTER TENANT %type alter_tenant_stmt + +// ALTER TENANT CAPABILITY +%type tenant_capability tenant_capability_list + +// ALTER TENANT CLUSTER SETTINGS %type alter_tenant_csetting_stmt +// ALTER TENANT CAPABILITY +%type alter_tenant_capability_stmt + // Other ALTER TENANT statements. %type alter_tenant_replication_stmt %type alter_tenant_rename_stmt @@ -1242,7 +1262,7 @@ func (u *sqlSymUnion) cteMaterializeClause() tree.CTEMaterializeClause { %type show_syntax_stmt %type show_last_query_stats_stmt %type show_tables_stmt -%type show_tenant_stmt +%type show_tenant_stmt opt_show_tenant_options show_tenant_options %type show_trace_stmt %type show_transaction_stmt %type show_transactions_stmt @@ -5568,51 +5588,64 @@ backup_kms: // %Help: SHOW TENANT - display tenant information // %Category: Experimental // %Text: -// SHOW { TENANT { | ALL } | TENANTS ] [WITH REPLICATION STATUS] +// SHOW TENANT { | ALL } [ WITH ] +// SHOW TENANTS [ WITH ] +// +// Options: +// REPLICATION STATUS +// CAPABILITIES show_tenant_stmt: - SHOW TENANTS + SHOW TENANTS opt_show_tenant_options { $$.val = &tree.ShowTenant{ TenantSpec: &tree.TenantSpec{All: true}, - WithReplication: false, + ShowTenantOptions: $3.showTenantOpts(), } } -| SHOW TENANT_ALL ALL +| SHOW TENANT_ALL ALL opt_show_tenant_options { $$.val = &tree.ShowTenant{ TenantSpec: &tree.TenantSpec{All: true}, - WithReplication: false, + ShowTenantOptions: $4.showTenantOpts(), } } -| SHOW TENANTS WITH REPLICATION STATUS +| SHOW TENANT tenant_spec opt_show_tenant_options { $$.val = &tree.ShowTenant{ - TenantSpec: &tree.TenantSpec{All: true}, - WithReplication: true, + TenantSpec: $3.tenantSpec(), + ShowTenantOptions: $4.showTenantOpts(), + } } -| SHOW TENANT_ALL ALL WITH REPLICATION STATUS +| SHOW TENANT error // SHOW HELP: SHOW TENANT + +opt_show_tenant_options: + /* EMPTY */ + { $$.val = tree.ShowTenantOptions{} } +| WITH show_tenant_options + { $$.val = $2.showTenantOpts() } + +show_tenant_options: + REPLICATION STATUS { - $$.val = &tree.ShowTenant{ - TenantSpec: &tree.TenantSpec{All: true}, - WithReplication: true, - } + $$.val = tree.ShowTenantOptions{WithReplication: true} } -| SHOW TENANT tenant_spec +| CAPABILITIES { - $$.val = &tree.ShowTenant{ - TenantSpec: $3.tenantSpec(), - WithReplication: false, - } + $$.val = tree.ShowTenantOptions{WithCapabilities: true} } -| SHOW TENANT tenant_spec WITH REPLICATION STATUS +| show_tenant_options ',' REPLICATION STATUS { - $$.val = &tree.ShowTenant{ - TenantSpec: $3.tenantSpec(), - WithReplication: true, - } + o := $1.showTenantOpts() + o.WithReplication = true + $$.val = o + } +| show_tenant_options ',' CAPABILITIES + { + o := $1.showTenantOpts() + o.WithCapabilities = true + $$.val = o } -| SHOW TENANT error // SHOW HELP: SHOW TENANT // %Help: PREPARE - prepare a statement for later execution // %Category: Misc @@ -6172,10 +6205,11 @@ set_csetting_stmt: // %Help: ALTER TENANT - alter tenant configuration // %Category: Group -// %SeeAlso: ALTER TENANT REPLICATION, ALTER TENANT CLUSTER SETTING, ALTER TENANT RENAME +// %SeeAlso: ALTER TENANT REPLICATION, ALTER TENANT CLUSTER SETTING, ALTER TENANT CAPABILITY, ALTER TENANT RENAME alter_tenant_stmt: alter_tenant_replication_stmt // EXTEND WITH HELP: ALTER TENANT REPLICATION | alter_tenant_csetting_stmt // EXTEND WITH HELP: ALTER TENANT CLUSTER SETTING +| alter_tenant_capability_stmt // EXTEND WITH HELP: ALTER TENANT | alter_tenant_rename_stmt // EXTEND WITH HELP: ALTER TENANT RENAME | ALTER TENANT error // SHOW HELP: ALTER TENANT @@ -6289,6 +6323,55 @@ to_or_eq: '=' | TO +// %Help: ALTER TENANT CAPABILITY - alter tenant capability +// %Category: Group +// %Text: +// ALTER TENANT GRANT CAPABILITY { TO | = } +// ALTER TENANT REVOKE CAPABILITY +alter_tenant_capability_stmt: + ALTER TENANT tenant_spec GRANT CAPABILITY tenant_capability_list + { + /* SKIP DOC */ + $$.val = &tree.AlterTenantCapability{ + TenantSpec: $3.tenantSpec(), + Capabilities: $6.tenantCapabilities(), + } + } +| ALTER TENANT tenant_spec REVOKE CAPABILITY tenant_capability_list + { + /* SKIP DOC */ + $$.val = &tree.AlterTenantCapability{ + TenantSpec: $3.tenantSpec(), + Capabilities: $6.tenantCapabilities(), + IsRevoke: true, + } + } + +tenant_capability: + var_name + { + $$.val = tree.TenantCapability{ + Name: strings.Join($1.strs(), "."), + } + } +| var_name to_or_eq var_value + { + $$.val = tree.TenantCapability{ + Name: strings.Join($1.strs(), "."), + Value: $3.expr(), + } + } + +tenant_capability_list: + tenant_capability + { + $$.val = []tree.TenantCapability{$1.tenantCapability()} + } +| tenant_capability_list ',' tenant_capability + { + $$.val = append($1.tenantCapabilities(), $3.tenantCapability()) + } + set_exprs_internal: /* SET ROW serves to accelerate parser.parseExprs(). It cannot be used by clients. */ @@ -15750,6 +15833,8 @@ unreserved_keyword: | CALLED | CANCEL | CANCELQUERY +| CAPABILITIES +| CAPABILITY | CASCADE | CHANGEFEED | CLOSE diff --git a/pkg/sql/parser/testdata/alter_tenant b/pkg/sql/parser/testdata/alter_tenant index 88301819bda9..4c1b1979f4bb 100644 --- a/pkg/sql/parser/testdata/alter_tenant +++ b/pkg/sql/parser/testdata/alter_tenant @@ -199,3 +199,51 @@ ALTER TENANT ('string'::INTERVAL MINUTE) SET CLUSTER SETTING ident = DEFAULT -- ALTER TENANT ((('string')::INTERVAL MINUTE)) SET CLUSTER SETTING ident = (DEFAULT) -- fully parenthesized ALTER TENANT ('_'::INTERVAL MINUTE) SET CLUSTER SETTING ident = DEFAULT -- literals removed ALTER TENANT ('string'::INTERVAL MINUTE) SET CLUSTER SETTING ident = DEFAULT -- identifiers removed + +parse +ALTER TENANT 123 GRANT CAPABILITY a = 3 +---- +ALTER TENANT 123 GRANT CAPABILITY a = 3 +ALTER TENANT (123) GRANT CAPABILITY a = (3) -- fully parenthesized +ALTER TENANT _ GRANT CAPABILITY a = _ -- literals removed +ALTER TENANT 123 GRANT CAPABILITY a = 3 -- identifiers removed + +parse +ALTER TENANT (1+1) GRANT CAPABILITY a = 3 +---- +ALTER TENANT (1 + 1) GRANT CAPABILITY a = 3 -- normalized! +ALTER TENANT ((((1) + (1)))) GRANT CAPABILITY a = (3) -- fully parenthesized +ALTER TENANT (_ + _) GRANT CAPABILITY a = _ -- literals removed +ALTER TENANT (1 + 1) GRANT CAPABILITY a = 3 -- identifiers removed + +parse +ALTER TENANT $1 GRANT CAPABILITY a = 3 +---- +ALTER TENANT $1 GRANT CAPABILITY a = 3 +ALTER TENANT ($1) GRANT CAPABILITY a = (3) -- fully parenthesized +ALTER TENANT $1 GRANT CAPABILITY a = _ -- literals removed +ALTER TENANT $1 GRANT CAPABILITY a = 3 -- identifiers removed + +parse +ALTER TENANT 123 REVOKE CAPABILITY a +---- +ALTER TENANT 123 REVOKE CAPABILITY a +ALTER TENANT (123) REVOKE CAPABILITY a -- fully parenthesized +ALTER TENANT _ REVOKE CAPABILITY a -- literals removed +ALTER TENANT 123 REVOKE CAPABILITY a -- identifiers removed + +parse +ALTER TENANT (1+1) REVOKE CAPABILITY a +---- +ALTER TENANT (1 + 1) REVOKE CAPABILITY a -- normalized! +ALTER TENANT ((((1) + (1)))) REVOKE CAPABILITY a -- fully parenthesized +ALTER TENANT (_ + _) REVOKE CAPABILITY a -- literals removed +ALTER TENANT (1 + 1) REVOKE CAPABILITY a -- identifiers removed + +parse +ALTER TENANT $1 REVOKE CAPABILITY a +---- +ALTER TENANT $1 REVOKE CAPABILITY a +ALTER TENANT ($1) REVOKE CAPABILITY a -- fully parenthesized +ALTER TENANT $1 REVOKE CAPABILITY a -- literals removed +ALTER TENANT $1 REVOKE CAPABILITY a -- identifiers removed diff --git a/pkg/sql/parser/testdata/show b/pkg/sql/parser/testdata/show index e3ac675b35ce..f970707965d0 100644 --- a/pkg/sql/parser/testdata/show +++ b/pkg/sql/parser/testdata/show @@ -1956,6 +1956,14 @@ SHOW TENANT (foo) WITH REPLICATION STATUS -- fully parenthesized SHOW TENANT foo WITH REPLICATION STATUS -- literals removed SHOW TENANT _ WITH REPLICATION STATUS -- identifiers removed +parse +SHOW TENANT foo WITH CAPABILITIES +---- +SHOW TENANT foo WITH CAPABILITIES +SHOW TENANT (foo) WITH CAPABILITIES -- fully parenthesized +SHOW TENANT foo WITH CAPABILITIES -- literals removed +SHOW TENANT _ WITH CAPABILITIES -- identifiers removed + parse SHOW TENANTS ---- diff --git a/pkg/sql/sem/tree/alter_tenant.go b/pkg/sql/sem/tree/alter_tenant.go index 66ff0ea47ee9..f4e5068284d0 100644 --- a/pkg/sql/sem/tree/alter_tenant.go +++ b/pkg/sql/sem/tree/alter_tenant.go @@ -10,6 +10,8 @@ package tree +import "github.com/cockroachdb/errors" + // ReplicationCutoverTime represent the user-specified cutover time type ReplicationCutoverTime struct { Timestamp Expr @@ -48,6 +50,57 @@ func (n *AlterTenantReplication) Format(ctx *FmtCtx) { } } +// TenantCapability is a key-value parameter representing a tenant capability. +type TenantCapability struct { + Name string + Value Expr +} + +func (c *TenantCapability) GetBoolValue(isRevoke bool) (bool, error) { + if c.Value == nil { + return false, nil + } + if isRevoke { + return false, errors.New("revoke must not specify value") + } + dBool, ok := AsDBool(c.Value) + if !ok { + return false, errors.New("value must be bool") + } + return bool(dBool), nil +} + +// AlterTenantCapability represents an ALTER TENANT CAPABILITY statement. +type AlterTenantCapability struct { + TenantSpec *TenantSpec + Capabilities []TenantCapability + IsRevoke bool +} + +var _ Statement = &AlterTenantCapability{} + +// Format implements the NodeFormatter interface. +func (n *AlterTenantCapability) Format(ctx *FmtCtx) { + ctx.WriteString("ALTER TENANT ") + ctx.FormatNode(n.TenantSpec) + if n.IsRevoke { + ctx.WriteString(" REVOKE CAPABILITY ") + } else { + ctx.WriteString(" GRANT CAPABILITY ") + } + for i, capability := range n.Capabilities { + if i > 0 { + ctx.WriteString(", ") + } + ctx.WriteString(capability.Name) + value := capability.Value + if value != nil { + ctx.WriteString(" = ") + ctx.FormatNode(value) + } + } +} + // TenantSpec designates a tenant for the ALTER TENANT statements. type TenantSpec struct { Expr Expr diff --git a/pkg/sql/sem/tree/show.go b/pkg/sql/sem/tree/show.go index 438289f7fafe..a4677a8271e7 100644 --- a/pkg/sql/sem/tree/show.go +++ b/pkg/sql/sem/tree/show.go @@ -860,10 +860,16 @@ func (node *ShowTableStats) Format(ctx *FmtCtx) { } } +// ShowTenantOptions represents the WITH clause in SHOW TENANT. +type ShowTenantOptions struct { + WithReplication bool + WithCapabilities bool +} + // ShowTenant represents a SHOW TENANT statement. type ShowTenant struct { - TenantSpec *TenantSpec - WithReplication bool + TenantSpec *TenantSpec + ShowTenantOptions } // Format implements the NodeFormatter interface. @@ -874,6 +880,10 @@ func (node *ShowTenant) Format(ctx *FmtCtx) { if node.WithReplication { ctx.WriteString(" WITH REPLICATION STATUS") } + + if node.WithCapabilities { + ctx.WriteString(" WITH CAPABILITIES") + } } // ShowHistogram represents a SHOW HISTOGRAM statement. diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index 6c6c35f718b5..fa852363e087 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -440,6 +440,15 @@ func (*AlterSchema) StatementTag() string { return "ALTER SCHEMA" } func (*AlterSchema) hiddenFromShowQueries() {} +// StatementReturnType implements the Statement interface. +func (*AlterTenantCapability) StatementReturnType() StatementReturnType { return Rows } + +// StatementType implements the Statement interface. +func (*AlterTenantCapability) StatementType() StatementType { return TypeDML } + +// StatementTag returns a short string identifying the type of statement. +func (*AlterTenantCapability) StatementTag() string { return "ALTER TENANT CAPABILITY" } + // StatementReturnType implements the Statement interface. func (*AlterTenantSetClusterSetting) StatementReturnType() StatementReturnType { return Ack } @@ -2111,6 +2120,7 @@ func (n *AlterTableSetVisible) String() string { return AsString( func (n *AlterTableSetNotNull) String() string { return AsString(n) } func (n *AlterTableOwner) String() string { return AsString(n) } func (n *AlterTableSetSchema) String() string { return AsString(n) } +func (n *AlterTenantCapability) String() string { return AsString(n) } func (n *AlterTenantSetClusterSetting) String() string { return AsString(n) } func (n *AlterTenantRename) String() string { return AsString(n) } func (n *AlterTenantReplication) String() string { return AsString(n) } diff --git a/pkg/sql/sem/tree/walk.go b/pkg/sql/sem/tree/walk.go index ecd862a67dbb..55fd33f0125a 100644 --- a/pkg/sql/sem/tree/walk.go +++ b/pkg/sql/sem/tree/walk.go @@ -948,6 +948,32 @@ func (n *ShowTenantClusterSettingList) walkStmt(v Visitor) Statement { return ret } +func (n *AlterTenantCapability) walkStmt(v Visitor) Statement { + ret := n + copyNodeOnce := func() { + if ret == n { + stmtCopy := *n + ret = &stmtCopy + } + } + for i, capability := range n.Capabilities { + value := capability.Value + if value != nil { + e, changed := WalkExpr(v, value) + if changed { + copyNodeOnce() + ret.Capabilities[i].Value = e + } + } + } + ts, changed := walkTenantSpec(v, n.TenantSpec) + if changed { + copyNodeOnce() + ret.TenantSpec = ts + } + return ret +} + // copyNode makes a copy of this Statement without recursing in any child Statements. func (n *AlterTenantSetClusterSetting) copyNode() *AlterTenantSetClusterSetting { stmtCopy := *n @@ -1823,6 +1849,7 @@ func (stmt *BeginTransaction) walkStmt(v Visitor) Statement { var _ walkableStmt = &AlterTenantRename{} var _ walkableStmt = &AlterTenantReplication{} +var _ walkableStmt = &AlterTenantCapability{} var _ walkableStmt = &AlterTenantSetClusterSetting{} var _ walkableStmt = &Backup{} var _ walkableStmt = &BeginTransaction{} diff --git a/pkg/sql/show_tenant.go b/pkg/sql/show_tenant.go index fd087c95a231..c5d617a01344 100644 --- a/pkg/sql/show_tenant.go +++ b/pkg/sql/show_tenant.go @@ -13,6 +13,7 @@ package sql import ( "context" "fmt" + "strconv" "time" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -44,15 +45,25 @@ type tenantValues struct { tenantStatus tenantStatus replicationInfo *streampb.StreamIngestionStats protectedTimestamp hlc.Timestamp + capabilities []showTenantNodeCapability +} + +type showTenantNodeCapability struct { + name string + value string } type showTenantNode struct { - tenantSpec tenantSpec - withReplication bool - columns colinfo.ResultColumns - row int - tenantIds []roachpb.TenantID - values *tenantValues + tenantSpec tenantSpec + withReplication bool + withCapabilities bool + columns colinfo.ResultColumns + tenantIDIndex int + tenantIds []roachpb.TenantID + initTenantValues bool + values *tenantValues + capabilityIndex int + capability showTenantNodeCapability } // ShowTenant constructs a showTenantNode. @@ -70,14 +81,19 @@ func (p *planner) ShowTenant(ctx context.Context, n *tree.ShowTenant) (planNode, } node := &showTenantNode{ - tenantSpec: tspec, - withReplication: n.WithReplication, + tenantSpec: tspec, + withReplication: n.WithReplication, + withCapabilities: n.WithCapabilities, + initTenantValues: true, } node.columns = colinfo.TenantColumns if n.WithReplication { node.columns = append(node.columns, colinfo.TenantColumnsWithReplication...) } + if n.WithCapabilities { + node.columns = append(node.columns, colinfo.TenantColumnsWithCapabilities...) + } return node, nil } @@ -168,67 +184,106 @@ func getTenantStatus( func (n *showTenantNode) getTenantValues( params runParams, tenantInfo *descpb.TenantInfo, ) (*tenantValues, error) { + + // Common fields. var values tenantValues values.tenantInfo = tenantInfo - jobId := values.tenantInfo.TenantReplicationJobID + + // Tenant status + replication status fields. + jobId := tenantInfo.TenantReplicationJobID + tenantInfoState := tenantInfo.State if jobId == 0 { // No replication job, this is a non-replicating tenant. if n.withReplication { return nil, errors.Newf("tenant %q does not have an active replication job", tenantInfo.Name) } - values.tenantStatus = tenantStatus(values.tenantInfo.State.String()) - return &values, nil + values.tenantStatus = tenantStatus(tenantInfoState.String()) + } else { + switch tenantInfoState { + case descpb.TenantInfo_ADD: + // There is a replication job, we need to get the job info and the + // replication stats in order to generate the exact tenant status. + ctx := params.ctx + p := params.p + registry := p.execCfg.JobRegistry + job, err := registry.LoadJobWithTxn(ctx, jobId, p.InternalSQLTxn()) + if err != nil { + log.Errorf(ctx, "cannot load job info for replicated tenant %q and job %d: %v", + tenantInfo.Name, jobId, err) + values.tenantStatus = tenantStatus(fmt.Sprintf(string(replicationUnknownFormat), err)) + } else { + stats, protectedTimestamp, err := getReplicationStats(params, job) + if err != nil { + log.Errorf(ctx, "cannot load replication stats for replicated tenant %q and job %d: %v", + tenantInfo.Name, jobId, err) + values.tenantStatus = tenantStatus(fmt.Sprintf(string(replicationUnknownFormat), err)) + } else { + if n.withReplication { + values.replicationInfo = stats + if protectedTimestamp != nil { + values.protectedTimestamp = *protectedTimestamp + } + } + values.tenantStatus = getTenantStatus(job.Status(), stats) + } + } + case descpb.TenantInfo_ACTIVE, descpb.TenantInfo_DROP: + values.tenantStatus = tenantStatus(tenantInfoState.String()) + default: + return nil, errors.Newf("tenant %q state is unknown: %s", tenantInfo.Name, tenantInfoState.String()) + } } - switch values.tenantInfo.State { - case descpb.TenantInfo_ADD: - // There is a replication job, we need to get the job info and the - // replication stats in order to generate the exact tenant status. - registry := params.p.execCfg.JobRegistry - job, err := registry.LoadJobWithTxn(params.ctx, jobId, params.p.InternalSQLTxn()) - if err != nil { - log.Errorf(params.ctx, "cannot load job info for replicated tenant %q and job %d: %v", - tenantInfo.Name, jobId, err) - values.tenantStatus = tenantStatus(fmt.Sprintf(string(replicationUnknownFormat), err)) - return &values, nil - } - stats, protectedTimestamp, err := getReplicationStats(params, job) - if err != nil { - log.Errorf(params.ctx, "cannot load replication stats for replicated tenant %q and job %d: %v", - tenantInfo.Name, jobId, err) - values.tenantStatus = tenantStatus(fmt.Sprintf(string(replicationUnknownFormat), err)) - return &values, nil - } - values.replicationInfo = stats - if protectedTimestamp != nil { - values.protectedTimestamp = *protectedTimestamp + if n.withCapabilities { + capabilities := tenantInfo.Capabilities + values.capabilities = []showTenantNodeCapability{ + { + name: CanAdminSplitCapabilityName, + value: strconv.FormatBool(capabilities.CanAdminSplit), + }, + { + name: CanAdminUnsplitCapabilityName, + value: strconv.FormatBool(false), + }, } - - values.tenantStatus = getTenantStatus(job.Status(), values.replicationInfo) - case descpb.TenantInfo_ACTIVE, descpb.TenantInfo_DROP: - values.tenantStatus = tenantStatus(values.tenantInfo.State.String()) - default: - return nil, errors.Newf("tenant %q state is unknown: %s", tenantInfo.Name, values.tenantInfo.State.String()) } + return &values, nil } func (n *showTenantNode) Next(params runParams) (bool, error) { - if n.row >= len(n.tenantIds) { + if n.tenantIDIndex >= len(n.tenantIds) { return false, nil } - tenantInfo, err := GetTenantRecordByID(params.ctx, params.p.InternalSQLTxn(), n.tenantIds[n.row]) - if err != nil { - return false, err + if n.initTenantValues { + tenantInfo, err := GetTenantRecordByID(params.ctx, params.p.InternalSQLTxn(), n.tenantIds[n.tenantIDIndex]) + if err != nil { + return false, err + } + values, err := n.getTenantValues(params, tenantInfo) + if err != nil { + return false, err + } + n.values = values + n.initTenantValues = false } - values, err := n.getTenantValues(params, tenantInfo) - if err != nil { - return false, err + if n.withCapabilities { + capabilities := n.values.capabilities + n.capability = capabilities[n.capabilityIndex] + if n.capabilityIndex == len(capabilities)-1 { + n.capabilityIndex = 0 + n.tenantIDIndex++ + n.initTenantValues = true + } else { + n.capabilityIndex++ + } + } else { + n.tenantIDIndex++ + n.initTenantValues = true } - n.values = values - n.row++ + return true, nil } @@ -286,6 +341,14 @@ func (n *showTenantNode) Values() tree.Datums { ) } + if n.withCapabilities { + capability := n.capability + result = append(result, + tree.NewDString(capability.name), + tree.NewDString(capability.value), + ) + } + return result } diff --git a/pkg/sql/tenant_capability.go b/pkg/sql/tenant_capability.go new file mode 100644 index 000000000000..5951c05e6506 --- /dev/null +++ b/pkg/sql/tenant_capability.go @@ -0,0 +1,84 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/errors" +) + +const CanAdminSplitCapabilityName = "can_admin_split" +const CanAdminUnsplitCapabilityName = "can_admin_unsplit" + +type alterTenantCapabilityNode struct { + *tree.AlterTenantCapability +} + +func (p *planner) AlterTenantCapability( + _ context.Context, n *tree.AlterTenantCapability, +) (planNode, error) { + return &alterTenantCapabilityNode{ + AlterTenantCapability: n, + }, nil +} + +func (n *alterTenantCapabilityNode) startExec(params runParams) error { + const op = "ALTER TENANT CAPABILITY" + execCfg := params.ExecCfg() + if err := rejectIfCantCoordinateMultiTenancy(execCfg.Codec, op); err != nil { + return err + } + planner := params.p + ctx := params.ctx + tSpec, err := planner.planTenantSpec(ctx, n.TenantSpec, op) + if err != nil { + return err + } + tenantInfo, err := tSpec.getTenantInfo(ctx, planner) + if err != nil { + return err + } + if err := rejectIfSystemTenant(tenantInfo.ID, op); err != nil { + return err + } + isRevoke := n.IsRevoke + capabilities := &tenantInfo.Capabilities + for _, capability := range n.Capabilities { + capabilityName := capability.Name + switch capabilityName { + case CanAdminSplitCapabilityName: + capabilities.CanAdminSplit, err = capability.GetBoolValue(isRevoke) + default: + err = errors.Newf("invalid capability") + } + if err != nil { + return pgerror.Wrapf( + err, + pgcode.InvalidParameterValue, + "error parsing capability %q", + capabilityName, + ) + } + } + + if err := UpdateTenantRecord(params.ctx, execCfg.Settings, planner.InternalSQLTxn(), tenantInfo); err != nil { + return err + } + return nil +} + +func (n *alterTenantCapabilityNode) Next(runParams) (bool, error) { return false, nil } +func (n *alterTenantCapabilityNode) Values() tree.Datums { return nil } +func (n *alterTenantCapabilityNode) Close(context.Context) {} diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index acf57492ce24..2c07294bf95e 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -215,6 +215,7 @@ func (v *planVisitor) visitInternal(plan planNode, name string) { n.sourcePlan = v.visit(n.sourcePlan) } + case *alterTenantCapabilityNode: case *alterTenantSetClusterSettingNode: case *createViewNode: case *setVarNode: @@ -360,6 +361,7 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&alterTableOwnerNode{}): "alter table owner", reflect.TypeOf(&alterTableSetLocalityNode{}): "alter table set locality", reflect.TypeOf(&alterTableSetSchemaNode{}): "alter table set schema", + reflect.TypeOf(&alterTenantCapabilityNode{}): "alter tenant capability", reflect.TypeOf(&alterTenantSetClusterSettingNode{}): "alter tenant set cluster setting", reflect.TypeOf(&alterTypeNode{}): "alter type", reflect.TypeOf(&alterRoleNode{}): "alter role",