Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
95631: schemachanger: generalized statement-time execution r=postamar a=postamar

This change makes it possible to plan and execute the following change
correctly:

    BEGIN;
    DROP TABLE foo;
    CREATE UNIQUE INDEX idx ON bar (x);
    COMMIT;

Inside the transaction, the table is seen as dropped right after the
DROP TABLE statement executes, but the table is not seen as dropped by
other transactions until the new, unrelated index has been validated, at
which point the schema change can no longer fail.

Fixes #88294.

Significant changes:
- descs.Collection has a new ResetUncommitted method.
- scstage.BuildStages is rewritten with unrolled loops and a new scheme:
  statement phase has revertible op-edges, pre-commit has two stages,
  one which resets the uncommitted state by scheduling the new
  scop.UndoAllInTxnImmediateMutationOpSideEffects op, which triggers a call
  to ResetUncommitted.
- This happens in scexec.executeMutationOps, which is now split into two
  pieces, the first one which executes ops whose side-effects can be undone
  by ResetUncommitted, and the other which handles those which can’t.
- These ops implement scop.ImmediateMutationOp and scop.DeferredMutationOp
  respectively.
- These have their own visitors with their own state and their own
  dependencies.

This all means that we can handle DROPs properly now: the TXN_DROPPED element
state is removed as are all the synthetic descriptor manipulations (outside of
index and constraint validation, that is). Furthermore, the
scgraph.PreviousTransactionPrecedence edge kind no longer serves any purpose
and has been replaced by PreviousStagePrecedence everywhere.

There’s a bunch of other more-or-less related changes, mostly as a consequence
to the above:
- scbuild.Build must produce a tighter target set: no-op targets are
  elided.
- nstree.MutableCatalog has new methods for deleting comments and zone configs
  which are useful.
- sctestdeps.TestState has better storage layers modelling
  (in-memory > stored > committed)
- Added missing “relation dropped before column|index no longer public” rules
  which fix DROP COLUMN CASCADE when the column is referenced in a view.
- scgraph.Graph has a new GetOpEdgeTo method which is useful.
- added debug info to assertion error messages in scstage.BuildStages to make
  debugging easier during development.
- EXPLAIN (DDL) output has a nicer root node label which puts the last
  statement first.

Release note: None

95636: sql: increase the online help for SHOW RANGES r=ecwall a=knz

The output of `\h SHOW RANGES` wasn't mentioning the KEYS option. Now it does. Also this adds some additional details about the various options.

Release note: None
Epic: None

95691: sql: improve tenant records r=stevendanna,dt,ajwerner a=knz

supersedes #95574, #95581, #95655
Epic: CRDB-21836

**TLDR** This commit contains the following changes:
- rename "state" to "data_state", "active" to "ready"
- stored, non-virtual columns for "name", "data_state", "service_mode"
- deprecate the column "active" since it mirrors "data_state'
- move `descpb.TenantInfo` to new package `mtinfopb`.
- new statements `ALTER TENANT ... START SERVICE EXTERNAL/SHARED`,
  `STOP SERVICE` to change the service mode.

Details follow.

**rename TenantInfo.State to DataState, "ACTIVE" to "READY"**

We've discovered that we'd like to separate the readiness of the data
from the activation of the service. To emphasize this, this commit
renames the field "State" to "DataState".

Additionally, the state "ACTIVE" was confusing as it suggests that
something is running, whereas it merely marks the tenant data as ready
for use. So this commit also renames that state accordingly.

**new tenant info field ServiceMode**

Summary of changes:
- the new TenantInfo.ServiceMode field indicates how to run servers.
- new syntax: `ALTER TENANT ... START SERVICE EXTERNAL/SHARED`,
  `ALTER TENANT ... STOP SERVICE`.
- tenants created via `create_tenant(<id>)`
  (via CC serverless control plane) start in service mode EXTERNAL.
- other tenants start in service mode NONE.
- need ALTER TENANT STOP SERVICE before dropping a tenant.
  - except in the case of `crdb_internal.destroy_tenant`
    for compat with CC serverless control plane.

