From f8a9d4094a300ca317f03aa49018b7b66272aab1 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Thu, 23 Feb 2023 12:58:28 +0100 Subject: [PATCH 1/4] kvserver: assert uniqueness in registerProposalLocked We routinely overwrite entries in the `r.mu.proposals` map. That is "fine" (better if we didn't, but currently it is by design - it happens in refreshProposalsLocked and during tryReproposeWithNewLeaseIndex) but our overwrites should be no-ops, i.e. reference the exact same `*ProposalData`. This is now asserted. One way this would trip is a CmdID collision. Epic: none Release note: None --- pkg/kv/kvserver/replica_proposal_buf.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 2a5de9ba2e49..5b477d257bb8 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -1192,6 +1192,9 @@ func (rp *replicaProposer) registerProposalLocked(p *ProposalData) { if buildutil.CrdbTestBuild && (p.ec.repl == nil || p.ec.g == nil) { log.Fatalf(rp.store.AnnotateCtx(context.Background()), "finished proposal inserted into map: %+v", p) } + if prev := rp.mu.proposals[p.idKey]; prev != nil && prev != p { + log.Fatalf(rp.store.AnnotateCtx(context.Background()), "two proposals under same ID:\n%+v,\n%+v", prev, p) + } rp.mu.proposals[p.idKey] = p } From f95866d3c8df8e0d3ff58369c98dc007066d95df Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Fri, 24 Feb 2023 08:38:40 +0100 Subject: [PATCH 2/4] kvserver: disable assertion 'finished proposal inserted' Touches #97605. Epic: none Release note: None --- pkg/kv/kvserver/replica_proposal_buf.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 2a5de9ba2e49..0480b41996e5 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -1189,7 +1189,10 @@ func (rp *replicaProposer) registerProposalLocked(p *ProposalData) { if p.createdAtTicks == 0 { p.createdAtTicks = rp.mu.ticks } - if buildutil.CrdbTestBuild && (p.ec.repl == nil || p.ec.g == nil) { + // TODO(tbg): this assertion fires. Figure out why. See: + // https://github.com/cockroachdb/cockroach/issues/97605 + const enableAssertion = false + if enableAssertion && buildutil.CrdbTestBuild && (p.ec.repl == nil || p.ec.g == nil) { log.Fatalf(rp.store.AnnotateCtx(context.Background()), "finished proposal inserted into map: %+v", p) } rp.mu.proposals[p.idKey] = p From 304e0b48fa8e3b427d7229ad6973ae89ae50f1d6 Mon Sep 17 00:00:00 2001 From: Renato Costa Date: Tue, 24 Jan 2023 17:03:25 -0500 Subject: [PATCH 3/4] roachtest: improvements to `mixedversion` package This commit introduces a few improvements to the `mixedversion` package, the recently introduced framework for mixed-version (and mixed-binary) roachtests. Specifically, the following improvements are made: * Removal of `DBNode()` function: having the planner pick a database that the individual steps will connect to is insufficient in many cases and could be misleading. The idea was that the user would be able to see, from the test plan itself, what node a certain step would be interacting with. However, the reality is that steps often need to run statements on multiple different nodes, or perhaps they need to pick one node specifically (e.g., the statement needs to run on a node in the old version). For that reason, the `DBNode()` function was dropped. Instead, steps have access to a random number generator that they can use to pick an arbitrary node themselves. The random number generators are unique to each user function, meaning each test run will see the same numbers being generated even if other steps are scheduled concurrently. The numbers observed by a user function will also be the same if the seed passed to `mixedversion.Test` is the same. * Definition of a "test context" that is available to mixed-version tests. For now, the test context includes things like which version we are upgrading (or downgrading) to and from and which nodes are running which version. This allows tests to take actions based on, for example, the number of nodes upgraded. It also allows them to run certain operations on nodes that are known to be in a specific version. * Introduction of a `helper` struct that is passed to user-functions. For now, the helper includes functions to connect to a specific node and get the current test context. The struct will help us provide common functionality to tests so that they don't have to duplicate code. * Log cached binary and cluster versions before executing a step. This makes it easier to understand the state of the cluster when looking at the logs of one specific step. * Internal improvement to the test runner: instead of assuming the first step of a mixed-version test plan will start the the cockroach nodes, we now check that that is the case, providing a clear error message if/when that assumption doesn't hold anymore (instead of a cryptic connection failure error). Epic: CRDB-19321 Release note: None --- .../mixedversion/mixedversion.go | 107 +++++++-------- .../roachtestutil/mixedversion/planner.go | 38 ++++-- .../mixedversion/planner_test.go | 21 ++- .../roachtestutil/mixedversion/runner.go | 125 +++++++++++++++--- pkg/cmd/roachtest/tests/versionupgrade.go | 16 +-- 5 files changed, 198 insertions(+), 109 deletions(-) diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go index f82c40430f69..9bb0700e7440 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/mixedversion.go @@ -65,7 +65,6 @@ package mixedversion import ( "context" - gosql "database/sql" "fmt" "math/rand" "strings" @@ -93,11 +92,6 @@ const ( // of migration steps before the new cluster version can be // finalized. runWhileMigratingProbability = 0.5 - - // noDBNeeded is an internal sentinel value expected to be returned - // by steps generated by the test plan; it indicates that the step - // requires no connection to a cockroach node. - noDBNeeded = -1 ) var ( @@ -131,7 +125,7 @@ type ( Finalizing bool } - userFunc func(*logger.Logger, *gosql.DB) error + userFunc func(*logger.Logger, *rand.Rand, *Helper) error predicateFunc func(Context) bool // versionUpgradeHook is a hook that can be called at any time @@ -159,17 +153,12 @@ type ( // ID returns a unique ID associated with the step, making it easy // to reference test output with the exact step it relates to ID() int - // DBNode returns the database node that that step connects to - // during its execution. If the step does not require a database - // connection, this function should return the `noDBNeeded` - // constant - DBNode() int // Description is a string representation of the step, intended // for human-consumption. Displayed when pretty-printing the test // plan. Description() string // Run implements the actual functionality of the step. - Run(context.Context, *logger.Logger, cluster.Cluster, func(int) *gosql.DB) error + Run(context.Context, *logger.Logger, cluster.Cluster, *Helper) error } hooks []versionUpgradeHook @@ -218,7 +207,7 @@ func NewTest( } prng, seed := randutil.NewPseudoRand() - testLogger.Printf("random seed: %d", seed) + testLogger.Printf("mixed-version random seed: %d", seed) return &Test{ ctx: ctx, @@ -339,8 +328,7 @@ type startFromCheckpointStep struct { crdbNodes option.NodeListOption } -func (s startFromCheckpointStep) ID() int { return s.id } -func (s startFromCheckpointStep) DBNode() int { return noDBNeeded } +func (s startFromCheckpointStep) ID() int { return s.id } func (s startFromCheckpointStep) Description() string { return fmt.Sprintf("starting cluster from fixtures for version %q", s.version) @@ -350,7 +338,7 @@ func (s startFromCheckpointStep) Description() string { // upload the binary associated with that given version, and finally // start the cockroach binary on these nodes. func (s startFromCheckpointStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { if err := clusterupgrade.InstallFixtures(ctx, l, c, s.crdbNodes, s.version); err != nil { return err @@ -376,8 +364,7 @@ type waitForStableClusterVersionStep struct { nodes option.NodeListOption } -func (s waitForStableClusterVersionStep) ID() int { return s.id } -func (s waitForStableClusterVersionStep) DBNode() int { return noDBNeeded } +func (s waitForStableClusterVersionStep) ID() int { return s.id } func (s waitForStableClusterVersionStep) Description() string { return fmt.Sprintf( @@ -387,38 +374,38 @@ func (s waitForStableClusterVersionStep) Description() string { } func (s waitForStableClusterVersionStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { - return clusterupgrade.WaitForClusterUpgrade(ctx, l, s.nodes, dbFunc) + return clusterupgrade.WaitForClusterUpgrade(ctx, l, s.nodes, helper.Connect) } // preserveDowngradeOptionStep sets the `preserve_downgrade_option` // cluster setting to the binary version running in `node`. type preserveDowngradeOptionStep struct { - id int - node int + id int + crdbNodes option.NodeListOption + prng *rand.Rand } -func (s preserveDowngradeOptionStep) ID() int { return s.id } -func (s preserveDowngradeOptionStep) DBNode() int { return s.node } +func (s preserveDowngradeOptionStep) ID() int { return s.id } func (s preserveDowngradeOptionStep) Description() string { return "preventing auto-upgrades by setting `preserve_downgrade_option`" } func (s preserveDowngradeOptionStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { - db := dbFunc(s.node) - - l.Printf("checking binary version on node %d", s.node) + node, db := helper.RandomDB(s.prng, s.crdbNodes) + l.Printf("checking binary version (via node %d)", node) bv, err := clusterupgrade.BinaryVersion(db) if err != nil { return err } + node, db = helper.RandomDB(s.prng, s.crdbNodes) downgradeOption := bv.String() - l.Printf("setting `preserve_downgrade_option` to %s", downgradeOption) + l.Printf("setting `preserve_downgrade_option` to %s (via node %d)", downgradeOption, node) _, err = db.ExecContext(ctx, "SET CLUSTER SETTING cluster.preserve_downgrade_option = $1", downgradeOption) return err } @@ -434,15 +421,14 @@ type restartWithNewBinaryStep struct { node int } -func (s restartWithNewBinaryStep) ID() int { return s.id } -func (s restartWithNewBinaryStep) DBNode() int { return noDBNeeded } +func (s restartWithNewBinaryStep) ID() int { return s.id } func (s restartWithNewBinaryStep) Description() string { return fmt.Sprintf("restart node %d with binary version %s", s.node, versionMsg(s.version)) } func (s restartWithNewBinaryStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { return clusterupgrade.RestartNodesWithNewBinary( ctx, @@ -459,22 +445,22 @@ func (s restartWithNewBinaryStep) Run( // setting, allowing the upgrade migrations to run and the cluster // version to eventually reach the binary version on the nodes. type finalizeUpgradeStep struct { - id int - node int + id int + crdbNodes option.NodeListOption + prng *rand.Rand } -func (s finalizeUpgradeStep) ID() int { return s.id } -func (s finalizeUpgradeStep) DBNode() int { return s.node } +func (s finalizeUpgradeStep) ID() int { return s.id } func (s finalizeUpgradeStep) Description() string { return "finalize upgrade by resetting `preserve_downgrade_option`" } func (s finalizeUpgradeStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { - db := dbFunc(s.node) - l.Printf("resetting preserve_downgrade_option") + node, db := helper.RandomDB(s.prng, s.crdbNodes) + l.Printf("resetting preserve_downgrade_option (via node %d)", node) _, err := db.ExecContext(ctx, "RESET CLUSTER SETTING cluster.preserve_downgrade_option") return err } @@ -482,22 +468,23 @@ func (s finalizeUpgradeStep) Run( // runHookStep is a step used to run a user-provided hook (i.e., // callbacks passed to `OnStartup`, `InMixedVersion`, or `AfterTest`). type runHookStep struct { - id int - node int - hook versionUpgradeHook + id int + testContext Context + prng *rand.Rand + hook versionUpgradeHook } -func (s runHookStep) ID() int { return s.id } -func (s runHookStep) DBNode() int { return s.node } +func (s runHookStep) ID() int { return s.id } func (s runHookStep) Description() string { return fmt.Sprintf("run %q", s.hook.name) } func (s runHookStep) Run( - ctx context.Context, l *logger.Logger, c cluster.Cluster, dbFunc func(int) *gosql.DB, + ctx context.Context, l *logger.Logger, c cluster.Cluster, helper *Helper, ) error { - return s.hook.fn(l, dbFunc(s.node)) + helper.SetContext(&s.testContext) + return s.hook.fn(l, s.prng, helper) } // sequentialRunStep is a "meta-step" that indicates that a sequence @@ -556,11 +543,6 @@ func prefixedLogger(l *logger.Logger, prefix string) (*logger.Logger, error) { return l.ChildLogger(fileName, logger.LogPrefix(formattedPrefix)) } -func randomNode(rng *rand.Rand, nodes option.NodeListOption) int { - idx := rng.Intn(len(nodes)) - return nodes[idx] -} - func (h hooks) Filter(testContext Context) hooks { var result hooks for _, hook := range h { @@ -577,11 +559,12 @@ func (h hooks) Filter(testContext Context) hooks { // returned. Otherwise, a `concurrentRunStep` is returned, where every // hook is run concurrently. func (h hooks) AsSteps( - label string, idGen func() int, rng *rand.Rand, nodes option.NodeListOption, + label string, idGen func() int, prng *rand.Rand, nodes option.NodeListOption, testContext Context, ) []testStep { steps := make([]testStep, 0, len(h)) for _, hook := range h { - rhs := runHookStep{id: idGen(), node: randomNode(rng, nodes), hook: hook} + hookPrng := rngFromRNG(prng) + rhs := runHookStep{id: idGen(), prng: hookPrng, hook: hook, testContext: testContext} steps = append(steps, rhs) } @@ -589,7 +572,7 @@ func (h hooks) AsSteps( return steps } - return []testStep{newConcurrentRunStep(label, steps, rng)} + return []testStep{newConcurrentRunStep(label, steps, prng)} } func (th *testHooks) AddStartup(hook versionUpgradeHook) { @@ -604,16 +587,16 @@ func (th *testHooks) AddAfterUpgradeFinalized(hook versionUpgradeHook) { th.afterUpgradeFinalized = append(th.afterUpgradeFinalized, hook) } -func (th *testHooks) StartupSteps(idGen func() int) []testStep { - return th.startup.AsSteps(startupLabel, idGen, th.prng, th.crdbNodes) +func (th *testHooks) StartupSteps(idGen func() int, testContext Context) []testStep { + return th.startup.AsSteps(startupLabel, idGen, th.prng, th.crdbNodes, testContext) } func (th *testHooks) MixedVersionSteps(testContext Context, idGen func() int) []testStep { - return th.mixedVersion.Filter(testContext).AsSteps(mixedVersionLabel, idGen, th.prng, th.crdbNodes) + return th.mixedVersion.Filter(testContext).AsSteps(mixedVersionLabel, idGen, th.prng, th.crdbNodes, testContext) } -func (th *testHooks) AfterUpgradeFinalizedSteps(idGen func() int) []testStep { - return th.afterUpgradeFinalized.AsSteps(afterTestLabel, idGen, th.prng, th.crdbNodes) +func (th *testHooks) AfterUpgradeFinalizedSteps(idGen func() int, testContext Context) []testStep { + return th.afterUpgradeFinalized.AsSteps(afterTestLabel, idGen, th.prng, th.crdbNodes, testContext) } func randomDelay(rng *rand.Rand) time.Duration { @@ -621,6 +604,10 @@ func randomDelay(rng *rand.Rand) time.Duration { return time.Duration(possibleDelaysMs[idx]) * time.Millisecond } +func rngFromRNG(rng *rand.Rand) *rand.Rand { + return rand.New(rand.NewSource(rng.Int63())) +} + func versionMsg(version string) string { return clusterupgrade.VersionMsg(version) } diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go index 7a3c58922d62..0fbabe2c39f3 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner.go @@ -95,17 +95,33 @@ func (p *testPlanner) Plan() *TestPlan { } } +func (p *testPlanner) initialContext() Context { + return Context{ + FromVersion: p.initialVersion, + ToVersion: clusterupgrade.MainVersion, + FromVersionNodes: p.crdbNodes, + } +} + +func (p *testPlanner) finalContext(finalizing bool) Context { + return Context{ + FromVersion: p.initialVersion, + ToVersion: clusterupgrade.MainVersion, + ToVersionNodes: p.crdbNodes, + Finalizing: finalizing, + } +} + // initSteps returns the sequence of steps that should be executed // before we start changing binaries on nodes in the process of // upgrading/downgrading. It will also run any startup hooks the user // may have provided. func (p *testPlanner) initSteps() []testStep { - preserveDowngradeNode := randomNode(p.prng, p.crdbNodes) return append([]testStep{ startFromCheckpointStep{id: p.nextID(), version: p.initialVersion, rt: p.rt, crdbNodes: p.crdbNodes}, waitForStableClusterVersionStep{id: p.nextID(), nodes: p.crdbNodes}, - preserveDowngradeOptionStep{id: p.nextID(), node: preserveDowngradeNode}, - }, p.hooks.StartupSteps(p.nextID)...) + preserveDowngradeOptionStep{id: p.nextID(), prng: p.newRNG(), crdbNodes: p.crdbNodes}, + }, p.hooks.StartupSteps(p.nextID, p.initialContext())...) } // finalSteps are the steps to be run once the nodes have been @@ -115,7 +131,7 @@ func (p *testPlanner) initSteps() []testStep { func (p *testPlanner) finalSteps() []testStep { return append([]testStep{ waitForStableClusterVersionStep{id: p.nextID(), nodes: p.crdbNodes}, - }, p.hooks.AfterUpgradeFinalizedSteps(p.nextID)...) + }, p.hooks.AfterUpgradeFinalizedSteps(p.nextID, p.finalContext(false /* finalizing */))...) } func (p *testPlanner) upgradeSteps(from, to string) []testStep { @@ -165,11 +181,9 @@ func (p *testPlanner) changeVersionSteps(from, to, label string) []testStep { // `preserve_downgrade_option` and potentially running mixed-version // hooks while the cluster version is changing. func (p *testPlanner) finalizeUpgradeSteps() []testStep { - testContext := Context{Finalizing: true} - finalizeNode := randomNode(p.prng, p.crdbNodes) return append([]testStep{ - finalizeUpgradeStep{id: p.nextID(), node: finalizeNode}, - }, p.hooks.MixedVersionSteps(testContext, p.nextID)...) + finalizeUpgradeStep{id: p.nextID(), prng: p.newRNG(), crdbNodes: p.crdbNodes}, + }, p.hooks.MixedVersionSteps(p.finalContext(true /* finalizing */), p.nextID)...) } func (p *testPlanner) nextID() int { @@ -177,6 +191,10 @@ func (p *testPlanner) nextID() int { return p.stepCount } +func (p *testPlanner) newRNG() *rand.Rand { + return rngFromRNG(p.prng) +} + // PrettyPrint displays a tree-like view of the mixed-version test // plan, useful when debugging mixed-version test failures. Each step // is assigned an ID, making it easy to correlate the step that @@ -211,10 +229,6 @@ func (plan *TestPlan) prettyPrintStep(out *strings.Builder, step testStep, prefi // concurrent execution), and what database node the step is // connecting to. writeSingle := func(rs singleStep, extraContext ...string) { - if node := rs.DBNode(); node != noDBNeeded { - dbinfo := fmt.Sprintf("with connection to node %d", node) - extraContext = append([]string{dbinfo}, extraContext...) - } var extras string if contextStr := strings.Join(extraContext, ", "); contextStr != "" { extras = ", " + contextStr diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go index b7d560b8c2ba..c078aaf62f74 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/planner_test.go @@ -12,7 +12,6 @@ package mixedversion import ( "context" - gosql "database/sql" "fmt" "io" "math/rand" @@ -81,31 +80,31 @@ func TestTestPlanner(t *testing.T) { mixed-version test plan for upgrading from %[1]s to : ├── starting cluster from fixtures for version "%[1]s" (1) ├── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (2) -├── preventing auto-upgrades by setting `+"`preserve_downgrade_option`"+`, with connection to node 4 (3) +├── preventing auto-upgrades by setting `+"`preserve_downgrade_option`"+` (3) ├── upgrade nodes :1-4 from "%[1]s" to "" │ ├── restart node 2 with binary version (4) │ ├── restart node 1 with binary version (5) -│ ├── run "mixed-version 1", with connection to node 3 (6) +│ ├── run "mixed-version 1" (6) │ ├── restart node 4 with binary version (7) │ ├── restart node 3 with binary version (8) -│ └── run "mixed-version 2", with connection to node 3 (9) +│ └── run "mixed-version 2" (9) ├── downgrade nodes :1-4 from "" to "%[1]s" │ ├── restart node 3 with binary version %[1]s (10) │ ├── restart node 4 with binary version %[1]s (11) │ ├── run mixed-version hooks concurrently -│ │ ├── run "mixed-version 1", with connection to node 1, after 200ms delay (12) -│ │ └── run "mixed-version 2", with connection to node 1, after 200ms delay (13) +│ │ ├── run "mixed-version 1", after 200ms delay (12) +│ │ └── run "mixed-version 2", after 200ms delay (13) │ ├── restart node 2 with binary version %[1]s (14) │ └── restart node 1 with binary version %[1]s (15) ├── upgrade nodes :1-4 from "%[1]s" to "" │ ├── restart node 3 with binary version (16) -│ ├── run "mixed-version 1", with connection to node 1 (17) +│ ├── run "mixed-version 1" (17) │ ├── restart node 4 with binary version (18) │ ├── restart node 1 with binary version (19) │ ├── restart node 2 with binary version (20) -│ └── run "mixed-version 2", with connection to node 2 (21) -├── finalize upgrade by resetting `+"`preserve_downgrade_option`"+`, with connection to node 3 (22) -├── run "mixed-version 2", with connection to node 1 (23) +│ └── run "mixed-version 2" (21) +├── finalize upgrade by resetting `+"`preserve_downgrade_option`"+` (22) +├── run "mixed-version 2" (23) └── wait for nodes :1-4 to all have the same cluster version (same as binary version of node 1) (24) `, previousVersion, ) @@ -184,6 +183,6 @@ func requireConcurrentHooks(t *testing.T, step testStep, names ...string) { } } -func dummyHook(*logger.Logger, *gosql.DB) error { +func dummyHook(*logger.Logger, *rand.Rand, *Helper) error { return nil } diff --git a/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go b/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go index 07422d12195c..7f9826f6fe09 100644 --- a/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go +++ b/pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go @@ -14,6 +14,7 @@ import ( "context" gosql "database/sql" "fmt" + "math/rand" "os" "path/filepath" "regexp" @@ -31,6 +32,17 @@ import ( ) type ( + // Helper is the struct passed to user-functions providing helper + // functions that mixed-version tests can use. + Helper struct { + ctx context.Context + testContext *Context + conns []*gosql.DB + crdbNodes option.NodeListOption + + stepLogger *logger.Logger + } + testRunner struct { ctx context.Context plan *TestPlan @@ -86,22 +98,28 @@ func (tr *testRunner) run() error { // runStep contains the logic of running a single test step, called // recursively in the case of sequentialRunStep and concurrentRunStep. func (tr *testRunner) runStep(step testStep) error { - if ss, ok := step.(singleStep); ok && ss.ID() > 1 { - // if we are running a singleStep that is *not* the first step, - // we can initialize the database connections. This represents the - // assumption that the first step in the test plan is the one that - // sets up binaries and make the `cockroach` process available on - // the nodes. - // TODO(renato): consider a way to make the test runner crash if - // the assumption does not hold - if err := tr.maybeInitConnections(); err != nil { - return err - } - if err := tr.refreshBinaryVersions(); err != nil { - return err - } - if err := tr.refreshClusterVersions(); err != nil { - return err + if ss, ok := step.(singleStep); ok { + if ss.ID() == 1 { + // if this is the first singleStep of the plan, ensure it is an + // "initialization step" (i.e., cockroach nodes are ready after + // it executes). This is an assumption of the test runner and + // makes for clear error messages if that assumption is broken. + if err := tr.ensureInitializationStep(ss); err != nil { + return err + } + } else { + // update the runner's view of the cluster's binary and cluster + // versions before every non-initialization `singleStep` is + // executed + if err := tr.maybeInitConnections(); err != nil { + return err + } + if err := tr.refreshBinaryVersions(); err != nil { + return err + } + if err := tr.refreshClusterVersions(); err != nil { + return err + } } } @@ -136,12 +154,13 @@ func (tr *testRunner) runStep(step testStep) error { } tr.logStep("STARTING", ss, stepLogger) + tr.logVersions(stepLogger) start := timeutil.Now() defer func() { prefix := fmt.Sprintf("FINISHED [%s]", timeutil.Since(start)) tr.logStep(prefix, ss, stepLogger) }() - if err := ss.Run(tr.ctx, stepLogger, tr.cluster, tr.conn); err != nil { + if err := ss.Run(tr.ctx, stepLogger, tr.cluster, tr.newHelper(stepLogger)); err != nil { return tr.reportError(err, ss, stepLogger) } @@ -188,6 +207,18 @@ func (tr *testRunner) logStep(prefix string, step singleStep, l *logger.Logger) l.Printf("%[1]s %s (%d): %s %[1]s", dashes, prefix, step.ID(), step.Description()) } +// logVersions writes the current cached versions of the binary and +// cluster versions on each node. The cached versions should exist for +// all steps but the first one (when we start the cluster itself). +func (tr *testRunner) logVersions(l *logger.Logger) { + if tr.binaryVersions == nil || tr.clusterVersions == nil { + return + } + + l.Printf("binary versions: %s", formatVersions(tr.binaryVersions)) + l.Printf("cluster versions: %s", formatVersions(tr.clusterVersions)) +} + // loggerFor creates a logger instance to be used by a test step. Logs // will be available under `mixed-version-test/{ID}.log`, making it // easy to go from the IDs displayed in the test plan to the @@ -231,6 +262,15 @@ func (tr *testRunner) refreshClusterVersions() error { return nil } +func (tr *testRunner) ensureInitializationStep(ss singleStep) error { + _, isInit := ss.(startFromCheckpointStep) + if !isInit { + return fmt.Errorf("unexpected initialization type in mixed-version test: %T", ss) + } + + return nil +} + // maybeInitConnections initialize connections if the connection cache // is empty. func (tr *testRunner) maybeInitConnections() error { @@ -251,6 +291,15 @@ func (tr *testRunner) maybeInitConnections() error { return nil } +func (tr *testRunner) newHelper(l *logger.Logger) *Helper { + return &Helper{ + ctx: tr.ctx, + conns: tr.connCache, + crdbNodes: tr.crdbNodes, + stepLogger: l, + } +} + // conn returns a database connection to the given node. Assumes the // connection cache has been previously initialized. func (tr *testRunner) conn(node int) *gosql.DB { @@ -265,6 +314,48 @@ func (tr *testRunner) closeConnections() { } } +// RandomDB returns a (nodeID, connection) tuple for a randomly picked +// cockroach node according to the parameters passed. +func (h *Helper) RandomDB(prng *rand.Rand, nodes option.NodeListOption) (int, *gosql.DB) { + node := nodes[prng.Intn(len(nodes))] + return node, h.Connect(node) +} + +// QueryRow performs `db.QueryRowContext` on a randomly picked +// database node. The query and the node picked are logged in the logs +// of the step that calls this function. +func (h *Helper) QueryRow(rng *rand.Rand, query string, args ...interface{}) *gosql.Row { + node, db := h.RandomDB(rng, h.crdbNodes) + h.stepLogger.Printf("running SQL statement:\n%s\nArgs: %v\nNode: %d", query, args, node) + return db.QueryRowContext(h.ctx, query, args...) +} + +// Exec performs `db.ExecContext` on a randomly picked database node. +// The query and the node picked are logged in the logs of the step +// that calls this function. +func (h *Helper) Exec(rng *rand.Rand, query string, args ...interface{}) error { + node, db := h.RandomDB(rng, h.crdbNodes) + h.stepLogger.Printf("running SQL statement:\n%s\nArgs: %v\nNode: %d", query, args, node) + _, err := db.ExecContext(h.ctx, query, args...) + return err +} + +func (h *Helper) Connect(node int) *gosql.DB { + return h.conns[node-1] +} + +// SetContext should be called by steps that need access to the test +// context, as that is only visible to them. +func (h *Helper) SetContext(c *Context) { + h.testContext = c +} + +// Context returns the test context associated with a certain step. It +// is made available for user-functions (see runHookStep). +func (h *Helper) Context() *Context { + return h.testContext +} + func renameFailedLogger(l *logger.Logger) error { currentFileName := l.File.Name() newLogName := strings.TrimSuffix(currentFileName, filepath.Ext(currentFileName)) diff --git a/pkg/cmd/roachtest/tests/versionupgrade.go b/pkg/cmd/roachtest/tests/versionupgrade.go index baf111b90966..cfc4faf8265f 100644 --- a/pkg/cmd/roachtest/tests/versionupgrade.go +++ b/pkg/cmd/roachtest/tests/versionupgrade.go @@ -14,6 +14,7 @@ import ( "context" gosql "database/sql" "fmt" + "math/rand" "path/filepath" "runtime" "time" @@ -103,20 +104,17 @@ func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) { } mvt := mixedversion.NewTest(ctx, t, t.L(), c, c.All()) - mvt.InMixedVersion("run backup", func(l *logger.Logger, db *gosql.DB) error { + mvt.InMixedVersion("run backup", func(l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { // Verify that backups can be created in various configurations. This is // important to test because changes in system tables might cause backups to // fail in mixed-version clusters. dest := fmt.Sprintf("nodelocal://0/%d", timeutil.Now().UnixNano()) - l.Printf("writing backup to %s", dest) - _, err := db.ExecContext(ctx, `BACKUP TO $1`, dest) - return err + return h.Exec(rng, `BACKUP TO $1`, dest) }) - mvt.InMixedVersion("test features", func(l *logger.Logger, db *gosql.DB) error { + mvt.InMixedVersion("test features", func(l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { for _, featureTest := range versionUpgradeTestFeatures { l.Printf("running feature test %q", featureTest.name) - _, err := db.ExecContext(ctx, featureTest.statement) - if err != nil { + if err := h.Exec(rng, featureTest.statement); err != nil { l.Printf("%q: ERROR (%s)", featureTest.name, err) return err } @@ -125,7 +123,7 @@ func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) { return nil }) - mvt.AfterUpgradeFinalized("check if GC TTL is pinned", func(l *logger.Logger, db *gosql.DB) error { + mvt.AfterUpgradeFinalized("check if GC TTL is pinned", func(l *logger.Logger, rng *rand.Rand, h *mixedversion.Helper) error { // TODO(irfansharif): This can be removed when the predecessor version // in this test is v23.1, where the default is 4h. This test was only to // make sure that existing clusters that upgrade to 23.1 retained their @@ -139,7 +137,7 @@ func runVersionUpgrade(ctx context.Context, t test.Test, c cluster.Cluster) { WHERE target = 'RANGE default' LIMIT 1 ` - if err := db.QueryRowContext(ctx, query).Scan(&ttlSeconds); err != nil { + if err := h.QueryRow(rng, query).Scan(&ttlSeconds); err != nil { return fmt.Errorf("error querying GC TTL: %w", err) } expectedTTL := 24 * 60 * 60 // NB: 24h is what's used in the fixture From 5225a325296d4c18924daeb34e5daca2a6daaa31 Mon Sep 17 00:00:00 2001 From: Andy Yang Date: Wed, 15 Feb 2023 16:14:41 -0500 Subject: [PATCH 4/4] sql: add user_id column to system.database_role_settings table This patch adds a new `role_id` column to the `system.database_role_settings` table, which corresponds to the existing `role_name` column. Migrations are also added to alter and backfill the table in older clusters. Release note: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 17 ++ pkg/sql/alter_role.go | 12 ++ pkg/sql/catalog/bootstrap/testdata/testdata | 8 +- pkg/sql/catalog/systemschema/system.go | 29 ++- .../systemschema_test/testdata/bootstrap | 6 +- .../testdata/logic_test/crdb_internal_catalog | 2 +- .../testdata/logic_test/gen_test_objects | 8 +- .../testdata/logic_test/information_schema | 4 + .../logictest/testdata/logic_test/pg_catalog | 3 + pkg/upgrade/upgrades/BUILD.bazel | 2 + ...e_role_settings_table_user_id_migration.go | 86 +++++++++ ...e_settings_table_user_id_migration_test.go | 168 ++++++++++++++++++ pkg/upgrade/upgrades/upgrades.go | 12 ++ 15 files changed, 341 insertions(+), 20 deletions(-) create mode 100644 pkg/upgrade/upgrades/database_role_settings_table_user_id_migration.go create mode 100644 pkg/upgrade/upgrades/database_role_settings_table_user_id_migration_test.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 5b1d40014b45..c93dabfabf22 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -296,4 +296,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 1000022.2-52 set the active cluster version in the format '.' +version version 1000022.2-56 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 80b57643859d..d395b3d7a618 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -237,6 +237,6 @@
trace.opentelemetry.collector
stringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.span_registry.enabled
booleantrueif set, ongoing traces can be seen at https://<ui>/#/debug/tracez
trace.zipkin.collector
stringthe address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used. -
version
version1000022.2-52set the active cluster version in the format '<major>.<minor>' +
version
version1000022.2-56set the active cluster version in the format '<major>.<minor>' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 85d6934c47fe..985502b8bd35 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -435,6 +435,15 @@ const ( // system.privileges, on the path and username columns. V23_1AlterSystemPrivilegesAddIndexOnPathAndUsername + // V23_1DatabaseRoleSettingsHasRoleIDColumn is the version where the role_id + // column has been added to the system.database_role_settings table. + V23_1DatabaseRoleSettingsHasRoleIDColumn + + // V23_1DatabaseRoleSettingsRoleIDColumnBackfilled is the version where + // the role_id column in the system.database_role_settings table has been + // backfilled. + V23_1DatabaseRoleSettingsRoleIDColumnBackfilled + // ************************************************* // Step (1): Add new versions here. // Do not add new versions to a patch release. @@ -752,6 +761,14 @@ var rawVersionsSingleton = keyedVersions{ Key: V23_1AlterSystemPrivilegesAddIndexOnPathAndUsername, Version: roachpb.Version{Major: 22, Minor: 2, Internal: 52}, }, + { + Key: V23_1DatabaseRoleSettingsHasRoleIDColumn, + Version: roachpb.Version{Major: 22, Minor: 2, Internal: 54}, + }, + { + Key: V23_1DatabaseRoleSettingsRoleIDColumnBackfilled, + Version: roachpb.Version{Major: 22, Minor: 2, Internal: 56}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/sql/alter_role.go b/pkg/sql/alter_role.go index 55a76daeff31..475a8ae224dc 100644 --- a/pkg/sql/alter_role.go +++ b/pkg/sql/alter_role.go @@ -15,6 +15,7 @@ import ( "fmt" "strings" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -403,6 +404,9 @@ func (p *planner) processSetOrResetClause( } func (n *alterRoleSetNode) startExec(params runParams) error { + databaseRoleSettingsHasRoleIDCol := params.p.ExecCfg().Settings.Version.IsActive(params.ctx, + clusterversion.V23_1DatabaseRoleSettingsHasRoleIDColumn) + var opName string if n.isRole { sqltelemetry.IncIAMAlterCounter(sqltelemetry.Role) @@ -425,10 +429,18 @@ func (n *alterRoleSetNode) startExec(params runParams) error { `DELETE FROM %s WHERE database_id = $1 AND role_name = $2`, sessioninit.DatabaseRoleSettingsTableName, ) + var upsertQuery = fmt.Sprintf( `UPSERT INTO %s (database_id, role_name, settings) VALUES ($1, $2, $3)`, sessioninit.DatabaseRoleSettingsTableName, ) + if databaseRoleSettingsHasRoleIDCol { + upsertQuery = fmt.Sprintf(` +UPSERT INTO %s (database_id, role_name, settings, role_id) +VALUES ($1, $2, $3, (SELECT user_id FROM system.users WHERE username = $2))`, + sessioninit.DatabaseRoleSettingsTableName, + ) + } // Instead of inserting an empty settings array, this function will make // sure the row is deleted instead. diff --git a/pkg/sql/catalog/bootstrap/testdata/testdata b/pkg/sql/catalog/bootstrap/testdata/testdata index 370610da23ec..ffd20a00c7e5 100644 --- a/pkg/sql/catalog/bootstrap/testdata/testdata +++ b/pkg/sql/catalog/bootstrap/testdata/testdata @@ -1,4 +1,4 @@ -system hash=cb832e0b0f81671eb64b3f21c16763300e5a728c9755df61d23cf5c255a4315c +system hash=0545f34c676ce828c5d25a1ffd71f4a41d63816db3c25cc06c57495f6a658f29 ---- [{"key":"04646573632d696467656e","value":"01c801"} ,{"key":"8b"} @@ -36,7 +36,7 @@ system hash=cb832e0b0f81671eb64b3f21c16763300e5a728c9755df61d23cf5c255a4315c ,{"key":"8b89b18a89","value":"030aaa030a0b6a6f696e5f746f6b656e731829200128013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422b0a0673656372657410021a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10031a0d080910001800300050a009600020003000680070007800800100880100980100480452710a077072696d61727910011801220269642a067365637265742a0a65787069726174696f6e300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2012b0a077072696d61727910001a0269641a067365637265741a0a65787069726174696f6e2001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89b28a89","value":"030abe140a1473746174656d656e745f73746174697374696373182a200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c08081000180030005011600020003000680070007800800100880100980100422e0a09706c616e5f6861736810041a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510051a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410061a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10071a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110081a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310091a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a04706c616e100a1a0d081210001800300050da1d600020003000680070007800800100880100980100429c020a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38100b1a0c080110201800300050176000200030015a8c016d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f69642c20706c616e5f686173682c207472616e73616374696f6e5f66696e6765727072696e745f696429292c20383a3a3a494e543829680070007800800100880100980100425f0a15696e6465785f7265636f6d6d656e646174696f6e73100c1a1d080f100018003000380750f1075a0c080710001800300050196000600020002a1241525241595b5d3a3a3a535452494e475b5d3000680070007800800100880100980100426c0a0d696e64657865735f7573616765100d1a0d081210001800300050da1d6000200130005a3728737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27696e6465786573273a3a3a535452494e47680070007800800101880100980100480e52bd040a077072696d617279100118012268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f69642209706c616e5f6861736822086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a04706c616e2a15696e6465785f7265636f6d6d656e646174696f6e73300b30013002300330043005300640004000400040004000400040004a10080010001a00200028003000380040005a00700770087009700a700c7a0408002000800100880100900104980101a201c70108011268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f69642209706c616e5f68617368221a7472616e73616374696f6e5f66696e6765727072696e745f6964a80100b20100ba0100c00100c80100d00101e001005a9d010a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f696430023003380b3801380438053806400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e001005a7f0a11696e64657865735f75736167655f69647810031800220d696e64657865735f7573616765300d380b38013802380338043805380640004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80100e0010060046a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651802800101880103980100a201b9020abc01637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e543829126e636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381800280b300038014002b201a9020a077072696d61727910001a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a09706c616e5f686173681a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a04706c616e1a15696e6465785f7265636f6d6d656e646174696f6e73200b200120022003200420052006200720082009200a200c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} ,{"key":"8b89b38a89","value":"030ae60c0a167472616e73616374696f6e5f73746174697374696373182b200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510031a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410041a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10051a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110061a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310071a0d081210001800300050da1d60002000300068007000780080010088010098010042cf010a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3810081a0c080110201800300050176000200030015a656d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f696429292c20383a3a3a494e543829680070007800800100880100980100480952fb020a077072696d617279100118012243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f696422086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a7374617469737469637330083001300230033004400040004000400040004a10080010001a00200028003000380040005a007005700670077a0408002000800100880100900104980101a2017b08011243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f6964a80100b20100ba0100c00100c80100d00101e001005a7b0a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f69643002380838013803380440004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e0010060036a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651802800101880103980100a201ef010a9701637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e5438291249636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3818002808300038014002b201b8010a077072696d61727910001a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a73746174697374696373200820012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} -,{"key":"8b89b48a89","value":"030ae4030a1664617461626173655f726f6c655f73657474696e6773182c200128013a0042300a0b64617461626173655f696410011a0c080c100018003000501a600020003000680070007800800100880100980100422e0a09726f6c655f6e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100423e0a0873657474696e677310031a1d080f100018003000380750f1075a0c0807100018003000501960006000200030006800700078008001008801009801004804527d0a077072696d61727910011801220b64617461626173655f69642209726f6c655f6e616d652a0873657474696e677330013002400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201350a077072696d61727910001a0b64617461626173655f69641a09726f6c655f6e616d651a0873657474696e67732001200220032803b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} +,{"key":"8b89b48a89","value":"030ad0050a1664617461626173655f726f6c655f73657474696e6773182c200128013a0042300a0b64617461626173655f696410011a0c080c100018003000501a600020003000680070007800800100880100980100422e0a09726f6c655f6e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100423e0a0873657474696e677310031a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07726f6c655f696410041a0c080c100018003000501a60002001300068007000780080010088010098010048055288010a077072696d61727910011801220b64617461626173655f69642209726f6c655f6e616d652a0873657474696e67732a07726f6c655f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e001005aa4010a2e64617461626173655f726f6c655f73657474696e67735f64617461626173655f69645f726f6c655f69645f6b657910021801220b64617461626173655f69642207726f6c655f69642a0873657474696e6773300130043802400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201400a077072696d61727910001a0b64617461626173655f69641a09726f6c655f6e616d651a0873657474696e67731a07726f6c655f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} ,{"key":"8b89b58a89","value":"030a840a0a0c74656e616e745f7573616765182d200128013a00422e0a0974656e616e745f696410011a0c0801104018003000501460002000300068007000780080010088010098010042300a0b696e7374616e63655f696410021a0c0801104018003000501460002000300068007000780080010088010098010042350a106e6578745f696e7374616e63655f696410031a0c0801104018003000501460002000300068007000780080010088010098010042310a0b6c6173745f75706461746510041a0d080510001800300050da0860002000300068007000780080010088010098010042340a0e72755f62757273745f6c696d697410051a0d080210401800300050bd0560002001300068007000780080010088010098010042340a0e72755f726566696c6c5f7261746510061a0d080210401800300050bd0560002001300068007000780080010088010098010042300a0a72755f63757272656e7410071a0d080210401800300050bd0560002001300068007000780080010088010098010042370a1163757272656e745f73686172655f73756d10081a0d080210401800300050bd0560002001300068007000780080010088010098010042360a11746f74616c5f636f6e73756d7074696f6e10091a0c0808100018003000501160002001300068007000780080010088010098010042330a0e696e7374616e63655f6c65617365100a1a0c0808100018003000501160002001300068007000780080010088010098010042310a0c696e7374616e63655f736571100b1a0c0801104018003000501460002001300068007000780080010088010098010042350a0f696e7374616e63655f736861726573100c1a0d080210401800300050bd05600020013000680070007800800100880100980100480d52a5020a077072696d61727910011801220974656e616e745f6964220b696e7374616e63655f69642a106e6578745f696e7374616e63655f69642a0b6c6173745f7570646174652a0e72755f62757273745f6c696d69742a0e72755f726566696c6c5f726174652a0a72755f63757272656e742a1163757272656e745f73686172655f73756d2a11746f74616c5f636f6e73756d7074696f6e2a0e696e7374616e63655f6c656173652a0c696e7374616e63655f7365712a0f696e7374616e63655f73686172657330013002400040004a10080010001a00200028003000380040005a007003700470057006700770087009700a700b700c7a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201dd010a077072696d61727910001a0974656e616e745f69641a0b696e7374616e63655f69641a106e6578745f696e7374616e63655f69641a0b6c6173745f7570646174651a0e72755f62757273745f6c696d69741a0e72755f726566696c6c5f726174651a0a72755f63757272656e741a1163757272656e745f73686172655f73756d1a11746f74616c5f636f6e73756d7074696f6e1a0e696e7374616e63655f6c656173651a0c696e7374616e63655f7365711a0f696e7374616e63655f736861726573200120022003200420052006200720082009200a200b200c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89b68a89","value":"030ab4040a0d73716c5f696e7374616e636573182e200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046164647210021a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a73657373696f6e5f696410031a0c08081000180030005011600020013000680070007800800100880100980100422e0a086c6f63616c69747910041a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0873716c5f6164647210051a0c0807100018003000501960002001300068007000780080010088010098010048065287010a077072696d61727910011801220269642a04616464722a0a73657373696f6e5f69642a086c6f63616c6974792a0873716c5f61646472300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201410a077072696d61727910001a0269641a04616464721a0a73657373696f6e5f69641a086c6f63616c6974791a0873716c5f61646472200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89b78a89","value":"030aee030a137370616e5f636f6e66696775726174696f6e73182f200128013a00422e0a0973746172745f6b657910011a0c08081000180030005011600020003000680070007800800100880100980100422c0a07656e645f6b657910021a0c08081000180030005011600020003000680070007800800100880100980100422b0a06636f6e66696710031a0c08081000180030005011600020003000680070007800800100880100980100480452750a077072696d61727910011801220973746172745f6b65792a07656e645f6b65792a06636f6e666967300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100a2012f0a1373746172745f6b6579203c20656e645f6b6579120c636865636b5f626f756e6473180028012802300038004002b2012f0a077072696d61727910001a0973746172745f6b65791a07656e645f6b65791a06636f6e6669672001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} @@ -163,7 +163,7 @@ system hash=cb832e0b0f81671eb64b3f21c16763300e5a728c9755df61d23cf5c255a4315c ,{"key":"c1"} ] -tenant hash=3ee56ab06edbdfddb10deacc66f83b5790a9520479690063163f4b9a8a97e588 +tenant hash=a168ce2df0b23f4592666573d5f96568e58ff51f25d4abdca7e68865d54617a4 ---- [{"key":""} ,{"key":"8b89898a89","value":"0312390a0673797374656d10011a250a0d0a0561646d696e1080101880100a0c0a04726f6f7410801018801012046e6f646518022200280140004a00"} @@ -199,7 +199,7 @@ tenant hash=3ee56ab06edbdfddb10deacc66f83b5790a9520479690063163f4b9a8a97e588 ,{"key":"8b89b18a89","value":"030aaa030a0b6a6f696e5f746f6b656e731829200128013a0042280a02696410011a0d080e100018003000508617600020003000680070007800800100880100980100422b0a0673656372657410021a0c0808100018003000501160002000300068007000780080010088010098010042300a0a65787069726174696f6e10031a0d080910001800300050a009600020003000680070007800800100880100980100480452710a077072696d61727910011801220269642a067365637265742a0a65787069726174696f6e300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2012b0a077072696d61727910001a0269641a067365637265741a0a65787069726174696f6e2001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89b28a89","value":"030abe140a1473746174656d656e745f73746174697374696373182a200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100423f0a1a7472616e73616374696f6e5f66696e6765727072696e745f696410031a0c08081000180030005011600020003000680070007800800100880100980100422e0a09706c616e5f6861736810041a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510051a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410061a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10071a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110081a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310091a0d081210001800300050da1d600020003000680070007800800100880100980100422a0a04706c616e100a1a0d081210001800300050da1d600020003000680070007800800100880100980100429c020a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38100b1a0c080110201800300050176000200030015a8c016d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f69642c20706c616e5f686173682c207472616e73616374696f6e5f66696e6765727072696e745f696429292c20383a3a3a494e543829680070007800800100880100980100425f0a15696e6465785f7265636f6d6d656e646174696f6e73100c1a1d080f100018003000380750f1075a0c080710001800300050196000600020002a1241525241595b5d3a3a3a535452494e475b5d3000680070007800800100880100980100426c0a0d696e64657865735f7573616765100d1a0d081210001800300050da1d6000200130005a3728737461746973746963732d3e2773746174697374696373273a3a3a535452494e47292d3e27696e6465786573273a3a3a535452494e47680070007800800101880100980100480e52bd040a077072696d617279100118012268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f69642209706c616e5f6861736822086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a737461746973746963732a04706c616e2a15696e6465785f7265636f6d6d656e646174696f6e73300b30013002300330043005300640004000400040004000400040004a10080010001a00200028003000380040005a00700770087009700a700c7a0408002000800100880100900104980101a201c70108011268637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f69642209706c616e5f68617368221a7472616e73616374696f6e5f66696e6765727072696e745f6964a80100b20100ba0100c00100c80100d00101e001005a9d010a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f6964221a7472616e73616374696f6e5f66696e6765727072696e745f696430023003380b3801380438053806400040004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e001005a7f0a11696e64657865735f75736167655f69647810031800220d696e64657865735f7573616765300d380b38013802380338043805380640004a10080010001a00200028003000380040005a007a0408002000800101880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100d80100e0010060046a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651802800101880103980100a201b9020abc01637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e543829126e636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381800280b300038014002b201a9020a077072696d61727910001a68637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f706c616e5f686173685f7472616e73616374696f6e5f66696e6765727072696e745f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a1a7472616e73616374696f6e5f66696e6765727072696e745f69641a09706c616e5f686173681a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a737461746973746963731a04706c616e1a15696e6465785f7265636f6d6d656e646174696f6e73200b200120022003200420052006200720082009200a200c2800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} ,{"key":"8b89b38a89","value":"030ae60c0a167472616e73616374696f6e5f73746174697374696373182b200128013a0042330a0d616767726567617465645f747310011a0d080910001800300050a00960002000300068007000780080010088010098010042330a0e66696e6765727072696e745f696410021a0c08081000180030005011600020003000680070007800800100880100980100422d0a086170705f6e616d6510031a0c08071000180030005019600020003000680070007800800100880100980100422c0a076e6f64655f696410041a0c0801104018003000501460002000300068007000780080010088010098010042380a0c6167675f696e74657276616c10051a13080610001800300050a20960006a040800100020003000680070007800800100880100980100422e0a086d6574616461746110061a0d081210001800300050da1d60002000300068007000780080010088010098010042300a0a7374617469737469637310071a0d081210001800300050da1d60002000300068007000780080010088010098010042cf010a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3810081a0c080110201800300050176000200030015a656d6f6428666e76333228637264625f696e7465726e616c2e646174756d735f746f5f627974657328616767726567617465645f74732c206170705f6e616d652c2066696e6765727072696e745f69642c206e6f64655f696429292c20383a3a3a494e543829680070007800800100880100980100480952fb020a077072696d617279100118012243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f38220d616767726567617465645f7473220e66696e6765727072696e745f696422086170705f6e616d6522076e6f64655f69642a0c6167675f696e74657276616c2a086d657461646174612a0a7374617469737469637330083001300230033004400040004000400040004a10080010001a00200028003000380040005a007005700670077a0408002000800100880100900104980101a2017b08011243637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381808220d616767726567617465645f747322086170705f6e616d65220e66696e6765727072696e745f696422076e6f64655f6964a80100b20100ba0100c00100c80100d00101e001005a7b0a1566696e6765727072696e745f73746174735f69647810021800220e66696e6765727072696e745f69643002380838013803380440004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e0010060036a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651802800101880103980100a201ef010a9701637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3820494e2028303a3a3a494e54382c20313a3a3a494e54382c20323a3a3a494e54382c20333a3a3a494e54382c20343a3a3a494e54382c20353a3a3a494e54382c20363a3a3a494e54382c20373a3a3a494e5438291249636865636b5f637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f3818002808300038014002b201b8010a077072696d61727910001a43637264625f696e7465726e616c5f616767726567617465645f74735f6170705f6e616d655f66696e6765727072696e745f69645f6e6f64655f69645f73686172645f381a0d616767726567617465645f74731a0e66696e6765727072696e745f69641a086170705f6e616d651a076e6f64655f69641a0c6167675f696e74657276616c1a086d657461646174611a0a73746174697374696373200820012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} -,{"key":"8b89b48a89","value":"030ae4030a1664617461626173655f726f6c655f73657474696e6773182c200128013a0042300a0b64617461626173655f696410011a0c080c100018003000501a600020003000680070007800800100880100980100422e0a09726f6c655f6e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100423e0a0873657474696e677310031a1d080f100018003000380750f1075a0c0807100018003000501960006000200030006800700078008001008801009801004804527d0a077072696d61727910011801220b64617461626173655f69642209726f6c655f6e616d652a0873657474696e677330013002400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201350a077072696d61727910001a0b64617461626173655f69641a09726f6c655f6e616d651a0873657474696e67732001200220032803b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} +,{"key":"8b89b48a89","value":"030ad0050a1664617461626173655f726f6c655f73657474696e6773182c200128013a0042300a0b64617461626173655f696410011a0c080c100018003000501a600020003000680070007800800100880100980100422e0a09726f6c655f6e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100423e0a0873657474696e677310031a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07726f6c655f696410041a0c080c100018003000501a60002001300068007000780080010088010098010048055288010a077072696d61727910011801220b64617461626173655f69642209726f6c655f6e616d652a0873657474696e67732a07726f6c655f696430013002400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e001005aa4010a2e64617461626173655f726f6c655f73657474696e67735f64617461626173655f69645f726f6c655f69645f6b657910021801220b64617461626173655f69642207726f6c655f69642a0873657474696e6773300130043802400040004a10080010001a00200028003000380040005a0070037a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201400a077072696d61727910001a0b64617461626173655f69641a09726f6c655f6e616d651a0873657474696e67731a07726f6c655f696420012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} ,{"key":"8b89b68a89","value":"030ab4040a0d73716c5f696e7374616e636573182e200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046164647210021a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a73657373696f6e5f696410031a0c08081000180030005011600020013000680070007800800100880100980100422e0a086c6f63616c69747910041a0d081210001800300050da1d600020013000680070007800800100880100980100422d0a0873716c5f6164647210051a0c0807100018003000501960002001300068007000780080010088010098010048065287010a077072696d61727910011801220269642a04616464722a0a73657373696f6e5f69642a086c6f63616c6974792a0873716c5f61646472300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201410a077072696d61727910001a0269641a04616464721a0a73657373696f6e5f69641a086c6f63616c6974791a0873716c5f61646472200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89b88a89","value":"030abe020a0b726f6c655f69645f7365711830200128013a00422a0a0576616c756510011a0c080110401800300050146000200030006800700078008001008801009801004800525c0a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e0010060006a250a0d0a0561646d696e10a00618a0060a0c0a04726f6f7410a00618a00612046e6f64651802800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e201180801106418ffffffff072064280032040800100038014200e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300"} ,{"key":"8b89ba8a89","value":"030aa5030a0a7370616e5f636f756e741832200128013a0042340a0973696e676c65746f6e10011a0c08001000180030005010600020002a04747275653000680070007800800100880100980100422f0a0a7370616e5f636f756e7410021a0c080110401800300050146000200030006800700078008001008801009801004803526e0a077072696d61727910011801220973696e676c65746f6e2a0a7370616e5f636f756e74300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100a201210a0973696e676c65746f6e120a73696e676c655f726f7718002801300038004002b201280a077072696d61727910001a0973696e676c65746f6e1a0a7370616e5f636f756e74200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 476f86033905..9460ceabf3ac 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -602,11 +602,14 @@ CREATE TABLE system.database_role_settings ( database_id OID NOT NULL, role_name STRING NOT NULL, settings STRING[] NOT NULL, + role_id OID NULL, CONSTRAINT "primary" PRIMARY KEY (database_id, role_name), + UNIQUE INDEX (database_id, role_id) STORING (settings), FAMILY "primary" ( database_id, - role_name, - settings + role_name, + settings, + role_id ) );` @@ -2660,14 +2663,14 @@ var ( {Name: "database_id", ID: 1, Type: types.Oid, Nullable: false}, {Name: "role_name", ID: 2, Type: types.String, Nullable: false}, {Name: "settings", ID: 3, Type: types.StringArray, Nullable: false}, + {Name: "role_id", ID: 4, Type: types.Oid, Nullable: true}, }, []descpb.ColumnFamilyDescriptor{ { - Name: "primary", - ID: 0, - ColumnNames: []string{"database_id", "role_name", "settings"}, - ColumnIDs: []descpb.ColumnID{1, 2, 3}, - DefaultColumnID: 3, + Name: "primary", + ID: 0, + ColumnNames: []string{"database_id", "role_name", "settings", "role_id"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4}, }, }, descpb.IndexDescriptor{ @@ -2680,6 +2683,18 @@ var ( }, KeyColumnIDs: []descpb.ColumnID{1, 2}, }, + descpb.IndexDescriptor{ + Name: "database_role_settings_database_id_role_id_key", + ID: 2, + Unique: true, + KeyColumnNames: []string{"database_id", "role_id"}, + KeyColumnDirections: []catenumpb.IndexColumn_Direction{catenumpb.IndexColumn_ASC, catenumpb.IndexColumn_ASC}, + KeyColumnIDs: []descpb.ColumnID{1, 4}, + KeySuffixColumnIDs: []descpb.ColumnID{2}, + StoreColumnNames: []string{"settings"}, + StoreColumnIDs: []descpb.ColumnID{3}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + }, )) // The tenant_usage table receives periodic updates from all SQL pods. We want diff --git a/pkg/sql/catalog/systemschema_test/testdata/bootstrap b/pkg/sql/catalog/systemschema_test/testdata/bootstrap index a9c8ccd8e2ae..655a5e9ae973 100644 --- a/pkg/sql/catalog/systemschema_test/testdata/bootstrap +++ b/pkg/sql/catalog/systemschema_test/testdata/bootstrap @@ -350,7 +350,9 @@ CREATE TABLE public.database_role_settings ( database_id OID NOT NULL, role_name STRING NOT NULL, settings STRING[] NOT NULL, - CONSTRAINT "primary" PRIMARY KEY (database_id ASC, role_name ASC) + role_id OID NULL, + CONSTRAINT "primary" PRIMARY KEY (database_id ASC, role_name ASC), + UNIQUE INDEX database_role_settings_database_id_role_id_key (database_id ASC, role_id ASC) STORING (settings) ); CREATE TABLE public.tenant_usage ( tenant_id INT8 NOT NULL, @@ -449,7 +451,7 @@ schema_telemetry {"database":{"name":"postgres","id":102,"modificationTime":{"wallTime":"0"},"version":"1","privileges":{"users":[{"userProto":"admin","privileges":"2","withGrantOption":"2"},{"userProto":"public","privileges":"2048"},{"userProto":"root","privileges":"2","withGrantOption":"2"}],"ownerProto":"root","version":2},"schemas":{"public":{"id":103}},"defaultPrivileges":{}}} {"database":{"name":"system","id":1,"modificationTime":{"wallTime":"0"},"version":"1","privileges":{"users":[{"userProto":"admin","privileges":"2048","withGrantOption":"2048"},{"userProto":"root","privileges":"2048","withGrantOption":"2048"}],"ownerProto":"node","version":2}}} {"table":{"name":"comments","id":24,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"type","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"object_id","id":2,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"sub_id","id":3,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"comment","id":4,"type":{"family":"StringFamily","oid":25}}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["type","object_id","sub_id"],"columnIds":[1,2,3]},{"name":"fam_4_comment","id":4,"columnNames":["comment"],"columnIds":[4],"defaultColumnId":4}],"nextFamilyId":5,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["type","object_id","sub_id"],"keyColumnDirections":["ASC","ASC","ASC"],"storeColumnNames":["comment"],"keyColumnIds":[1,2,3],"storeColumnIds":[4],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"public","privileges":"32"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} -{"table":{"name":"database_role_settings","id":44,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"database_id","id":1,"type":{"family":"OidFamily","oid":26}},{"name":"role_name","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"settings","id":3,"type":{"family":"ArrayFamily","arrayElemType":"StringFamily","oid":1009,"arrayContents":{"family":"StringFamily","oid":25}}}],"nextColumnId":4,"families":[{"name":"primary","columnNames":["database_id","role_name","settings"],"columnIds":[1,2,3],"defaultColumnId":3}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["database_id","role_name"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["settings"],"keyColumnIds":[1,2],"storeColumnIds":[3],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} +{"table":{"name":"database_role_settings","id":44,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"database_id","id":1,"type":{"family":"OidFamily","oid":26}},{"name":"role_name","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"settings","id":3,"type":{"family":"ArrayFamily","arrayElemType":"StringFamily","oid":1009,"arrayContents":{"family":"StringFamily","oid":25}}},{"name":"role_id","id":4,"type":{"family":"OidFamily","oid":26},"nullable":true}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["database_id","role_name","settings","role_id"],"columnIds":[1,2,3,4]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["database_id","role_name"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["settings","role_id"],"keyColumnIds":[1,2],"storeColumnIds":[3,4],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":2},"indexes":[{"name":"database_role_settings_database_id_role_id_key","id":2,"unique":true,"version":3,"keyColumnNames":["database_id","role_id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["settings"],"keyColumnIds":[1,4],"keySuffixColumnIds":[2],"storeColumnIds":[3],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"constraintId":1}],"nextIndexId":3,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":3}} {"table":{"name":"descriptor","id":3,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"descriptor","id":2,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":3,"families":[{"name":"primary","columnNames":["id"],"columnIds":[1]},{"name":"fam_2_descriptor","id":2,"columnNames":["descriptor"],"columnIds":[2],"defaultColumnId":2}],"nextFamilyId":3,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["descriptor"],"keyColumnIds":[1],"storeColumnIds":[2],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"32","withGrantOption":"32"},{"userProto":"root","privileges":"32","withGrantOption":"32"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"descriptor_id_seq","id":7,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"value","id":1,"type":{"family":"IntFamily","width":64,"oid":20}}],"families":[{"name":"primary","columnNames":["value"],"columnIds":[1],"defaultColumnId":1}],"primaryIndex":{"name":"primary","id":1,"version":4,"keyColumnNames":["value"],"keyColumnDirections":["ASC"],"keyColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{}},"privileges":{"users":[{"userProto":"admin","privileges":"32","withGrantOption":"32"},{"userProto":"root","privileges":"32","withGrantOption":"32"}],"ownerProto":"node","version":2},"formatVersion":3,"sequenceOpts":{"increment":"1","minValue":"1","maxValue":"9223372036854775807","start":"1","sequenceOwner":{},"cacheSize":"1"},"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"}}} {"table":{"name":"eventlog","id":12,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"timestamp","id":1,"type":{"family":"TimestampFamily","oid":1114}},{"name":"eventType","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"targetID","id":3,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"reportingID","id":4,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"info","id":5,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"uniqueID","id":6,"type":{"family":"BytesFamily","oid":17},"defaultExpr":"uuid_v4()"}],"nextColumnId":7,"families":[{"name":"primary","columnNames":["timestamp","uniqueID"],"columnIds":[1,6]},{"name":"fam_2_eventType","id":2,"columnNames":["eventType"],"columnIds":[2],"defaultColumnId":2},{"name":"fam_3_targetID","id":3,"columnNames":["targetID"],"columnIds":[3],"defaultColumnId":3},{"name":"fam_4_reportingID","id":4,"columnNames":["reportingID"],"columnIds":[4],"defaultColumnId":4},{"name":"fam_5_info","id":5,"columnNames":["info"],"columnIds":[5],"defaultColumnId":5}],"nextFamilyId":6,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["timestamp","uniqueID"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["eventType","targetID","reportingID","info"],"keyColumnIds":[1,6],"storeColumnIds":[2,3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index 7a5c2dee5614..d03f5961fd40 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -138,7 +138,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 41 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "secret", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "expiration", "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 41, "name": "join_tokens", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3], "storeColumnNames": ["secret", "expiration"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 42 {"table": {"checks": [{"columnIds": [11], "constraintId": 2, "expr": "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)", "fromHashShardedColumn": true, "name": "check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8"}], "columns": [{"id": 1, "name": "aggregated_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "agg_interval", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "metadata", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "statistics", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 10, "name": "plan", "type": {"family": "JsonFamily", "oid": 3802}}, {"computeExpr": "mod(fnv32(crdb_internal.datums_to_bytes(aggregated_ts, app_name, fingerprint_id, node_id, plan_hash, transaction_fingerprint_id)), 8:::INT8)", "hidden": true, "id": 11, "name": "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"defaultExpr": "ARRAY[]:::STRING[]", "id": 12, "name": "index_recommendations", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"computeExpr": "(statistics->'statistics':::STRING)->'indexes':::STRING", "id": 13, "name": "indexes_usage", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}, "virtual": true}], "formatVersion": 3, "id": 42, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [2, 3], "keyColumnNames": ["fingerprint_id", "transaction_fingerprint_id"], "keySuffixColumnIds": [11, 1, 4, 5, 6], "name": "fingerprint_stats_idx", "partitioning": {}, "sharded": {}, "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "invertedColumnKinds": ["DEFAULT"], "keyColumnDirections": ["ASC"], "keyColumnIds": [13], "keyColumnNames": ["indexes_usage"], "keySuffixColumnIds": [11, 1, 2, 3, 4, 5, 6], "name": "indexes_usage_idx", "partitioning": {}, "sharded": {}, "type": "INVERTED", "version": 3}], "name": "statement_statistics", "nextColumnId": 14, "nextConstraintId": 3, "nextIndexId": 4, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC", "ASC", "ASC", "ASC", "ASC", "ASC"], "keyColumnIds": [11, 1, 2, 3, 4, 5, 6], "keyColumnNames": ["crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", "aggregated_ts", "fingerprint_id", "transaction_fingerprint_id", "plan_hash", "app_name", "node_id"], "name": "primary", "partitioning": {}, "sharded": {"columnNames": ["aggregated_ts", "app_name", "fingerprint_id", "node_id", "plan_hash", "transaction_fingerprint_id"], "isSharded": true, "name": "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8", "shardBuckets": 8}, "storeColumnIds": [7, 8, 9, 10, 12], "storeColumnNames": ["agg_interval", "metadata", "statistics", "plan", "index_recommendations"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "admin", "withGrantOption": "32"}, {"privileges": "32", "userProto": "root", "withGrantOption": "32"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 43 {"table": {"checks": [{"columnIds": [8], "constraintId": 2, "expr": "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8)", "fromHashShardedColumn": true, "name": "check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8"}], "columns": [{"id": 1, "name": "aggregated_ts", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "agg_interval", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 6, "name": "metadata", "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "statistics", "type": {"family": "JsonFamily", "oid": 3802}}, {"computeExpr": "mod(fnv32(crdb_internal.datums_to_bytes(aggregated_ts, app_name, fingerprint_id, node_id)), 8:::INT8)", "hidden": true, "id": 8, "name": "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8", "type": {"family": "IntFamily", "oid": 23, "width": 32}}], "formatVersion": 3, "id": 43, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["fingerprint_id"], "keySuffixColumnIds": [8, 1, 3, 4], "name": "fingerprint_stats_idx", "partitioning": {}, "sharded": {}, "version": 3}], "name": "transaction_statistics", "nextColumnId": 9, "nextConstraintId": 3, "nextIndexId": 3, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC", "ASC", "ASC", "ASC"], "keyColumnIds": [8, 1, 2, 3, 4], "keyColumnNames": ["crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8", "aggregated_ts", "fingerprint_id", "app_name", "node_id"], "name": "primary", "partitioning": {}, "sharded": {"columnNames": ["aggregated_ts", "app_name", "fingerprint_id", "node_id"], "isSharded": true, "name": "crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8", "shardBuckets": 8}, "storeColumnIds": [5, 6, 7], "storeColumnNames": ["agg_interval", "metadata", "statistics"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "admin", "withGrantOption": "32"}, {"privileges": "32", "userProto": "root", "withGrantOption": "32"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} -44 {"table": {"columns": [{"id": 1, "name": "database_id", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "settings", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 44, "name": "database_role_settings", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 2], "keyColumnNames": ["database_id", "role_name"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [3], "storeColumnNames": ["settings"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} +44 {"table": {"columns": [{"id": 1, "name": "database_id", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "settings", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 4, "name": "role_id", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 44, "indexes": [{"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 4], "keyColumnNames": ["database_id", "role_id"], "keySuffixColumnIds": [2], "name": "database_role_settings_database_id_role_id_key", "partitioning": {}, "sharded": {}, "storeColumnIds": [3], "storeColumnNames": ["settings"], "unique": true, "version": 3}], "name": "database_role_settings", "nextColumnId": 5, "nextConstraintId": 3, "nextIndexId": 3, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 2, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 2], "keyColumnNames": ["database_id", "role_name"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 4], "storeColumnNames": ["settings", "role_id"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 45 {"table": {"columns": [{"id": 1, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "instance_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "next_instance_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "last_update", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 5, "name": "ru_burst_limit", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "ru_refill_rate", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 7, "name": "ru_current", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 8, "name": "current_share_sum", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 9, "name": "total_consumption", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 10, "name": "instance_lease", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 11, "name": "instance_seq", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "instance_shares", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 45, "name": "tenant_usage", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 2], "keyColumnNames": ["tenant_id", "instance_id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 4, 5, 6, 7, 8, 9, 10, 11, 12], "storeColumnNames": ["next_instance_id", "last_update", "ru_burst_limit", "ru_refill_rate", "ru_current", "current_share_sum", "total_consumption", "instance_lease", "instance_seq", "instance_shares"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 46 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "addr", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "session_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "locality", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "sql_addr", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 46, "name": "sql_instances", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5], "storeColumnNames": ["addr", "session_id", "locality", "sql_addr"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 47 {"table": {"checks": [{"columnIds": [1, 2], "constraintId": 2, "expr": "start_key < end_key", "name": "check_bounds"}], "columns": [{"id": 1, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 2, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "config", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 47, "name": "span_configurations", "nextColumnId": 4, "nextConstraintId": 3, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["start_key"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3], "storeColumnNames": ["end_key", "config"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} diff --git a/pkg/sql/logictest/testdata/logic_test/gen_test_objects b/pkg/sql/logictest/testdata/logic_test/gen_test_objects index 261c0f9fad31..5038b9d52580 100644 --- a/pkg/sql/logictest/testdata/logic_test/gen_test_objects +++ b/pkg/sql/logictest/testdata/logic_test/gen_test_objects @@ -193,16 +193,17 @@ ORDER BY table_name, column_name LIMIT 20 ---- database_role_settings "dataBase_ḯd" oid +database_role_settings "r""ole_i d" oid database_role_settings role_name text database_role_settings rowid bigint database_role_settings settings ARRAY ev̅entlog "eventType" text -ev̅entlog in😜f̈́o text +ev̅entlog info text ev̅entlog "reportingID" bigint ev̅entlog rowid bigint ev̅entlog "targetID" bigint -ev̅entlog "timestam̉ p" timestamp without time zone -ev̅entlog "uniqueID" bytea +ev̅entlog "uni̹queID" bytea +ev̅entlog "🙃t%vimestamp" timestamp without time zone namespace "name " text namespace "parent%42ID" bigint namespace "parentSchemaID" bigint @@ -211,7 +212,6 @@ namespace 😁id bigint "prote""cted_ts""_records" "Id" uuid "prote""cted_ts""_records" meta bytea "prote""cted_ts""_records" meta_type text -"prote""cted_ts""_records" num_spans bigint subtest templates/different_templates_in_each_db diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index f5fff214f3f7..2f96408959fa 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -1512,6 +1512,7 @@ system public 29_44_1_not_null system public 29_44_2_not_null system public database_role_settings CHECK NO NO system public 29_44_3_not_null system public database_role_settings CHECK NO NO system public primary system public database_role_settings PRIMARY KEY NO NO +system public database_role_settings_database_id_role_id_key system public database_role_settings UNIQUE NO NO system public 29_3_1_not_null system public descriptor CHECK NO NO system public primary system public descriptor PRIMARY KEY NO NO system public 29_7_1_not_null system public descriptor_id_seq CHECK NO NO @@ -1951,7 +1952,9 @@ table_catalog table_schema table_name column_name system public comments object_id system public primary system public comments sub_id system public primary system public comments type system public primary +system public database_role_settings database_id system public database_role_settings_database_id_role_id_key system public database_role_settings database_id system public primary +system public database_role_settings role_id system public database_role_settings_database_id_role_id_key system public database_role_settings role_name system public primary system public descriptor id system public primary system public descriptor_id_seq value system public primary @@ -2148,6 +2151,7 @@ system public comments object_id system public comments sub_id 3 system public comments type 1 system public database_role_settings database_id 1 +system public database_role_settings role_id 4 system public database_role_settings role_name 2 system public database_role_settings settings 3 system public descriptor descriptor 2 diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index b0eaf5da2968..e59d8a3f9c3f 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1130,6 +1130,7 @@ indexrelid indrelid indnatts indisunique indnullsnotdistinct indisprimary 969972501 57 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1 1062763829 25 4 true false true false true false true false false true false 1 2 3 4 0 0 3403232968 3403232968 0 0 0 0 2 2 2 2 NULL NULL 4 1183313104 44 2 true false true false true false true false false true false 1 2 0 3403232968 0 0 2 2 NULL NULL 2 +1183313107 44 3 true false false false true false true false false true false 1 4 3 0 0 0 0 2 2 NULL NULL 2 1276104432 12 2 true false true false true false true false false true false 1 6 0 0 0 0 2 2 NULL NULL 2 1322500096 28 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1 1489445036 35 6 false false false false false false true false false true false 2 1 3 6 7 8 0 0 0 0 2 2 NULL NULL 2 @@ -1225,6 +1226,8 @@ indexrelid operator_argument_type_oid operator_argument_position 1062763829 0 4 1183313104 0 1 1183313104 0 2 +1183313107 0 1 +1183313107 0 2 1276104432 0 1 1276104432 0 2 1322500096 0 1 diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index f39274e13dc2..d32e8b4d7c9f 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "alter_table_statistics_partial_predicate_and_id.go", "create_index_usage_statement_statistics.go", "create_jobs_metrics_polling_job.go", + "database_role_settings_table_user_id_migration.go", "delete_descriptors_of_dropped_functions.go", "desc_id_sequence_for_system_tenant.go", "descriptor_utils.go", @@ -95,6 +96,7 @@ go_test( "builtins_test.go", "create_index_usage_statement_statistics_test.go", "create_jobs_metrics_polling_job_test.go", + "database_role_settings_table_user_id_migration_test.go", "delete_descriptors_of_dropped_functions_test.go", "desc_id_sequence_for_system_tenant_test.go", "descriptor_utils_test.go", diff --git a/pkg/upgrade/upgrades/database_role_settings_table_user_id_migration.go b/pkg/upgrade/upgrades/database_role_settings_table_user_id_migration.go new file mode 100644 index 000000000000..7763263373cb --- /dev/null +++ b/pkg/upgrade/upgrades/database_role_settings_table_user_id_migration.go @@ -0,0 +1,86 @@ +// 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 upgrades + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/upgrade" +) + +const addRoleIDColumnToDatabaseRoleSettingsTableStmt = ` +ALTER TABLE system.database_role_settings +ADD COLUMN IF NOT EXISTS role_id OID +FAMILY "primary" +` + +const createUniqueIndexOnDatabaseIDAndRoleIDOnDatabaseRoleSettingsTableStmt = ` +CREATE UNIQUE INDEX IF NOT EXISTS database_role_settings_database_id_role_id_key +ON system.database_role_settings (database_id, role_id) +STORING (settings) +` + +func alterDatabaseRoleSettingsTableAddRoleIDColumn( + ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, +) error { + for _, op := range []operation{ + { + name: "add-role-id-column-database-role-settings-table", + schemaList: []string{"role_id"}, + query: addRoleIDColumnToDatabaseRoleSettingsTableStmt, + schemaExistsFn: columnExists, + }, + { + name: "add-database-id-role-id-unique-index-database-role-settings-table", + schemaList: []string{"database_role_settings_database_id_role_id_key"}, + query: createUniqueIndexOnDatabaseIDAndRoleIDOnDatabaseRoleSettingsTableStmt, + schemaExistsFn: hasIndex, + }, + } { + if err := migrateTable(ctx, cs, d, op, keys.DatabaseRoleSettingsTableID, systemschema.DatabaseRoleSettingsTable); err != nil { + return err + } + } + + return nil +} + +const backfillRoleIDColumnDatabaseRoleSettingsTableStmt = ` +UPDATE system.database_role_settings +SET role_id = user_id +FROM system.users +WHERE role_id IS NULL AND role_name = username +LIMIT 1000 +` + +func backfillDatabaseRoleSettingsTableRoleIDColumn( + ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, +) error { + ie := d.DB.Executor() + for { + rowsAffected, err := ie.ExecEx(ctx, "backfill-role-id-database-role-settings-table", nil, /* txn */ + sessiondata.NodeUserSessionDataOverride, + backfillRoleIDColumnDatabaseRoleSettingsTableStmt, + ) + if err != nil { + return err + } + if rowsAffected == 0 { + break + } + } + + return nil +} diff --git a/pkg/upgrade/upgrades/database_role_settings_table_user_id_migration_test.go b/pkg/upgrade/upgrades/database_role_settings_table_user_id_migration_test.go new file mode 100644 index 000000000000..ff6aaa25ece7 --- /dev/null +++ b/pkg/upgrade/upgrades/database_role_settings_table_user_id_migration_test.go @@ -0,0 +1,168 @@ +// 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 upgrades_test + +import ( + "context" + "fmt" + "strconv" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catenumpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func TestDatabaseRoleSettingsUserIDMigrationNoUsers(t *testing.T) { + runTestDatabaseRoleSettingsUserIDMigration(t, 0) +} + +func TestDatabaseRoleSettingsUserIDMigration10Users(t *testing.T) { + runTestDatabaseRoleSettingsUserIDMigration(t, 10) +} + +func TestDatabaseRoleSettingsUserIDMigration1500Users(t *testing.T) { + skip.UnderRace(t) + skip.UnderStress(t) + runTestDatabaseRoleSettingsUserIDMigration(t, 1500) +} + +func runTestDatabaseRoleSettingsUserIDMigration(t *testing.T, numUsers int) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + + settings := cluster.MakeTestingClusterSettingsWithVersions( + clusterversion.TestingBinaryVersion, + clusterversion.ByKey(clusterversion.V23_1DatabaseRoleSettingsHasRoleIDColumn-1), + false, /* initializeVersion */ + ) + tc := testcluster.StartTestCluster(t, 1 /* nodes */, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: clusterversion.ByKey(clusterversion.V23_1DatabaseRoleSettingsHasRoleIDColumn - 1), + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + db := tc.ServerConn(0) + defer db.Close() + tdb := sqlutils.MakeSQLRunner(db) + s := tc.Server(0) + + // Inject the descriptor for the system.database_role_settings table from + // before the role_id column was added. + upgrades.InjectLegacyTable(ctx, t, s, systemschema.DatabaseRoleSettingsTable, getTableDescForDatabaseRoleSettingsTableBeforeRoleIDCol) + + // Create test users. + tx, err := db.BeginTx(ctx, nil /* opts */) + require.NoError(t, err) + txRunner := sqlutils.MakeSQLRunner(tx) + for i := 0; i < numUsers; i++ { + // Group statements into transactions of 100 users to speed up creation. + if i != 0 && i%100 == 0 { + err := tx.Commit() + require.NoError(t, err) + tx, err = db.BeginTx(ctx, nil /* opts */) + require.NoError(t, err) + txRunner = sqlutils.MakeSQLRunner(tx) + } + txRunner.Exec(t, fmt.Sprintf("CREATE USER testuser%d", i)) + txRunner.Exec(t, fmt.Sprintf(`ALTER USER testuser%d SET application_name = 'roach sql'`, i)) + } + err = tx.Commit() + require.NoError(t, err) + tdb.CheckQueryResults(t, "SELECT count(*) FROM system.database_role_settings", [][]string{{strconv.Itoa(numUsers)}}) + + // Run migrations. + _, err = tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.V23_1DatabaseRoleSettingsHasRoleIDColumn).String()) + require.NoError(t, err) + _, err = tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.V23_1DatabaseRoleSettingsRoleIDColumnBackfilled).String()) + require.NoError(t, err) + + // Verify that the final schema matches the expected one. + expectedSchema := `CREATE TABLE public.database_role_settings ( + database_id OID NOT NULL, + role_name STRING NOT NULL, + settings STRING[] NOT NULL, + role_id OID NULL, + CONSTRAINT "primary" PRIMARY KEY (database_id ASC, role_name ASC), + UNIQUE INDEX database_role_settings_database_id_role_id_key (database_id ASC, role_id ASC) STORING (settings) +)` + r := tdb.QueryRow(t, "SELECT create_statement FROM [SHOW CREATE TABLE system.database_role_settings]") + var actualSchema string + r.Scan(&actualSchema) + require.Equal(t, expectedSchema, actualSchema) + + // Check that the backfill was successful and correct. + tdb.CheckQueryResults(t, "SELECT * FROM system.database_role_settings WHERE role_id IS NULL", [][]string{}) + tdb.CheckQueryResults(t, "SELECT count(*) FROM system.database_role_settings", [][]string{{strconv.Itoa(numUsers)}}) + tdb.CheckQueryResults(t, "SELECT count(*) FROM system.database_role_settings AS a JOIN system.users AS b ON a.role_name = b.username AND a.role_id <> b.user_id", [][]string{{"0"}}) +} + +func getTableDescForDatabaseRoleSettingsTableBeforeRoleIDCol() *descpb.TableDescriptor { + return &descpb.TableDescriptor{ + Name: string(catconstants.DatabaseRoleSettingsTableName), + ID: keys.DatabaseRoleSettingsTableID, + ParentID: keys.SystemDatabaseID, + UnexposedParentSchemaID: keys.PublicSchemaID, + Version: 1, + Columns: []descpb.ColumnDescriptor{ + {Name: "database_id", ID: 1, Type: types.Oid, Nullable: false}, + {Name: "role_name", ID: 2, Type: types.String, Nullable: false}, + {Name: "settings", ID: 3, Type: types.StringArray, Nullable: false}, + }, + NextColumnID: 4, + Families: []descpb.ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + ColumnNames: []string{"database_id", "role_name", "settings"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3}, + DefaultColumnID: 3, + }, + }, + NextFamilyID: 1, + PrimaryIndex: descpb.IndexDescriptor{ + Name: "primary", + ID: 1, + Unique: true, + KeyColumnNames: []string{"database_id", "role_name"}, + KeyColumnDirections: []catenumpb.IndexColumn_Direction{ + catenumpb.IndexColumn_ASC, catenumpb.IndexColumn_ASC, + }, + KeyColumnIDs: []descpb.ColumnID{1, 2}, + }, + NextIndexID: 2, + FormatVersion: descpb.InterleavedFormatVersion, + NextMutationID: 1, + NextConstraintID: 1, + } +} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index e8f7634a709c..e38724bdfd68 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -298,6 +298,18 @@ var upgrades = []upgradebase.Upgrade{ upgrade.NoPrecondition, alterSystemPrivilegesAddSecondaryIndex, ), + upgrade.NewTenantUpgrade( + "add role_id column to system.database_role_settings table", + toCV(clusterversion.V23_1DatabaseRoleSettingsHasRoleIDColumn), + upgrade.NoPrecondition, + alterDatabaseRoleSettingsTableAddRoleIDColumn, + ), + upgrade.NewTenantUpgrade( + "backfill role_id column in system.database_role_settings table", + toCV(clusterversion.V23_1DatabaseRoleSettingsRoleIDColumnBackfilled), + upgrade.NoPrecondition, + backfillDatabaseRoleSettingsTableRoleIDColumn, + ), } func init() {