**make the output columns of SHOW TENANT lowercase**

All the SHOW statements report status-like data in lowercase.
SHOW TENANT(s) should not be different.

**use actual SQL columns for the TenantInfo fields**

Release note: None

Co-authored-by: Marius Posta <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
3 people committed Jan 26, 2023
4 parents 9caf758 + 083e17a + 1040ff4 + 5fcce46 commit 6819d87
Show file tree
Hide file tree
Showing 397 changed files with 20,439 additions and 6,247 deletions.
3 changes: 2 additions & 1 deletion build/bazelutil/check.sh
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,8 @@ pkg/sql/opt/optgen/lang/gen.go://go:generate langgen -out expr.og.go exprs lang.
pkg/sql/opt/optgen/lang/gen.go://go:generate langgen -out operator.og.go ops lang.opt
pkg/sql/schemachanger/scexec/exec_backfill_test.go://go:generate mockgen -package scexec_test -destination=mocks_generated_test.go --self_package scexec . Catalog,Dependencies,Backfiller,Merger,BackfillerTracker,IndexSpanSplitter,PeriodicProgressFlusher
pkg/sql/schemachanger/scop/backfill.go://go:generate go run ./generate_visitor.go scop Backfill backfill.go backfill_visitor_generated.go
pkg/sql/schemachanger/scop/mutation.go://go:generate go run ./generate_visitor.go scop Mutation mutation.go mutation_visitor_generated.go
pkg/sql/schemachanger/scop/immediate_mutation.go://go:generate go run ./generate_visitor.go scop ImmediateMutation immediate_mutation.go immediate_mutation_visitor_generated.go
pkg/sql/schemachanger/scop/deferred_mutation.go://go:generate go run ./generate_visitor.go scop DeferredMutation deferred_mutation.go deferred_mutation_visitor_generated.go
pkg/sql/schemachanger/scop/validation.go://go:generate go run ./generate_visitor.go scop Validation validation.go validation_visitor_generated.go
pkg/sql/schemachanger/scpb/state.go://go:generate go run element_generator.go --in elements.proto --out elements_generated.go
pkg/sql/schemachanger/scpb/state.go://go:generate go run element_uml_generator.go --out uml/table.puml
Expand Down
1 change: 1 addition & 0 deletions docs/generated/http/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ genrule(
"//pkg/kv/kvserver/liveness/livenesspb:livenesspb_proto",
"//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb_proto",
"//pkg/kv/kvserver/readsummary/rspb:rspb_proto",
"//pkg/multitenant/mtinfopb:mtinfopb_proto",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb:tenantcapabilitiespb_proto",
"//pkg/roachpb:roachpb_proto",
"//pkg/server/diagnostics/diagnosticspb:diagnosticspb_proto",
Expand Down
3 changes: 3 additions & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -1337,11 +1337,13 @@ unreserved_keyword ::=
| 'SEQUENCE'
| 'SEQUENCES'
| 'SERVER'
| 'SERVICE'
| 'SESSION'
| 'SESSIONS'
| 'SET'
| 'SETS'
| 'SHARE'
| 'SHARED'
| 'SHOW'
| 'SIMPLE'
| 'SKIP'
Expand All @@ -1360,6 +1362,7 @@ unreserved_keyword ::=
| 'STATEMENTS'
| 'STATISTICS'
| 'STDIN'
| 'STOP'
| 'STORAGE'
| 'STORE'
| 'STORED'
Expand Down
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -1283,6 +1283,7 @@ GO_TARGETS = [
"//pkg/kv/kvserver:kvserver_test",
"//pkg/kv:kv",
"//pkg/kv:kv_test",
"//pkg/multitenant/mtinfopb:mtinfopb",
"//pkg/multitenant/multitenantcpu:multitenantcpu",
"//pkg/multitenant/multitenantio:multitenantio",
"//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer:tenantcapabilitiesauthorizer",
Expand Down Expand Up @@ -2645,6 +2646,7 @@ GET_X_DATA_TARGETS = [
"//pkg/kv/kvserver/txnwait:get_x_data",
"//pkg/kv/kvserver/uncertainty:get_x_data",
"//pkg/multitenant:get_x_data",
"//pkg/multitenant/mtinfopb:get_x_data",
"//pkg/multitenant/multitenantcpu:get_x_data",
"//pkg/multitenant/multitenantio:get_x_data",
"//pkg/multitenant/tenantcapabilities:get_x_data",
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@ go_library(
"//pkg/kv/kvserver/concurrency/lock",
"//pkg/kv/kvserver/protectedts",
"//pkg/kv/kvserver/protectedts/ptpb",
"//pkg/multitenant/mtinfopb",
"//pkg/roachpb",
"//pkg/scheduledjobs",
"//pkg/scheduledjobs/schedulebase",
Expand Down Expand Up @@ -229,6 +230,7 @@ go_test(
"//pkg/kv/kvserver/protectedts",
"//pkg/kv/kvserver/protectedts/ptpb",
"//pkg/kv/kvserver/protectedts/ptutil",
"//pkg/multitenant/mtinfopb",
"//pkg/roachpb",
"//pkg/scheduledjobs",
"//pkg/scheduledjobs/schedulebase",
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/backupccl/backup_metadata_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb"
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuptestutils"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql"
Expand Down Expand Up @@ -246,8 +247,8 @@ func checkIntroducedSpans(
func checkTenants(
ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata,
) {
var metaTenants []descpb.TenantInfoWithUsage
var tenant descpb.TenantInfoWithUsage
var metaTenants []mtinfopb.TenantInfoWithUsage
var tenant mtinfopb.TenantInfoWithUsage
it := bm.TenantIter(ctx)
defer it.Close()

Expand Down
20 changes: 13 additions & 7 deletions pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,10 +32,12 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb"
"github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/scheduledjobs"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
Expand Down Expand Up @@ -1324,22 +1326,26 @@ func checkForNewTables(
}

func getTenantInfo(
ctx context.Context, codec keys.SQLCodec, txn isql.Txn, jobDetails jobspb.BackupDetails,
) ([]roachpb.Span, []descpb.TenantInfoWithUsage, error) {
ctx context.Context,
settings *cluster.Settings,
codec keys.SQLCodec,
txn isql.Txn,
jobDetails jobspb.BackupDetails,
) ([]roachpb.Span, []mtinfopb.TenantInfoWithUsage, error) {
var spans []roachpb.Span
var tenants []descpb.TenantInfoWithUsage
var tenants []mtinfopb.TenantInfoWithUsage
var err error
if jobDetails.FullCluster && codec.ForSystemTenant() {
// Include all tenants.
tenants, err = retrieveAllTenantsMetadata(
ctx, txn,
ctx, txn, settings,
)
if err != nil {
return nil, nil, err
}
} else if len(jobDetails.SpecificTenantIds) > 0 {
for _, id := range jobDetails.SpecificTenantIds {
tenantInfo, err := retrieveSingleTenantMetadata(ctx, txn, id)
tenantInfo, err := retrieveSingleTenantMetadata(ctx, txn, id, settings)
if err != nil {
return nil, nil, err
}
Expand Down Expand Up @@ -1422,9 +1428,9 @@ func createBackupManifest(
}

var spans []roachpb.Span
var tenants []descpb.TenantInfoWithUsage
var tenants []mtinfopb.TenantInfoWithUsage
tenantSpans, tenantInfos, err := getTenantInfo(
ctx, execCfg.Codec, txn, jobDetails,
ctx, execCfg.Settings, execCfg.Codec, txn, jobDetails,
)
if err != nil {
return backuppb.BackupManifest{}, err
Expand Down
130 changes: 93 additions & 37 deletions pkg/ccl/backupccl/backup_planning_tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,105 +11,161 @@ package backupccl
import (
"context"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
)

const tenantMetadataQuery = `
func tenantMetadataQuery(ctx context.Context, settings *cluster.Settings) string {
q := `
SELECT
tenants.id, /* 0 */
tenants.active, /* 1 */
tenants.info, /* 2 */
tenant_usage.ru_burst_limit, /* 3 */
tenant_usage.ru_refill_rate, /* 4 */
tenant_usage.ru_current, /* 5 */
tenant_usage.total_consumption /* 6 */
tenants.info, /* 1 */
tenants.name, /* 2 */
tenants.data_state, /* 3 */
tenants.service_mode, /* 4 */
tenant_usage.ru_burst_limit, /* 5 */
tenant_usage.ru_refill_rate, /* 6 */
tenant_usage.ru_current, /* 7 */
tenant_usage.total_consumption /* 8 */
FROM
system.tenants
LEFT JOIN system.tenant_usage ON
tenants.id = tenant_usage.tenant_id AND tenant_usage.instance_id = 0
`
if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) {
q = `
SELECT
tenants.id, /* 0 */
tenants.info, /* 1 */
NULL, /* 2 */
NULL, /* 3 */
NULL, /* 4 */
tenant_usage.ru_burst_limit, /* 5 */
tenant_usage.ru_refill_rate, /* 6 */
tenant_usage.ru_current, /* 7 */
tenant_usage.total_consumption /* 8 */
FROM
system.tenants
LEFT JOIN system.tenant_usage ON
tenants.id = tenant_usage.tenant_id AND tenant_usage.instance_id = 0`
}
return q
}

func tenantMetadataFromRow(row tree.Datums) (descpb.TenantInfoWithUsage, error) {
if len(row) != 7 {
return descpb.TenantInfoWithUsage{}, errors.AssertionFailedf(
func tenantMetadataFromRow(row tree.Datums) (mtinfopb.TenantInfoWithUsage, error) {
if len(row) != 9 {
return mtinfopb.TenantInfoWithUsage{}, errors.AssertionFailedf(
"unexpected row size %d from tenant metadata query", len(row),
)
}

id := uint64(tree.MustBeDInt(row[0]))
res := descpb.TenantInfoWithUsage{
TenantInfo: descpb.TenantInfo{
res := mtinfopb.TenantInfoWithUsage{
ProtoInfo: mtinfopb.ProtoInfo{
// for compatibility
DeprecatedID: id,
},
SQLInfo: mtinfopb.SQLInfo{
ID: id,
},
}
infoBytes := []byte(tree.MustBeDBytes(row[2]))
if err := protoutil.Unmarshal(infoBytes, &res.TenantInfo); err != nil {
return descpb.TenantInfoWithUsage{}, err
infoBytes := []byte(tree.MustBeDBytes(row[1]))
if err := protoutil.Unmarshal(infoBytes, &res.ProtoInfo); err != nil {
return mtinfopb.TenantInfoWithUsage{}, err
}
if row[2] != tree.DNull {
res.Name = roachpb.TenantName(tree.MustBeDString(row[2]))
}
if row[3] != tree.DNull {
res.DataState = mtinfopb.TenantDataState(tree.MustBeDInt(row[3]))
} else {
// Pre-v23.1 info struct.
switch res.ProtoInfo.DeprecatedDataState {
case mtinfopb.ProtoInfo_READY:
res.DataState = mtinfopb.DataStateReady
case mtinfopb.ProtoInfo_ADD:
res.DataState = mtinfopb.DataStateAdd
case mtinfopb.ProtoInfo_DROP:
res.DataState = mtinfopb.DataStateDrop
default:
return res, errors.AssertionFailedf("unhandled: %d", res.ProtoInfo.DeprecatedDataState)
}
}
res.ServiceMode = mtinfopb.ServiceModeNone
if row[4] != tree.DNull {
res.ServiceMode = mtinfopb.TenantServiceMode(tree.MustBeDInt(row[4]))
} else if res.DataState == mtinfopb.DataStateReady {
// Records created for CC Serverless pre-v23.1.
res.ServiceMode = mtinfopb.ServiceModeExternal
}
// If this tenant had no reported consumption and its token bucket was not
// configured, the tenant_usage values are all NULL.
//
// It should be sufficient to check any one value, but we check all of them
// just to be defensive (in case the table contains invalid data).
for _, d := range row[3:5] {
for _, d := range row[5:] {
if d == tree.DNull {
return res, nil
}
}
res.Usage = &descpb.TenantInfoWithUsage_Usage{
RUBurstLimit: float64(tree.MustBeDFloat(row[3])),
RURefillRate: float64(tree.MustBeDFloat(row[4])),
RUCurrent: float64(tree.MustBeDFloat(row[5])),
res.Usage = &mtinfopb.UsageInfo{
RUBurstLimit: float64(tree.MustBeDFloat(row[5])),
RURefillRate: float64(tree.MustBeDFloat(row[6])),
RUCurrent: float64(tree.MustBeDFloat(row[7])),
}
if row[6] != tree.DNull {
consumptionBytes := []byte(tree.MustBeDBytes(row[6]))
if row[8] != tree.DNull {
consumptionBytes := []byte(tree.MustBeDBytes(row[8]))
if err := protoutil.Unmarshal(consumptionBytes, &res.Usage.Consumption); err != nil {
return descpb.TenantInfoWithUsage{}, err
return mtinfopb.TenantInfoWithUsage{}, err
}
}
return res, nil
}

func retrieveSingleTenantMetadata(
ctx context.Context, txn isql.Txn, tenantID roachpb.TenantID,
) (descpb.TenantInfoWithUsage, error) {
ctx context.Context, txn isql.Txn, tenantID roachpb.TenantID, settings *cluster.Settings,
) (mtinfopb.TenantInfoWithUsage, error) {
row, err := txn.QueryRow(
ctx, "backupccl.retrieveSingleTenantMetadata", txn.KV(),
tenantMetadataQuery+` WHERE id = $1`, tenantID.ToUint64(),
tenantMetadataQuery(ctx, settings)+` WHERE id = $1`, tenantID.ToUint64(),
)
if err != nil {
return descpb.TenantInfoWithUsage{}, err
return mtinfopb.TenantInfoWithUsage{}, err
}
if row == nil {
return descpb.TenantInfoWithUsage{}, errors.Errorf("tenant %s does not exist", tenantID)
return mtinfopb.TenantInfoWithUsage{}, errors.Errorf("tenant %s does not exist", tenantID)
}
if !tree.MustBeDBool(row[1]) {
return descpb.TenantInfoWithUsage{}, errors.Errorf("tenant %s is not active", tenantID)
info, err := tenantMetadataFromRow(row)
if err != nil {
return mtinfopb.TenantInfoWithUsage{}, err
}

return tenantMetadataFromRow(row)
if info.DataState != mtinfopb.DataStateReady {
return mtinfopb.TenantInfoWithUsage{}, errors.Errorf("tenant %s is not active", tenantID)
}
return info, nil
}

func retrieveAllTenantsMetadata(
ctx context.Context, txn isql.Txn,
) ([]descpb.TenantInfoWithUsage, error) {
ctx context.Context, txn isql.Txn, settings *cluster.Settings,
) ([]mtinfopb.TenantInfoWithUsage, error) {
rows, err := txn.QueryBuffered(
ctx, "backupccl.retrieveAllTenantsMetadata", txn.KV(),
// TODO(?): Should we add a `WHERE active`? We require the tenant to be active
// when it is specified.
// See: https://github.com/cockroachdb/cockroach/issues/89997
tenantMetadataQuery+` WHERE id != $1`,
tenantMetadataQuery(ctx, settings)+` WHERE id != $1`,
roachpb.SystemTenantID.ToUint64(),
)
if err != nil {
return nil, err
}
res := make([]descpb.TenantInfoWithUsage, len(rows))
res := make([]mtinfopb.TenantInfoWithUsage, len(rows))
for i := range rows {
res[i], err = tenantMetadataFromRow(rows[i])
if err != nil {
Expand Down
Loading

0 comments on commit 6819d87

Please sign in to comment.