diff --git a/dev b/dev index 09779affa734..f0cf57df5930 100755 --- a/dev +++ b/dev @@ -8,7 +8,7 @@ fi set -euo pipefail # Bump this counter to force rebuilding `dev` on all machines. -DEV_VERSION=70 +DEV_VERSION=71 THIS_DIR=$(cd "$(dirname "$0")" && pwd) BINARY_DIR=$THIS_DIR/bin/dev-versions diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 80ffae9365f9..f02a650f5db4 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -2061,6 +2061,7 @@ GO_TARGETS = [ "//pkg/testutils/storageutils:storageutils", "//pkg/testutils/testcluster:testcluster", "//pkg/testutils/testcluster:testcluster_test", + "//pkg/testutils/testfixtures:testfixtures", "//pkg/testutils/zerofields:zerofields", "//pkg/testutils/zerofields:zerofields_test", "//pkg/testutils:testutils", @@ -3202,6 +3203,7 @@ GET_X_DATA_TARGETS = [ "//pkg/testutils/sqlutils:get_x_data", "//pkg/testutils/storageutils:get_x_data", "//pkg/testutils/testcluster:get_x_data", + "//pkg/testutils/testfixtures:get_x_data", "//pkg/testutils/zerofields:get_x_data", "//pkg/ts:get_x_data", "//pkg/ts/catalog:get_x_data", diff --git a/pkg/ccl/storageccl/engineccl/.gitignore b/pkg/ccl/storageccl/engineccl/.gitignore index 77e451b2ff7a..ccb9cfba2d07 100644 --- a/pkg/ccl/storageccl/engineccl/.gitignore +++ b/pkg/ccl/storageccl/engineccl/.gitignore @@ -3,5 +3,4 @@ # Old benchmark data. mvcc_data -# New benchmark data. mvcc_data_* diff --git a/pkg/ccl/storageccl/engineccl/BUILD.bazel b/pkg/ccl/storageccl/engineccl/BUILD.bazel index 0af6249534ed..fddb73b09496 100644 --- a/pkg/ccl/storageccl/engineccl/BUILD.bazel +++ b/pkg/ccl/storageccl/engineccl/BUILD.bazel @@ -53,6 +53,7 @@ go_test( "//pkg/testutils", "//pkg/testutils/datapathutils", "//pkg/testutils/storageutils", + "//pkg/testutils/testfixtures", "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/leaktest", @@ -61,7 +62,6 @@ go_test( "//pkg/util/randutil", "//pkg/util/timeutil", "@com_github_cockroachdb_datadriven//:datadriven", - "@com_github_cockroachdb_errors//oserror", "@com_github_cockroachdb_pebble//:pebble", "@com_github_cockroachdb_pebble//vfs", "@com_github_cockroachdb_pebble//vfs/atomicfs", diff --git a/pkg/ccl/storageccl/engineccl/bench_test.go b/pkg/ccl/storageccl/engineccl/bench_test.go index 3bbd0b656c15..0b8fa255f042 100644 --- a/pkg/ccl/storageccl/engineccl/bench_test.go +++ b/pkg/ccl/storageccl/engineccl/bench_test.go @@ -12,7 +12,6 @@ import ( "context" "fmt" "math/rand" - "os" "path/filepath" "testing" @@ -22,11 +21,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testfixtures" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" - "github.com/cockroachdb/errors/oserror" ) // loadTestData writes numKeys keys in numBatches separate batches. Keys are @@ -44,85 +43,82 @@ import ( // The creation of the database is time consuming, so the caller can choose // whether to use a temporary or permanent location. func loadTestData( - dirPrefix string, numKeys, numBatches, batchTimeSpan, valueBytes int, -) (storage.Engine, error) { + tb testing.TB, dirPrefix string, numKeys, numBatches, batchTimeSpan, valueBytes int, +) storage.Engine { ctx := context.Background() verStr := fmt.Sprintf("v%s", clusterversion.TestingBinaryVersion.String()) - dir := fmt.Sprintf("%s_v%s_%d_%d_%d_%d", dirPrefix, verStr, numKeys, numBatches, batchTimeSpan, valueBytes) - - exists := true - if _, err := os.Stat(dir); oserror.IsNotExist(err) { - exists = false - } - - eng, err := storage.Open( - ctx, - storage.Filesystem(dir), - cluster.MakeTestingClusterSettings()) - if err != nil { - return nil, err - } - - absPath, err := filepath.Abs(dir) - if err != nil { - absPath = dir - } - - if exists { - log.Infof(context.Background(), "using existing test data: %s", absPath) - testutils.ReadAllFiles(filepath.Join(dir, "*")) - return eng, nil - } + name := fmt.Sprintf("%s_v%s_%d_%d_%d_%d", dirPrefix, verStr, numKeys, numBatches, batchTimeSpan, valueBytes) + dir := testfixtures.ReuseOrGenerate(tb, name, func(dir string) { + eng, err := storage.Open( + ctx, + storage.Filesystem(dir), + cluster.MakeTestingClusterSettings()) + if err != nil { + tb.Fatal(err) + } - log.Infof(context.Background(), "creating test data: %s", absPath) + log.Infof(context.Background(), "creating test data: %s", dir) - // Generate the same data every time. - rng := rand.New(rand.NewSource(1449168817)) + // Generate the same data every time. + rng := rand.New(rand.NewSource(1449168817)) - keys := make([]roachpb.Key, numKeys) - for i := 0; i < numKeys; i++ { - keys[i] = roachpb.Key(encoding.EncodeUvarintAscending([]byte("key-"), uint64(i))) - } + keys := make([]roachpb.Key, numKeys) + for i := 0; i < numKeys; i++ { + keys[i] = roachpb.Key(encoding.EncodeUvarintAscending([]byte("key-"), uint64(i))) + } - sstTimestamps := make([]int64, numBatches) - for i := 0; i < len(sstTimestamps); i++ { - sstTimestamps[i] = int64((i + 1) * batchTimeSpan) - } + sstTimestamps := make([]int64, numBatches) + for i := 0; i < len(sstTimestamps); i++ { + sstTimestamps[i] = int64((i + 1) * batchTimeSpan) + } - var batch storage.Batch - var minWallTime int64 - for i, key := range keys { - if scaled := len(keys) / numBatches; (i % scaled) == 0 { - if i > 0 { - log.Infof(ctx, "committing (%d/~%d)", i/scaled, numBatches) - if err := batch.Commit(false /* sync */); err != nil { - return nil, err - } - batch.Close() - if err := eng.Flush(); err != nil { - return nil, err + var batch storage.Batch + var minWallTime int64 + for i, key := range keys { + if scaled := len(keys) / numBatches; (i % scaled) == 0 { + if i > 0 { + log.Infof(ctx, "committing (%d/~%d)", i/scaled, numBatches) + if err := batch.Commit(false /* sync */); err != nil { + tb.Fatal(err) + } + batch.Close() + if err := eng.Flush(); err != nil { + tb.Fatal(err) + } } + batch = eng.NewBatch() + minWallTime = sstTimestamps[i/scaled] + } + timestamp := hlc.Timestamp{WallTime: minWallTime + rand.Int63n(int64(batchTimeSpan))} + value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, valueBytes)) + value.InitChecksum(key) + if err := storage.MVCCPut(ctx, batch, nil, key, timestamp, hlc.ClockTimestamp{}, value, nil); err != nil { + tb.Fatal(err) } - batch = eng.NewBatch() - minWallTime = sstTimestamps[i/scaled] } - timestamp := hlc.Timestamp{WallTime: minWallTime + rand.Int63n(int64(batchTimeSpan))} - value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, valueBytes)) - value.InitChecksum(key) - if err := storage.MVCCPut(ctx, batch, nil, key, timestamp, hlc.ClockTimestamp{}, value, nil); err != nil { - return nil, err + if err := batch.Commit(false /* sync */); err != nil { + tb.Fatal(err) } - } - if err := batch.Commit(false /* sync */); err != nil { - return nil, err - } - batch.Close() - if err := eng.Flush(); err != nil { - return nil, err - } + batch.Close() + if err := eng.Flush(); err != nil { + tb.Fatal(err) + } + eng.Close() + }) - return eng, nil + log.Infof(context.Background(), "using test data: %s", dir) + eng, err := storage.Open( + ctx, + storage.Filesystem(dir), + cluster.MakeTestingClusterSettings(), + storage.MustExist, + ) + if err != nil { + tb.Fatal(err) + } + testutils.ReadAllFiles(filepath.Join(dir, "*")) + return eng } // runIterate benchmarks iteration over the entire keyspace within time bounds @@ -140,10 +136,7 @@ func runIterate( // Store the database in this directory so we don't have to regenerate it on // each benchmark run. - eng, err := loadTestData("mvcc_data", numKeys, numBatches, batchTimeSpan, valueBytes) - if err != nil { - b.Fatal(err) - } + eng := loadTestData(b, "mvcc_data", numKeys, numBatches, batchTimeSpan, valueBytes) defer eng.Close() b.SetBytes(int64(numKeys * valueBytes)) diff --git a/pkg/cmd/dev/bench.go b/pkg/cmd/dev/bench.go index f158f02ae8e9..562f04f016f6 100644 --- a/pkg/cmd/dev/bench.go +++ b/pkg/cmd/dev/bench.go @@ -12,6 +12,8 @@ package main import ( "fmt" + "os" + "path/filepath" "strings" "github.com/spf13/cobra" @@ -154,8 +156,23 @@ func (d *dev) bench(cmd *cobra.Command, commandLine []string) error { } args = append(args, goTestArgs...) } + args = append(args, d.getGoTestEnvArgs()...) args = append(args, d.getTestOutputArgs(false /* stress */, verbose, showLogs, streamOutput)...) args = append(args, additionalBazelArgs...) logCommand("bazel", args...) return d.exec.CommandContextInheritingStdStreams(ctx, "bazel", args...) } + +func (d *dev) getGoTestEnvArgs() []string { + var goTestEnv []string + // Make the `$HOME/.cache/crdb-test-fixtures` directory available for reusable + // test fixtures, if available. See testfixtures.ReuseOrGenerate(). + if cacheDir, err := os.UserCacheDir(); err == nil { + dir := filepath.Join(cacheDir, "crdb-test-fixtures") + if err := os.MkdirAll(dir, 0755); err == nil { + goTestEnv = append(goTestEnv, "--test_env", fmt.Sprintf("COCKROACH_TEST_FIXTURES_DIR=%s", dir)) + goTestEnv = append(goTestEnv, fmt.Sprintf("--sandbox_writable_path=%s", dir)) + } + } + return goTestEnv +} diff --git a/pkg/cmd/roachtest/tests/mixed_version_backup.go b/pkg/cmd/roachtest/tests/mixed_version_backup.go index bf5395179fe2..529a5c2f1f31 100644 --- a/pkg/cmd/roachtest/tests/mixed_version_backup.go +++ b/pkg/cmd/roachtest/tests/mixed_version_backup.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/cockroach/pkg/util/version" "github.com/cockroachdb/errors" "golang.org/x/sync/errgroup" ) @@ -54,6 +55,14 @@ var ( Multiplier: 1.5, MaxRetries: 50, } + + v231 = func() *version.Version { + v, err := version.Parse("v23.1.0") + if err != nil { + panic(fmt.Sprintf("failure parsing version: %v", err)) + } + return v + }() ) // sanitizeVersionForBackup takes the string representation of a @@ -63,6 +72,25 @@ func sanitizeVersionForBackup(v string) string { return invalidVersionRE.ReplaceAllString(clusterupgrade.VersionMsg(v), "") } +// hasInternalSystemJobs returns true if the cluster is expected to +// have the `crdb_internal.system_jobs` vtable in the mixed-version +// context passed. If so, it should be used instead of `system.jobs` +// when querying job status. +func hasInternalSystemJobs(tc *mixedversion.Context) bool { + lowestVersion := tc.FromVersion // upgrades + if tc.FromVersion == clusterupgrade.MainVersion { + lowestVersion = tc.ToVersion // downgrades + } + + // Add 'v' prefix expected by `version` package. + lowestVersion = "v" + lowestVersion + sv, err := version.Parse(lowestVersion) + if err != nil { + panic(fmt.Errorf("internal error: test context version (%s) expected to be parseable: %w", lowestVersion, err)) + } + return sv.AtLeast(v231) +} + type ( // backupOption is an option passed to the `BACKUP` command (i.e., // `WITH ...` portion). @@ -298,11 +326,17 @@ func (mvb *mixedVersionBackup) waitForJobSuccess( var lastErr error node, db := h.RandomDB(rng, mvb.roachNodes) l.Printf("querying job status through node %d", node) + + jobsQuery := "system.jobs WHERE id = $1" + if hasInternalSystemJobs(h.Context()) { + jobsQuery = fmt.Sprintf("(%s)", jobutils.InternalSystemJobsBaseQuery) + } for r := retry.StartWithCtx(ctx, backupCompletionRetryOptions); r.Next(); { var status string var payloadBytes []byte - err := db.QueryRow(fmt.Sprintf(`SELECT status, payload FROM (%s)`, - jobutils.InternalSystemJobsBaseQuery), jobID).Scan(&status, &payloadBytes) + err := db.QueryRow( + fmt.Sprintf(`SELECT status, payload FROM %s`, jobsQuery), jobID, + ).Scan(&status, &payloadBytes) if err != nil { lastErr = fmt.Errorf("error reading (status, payload) for job %d: %w", jobID, err) l.Printf("%v", lastErr) diff --git a/pkg/cmd/roachtest/tests/multitenant_distsql.go b/pkg/cmd/roachtest/tests/multitenant_distsql.go index dad9efcc0ff5..91022c0201eb 100644 --- a/pkg/cmd/roachtest/tests/multitenant_distsql.go +++ b/pkg/cmd/roachtest/tests/multitenant_distsql.go @@ -57,9 +57,14 @@ func runMultiTenantDistSQL( timeoutMillis int, ) { c.Put(ctx, t.Cockroach(), "./cockroach") - c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(install.SecureOption(true)), c.Node(1)) - c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(install.SecureOption(true)), c.Node(2)) - c.Start(ctx, t.L(), option.DefaultStartOpts(), install.MakeClusterSettings(install.SecureOption(true)), c.Node(3)) + // This test sets a smaller default range size than the default due to + // performance and resource limitations. We set the minimum range max bytes to + // 1 byte to bypass the guardrails. + settings := install.MakeClusterSettings(install.SecureOption(true)) + settings.Env = append(settings.Env, "COCKROACH_MIN_RANGE_MAX_BYTES=1") + c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, c.Node(1)) + c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, c.Node(2)) + c.Start(ctx, t.L(), option.DefaultStartOpts(), settings, c.Node(3)) const ( tenantID = 11 diff --git a/pkg/config/system.go b/pkg/config/system.go index 0fc410029746..91bea3f4cb6c 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -495,7 +495,7 @@ func StaticSplits() []roachpb.RKey { // However, splits are not required between the tables of secondary tenants. func (s *SystemConfig) ComputeSplitKey( ctx context.Context, startKey, endKey roachpb.RKey, -) (rr roachpb.RKey) { +) (rr roachpb.RKey, _ error) { // Before dealing with splits necessitated by SQL tables, handle all of the // static splits earlier in the keyspace. Note that this list must be kept in // the proper order (ascending in the keyspace) for the logic below to work. @@ -508,11 +508,11 @@ func (s *SystemConfig) ComputeSplitKey( if split.Less(endKey) { // The split point is contained within [startKey, endKey), so we need to // create the split. - return split + return split, nil } // [startKey, endKey) is contained between the previous split point and // this split point. - return nil + return nil, nil } // [startKey, endKey) is somewhere greater than this split point. Continue. } @@ -521,12 +521,12 @@ func (s *SystemConfig) ComputeSplitKey( // anything, the key range must be somewhere in the SQL table part of the // keyspace. First, look for split keys within the system-tenant's keyspace. if split := s.systemTenantTableBoundarySplitKey(ctx, startKey, endKey); split != nil { - return split + return split, nil } // If the system tenant does not have any splits, look for split keys at the // boundary of each secondary tenant. - return s.tenantBoundarySplitKey(ctx, startKey, endKey) + return s.tenantBoundarySplitKey(ctx, startKey, endKey), nil } func (s *SystemConfig) systemTenantTableBoundarySplitKey( @@ -700,8 +700,15 @@ func (s *SystemConfig) tenantBoundarySplitKey( // NeedsSplit returns whether the range [startKey, endKey) needs a split due // to zone configs. -func (s *SystemConfig) NeedsSplit(ctx context.Context, startKey, endKey roachpb.RKey) bool { - return len(s.ComputeSplitKey(ctx, startKey, endKey)) > 0 +func (s *SystemConfig) NeedsSplit( + ctx context.Context, startKey, endKey roachpb.RKey, +) (bool, error) { + splits, err := s.ComputeSplitKey(ctx, startKey, endKey) + if err != nil { + return false, err + } + + return len(splits) > 0, nil } // shouldSplitOnSystemTenantObject checks if the ID is eligible for a split at diff --git a/pkg/config/system_test.go b/pkg/config/system_test.go index dce068d505de..1fcdf6d86e80 100644 --- a/pkg/config/system_test.go +++ b/pkg/config/system_test.go @@ -332,7 +332,8 @@ func TestComputeSplitKeySystemRanges(t *testing.T) { Values: kvs, } for tcNum, tc := range testCases { - splitKey := cfg.ComputeSplitKey(context.Background(), tc.start, tc.end) + splitKey, err := cfg.ComputeSplitKey(context.Background(), tc.start, tc.end) + require.NoError(t, err) expected := roachpb.RKey(tc.split) if !splitKey.Equal(expected) { t.Errorf("#%d: bad split:\ngot: %v\nexpected: %v", tcNum, splitKey, expected) @@ -443,7 +444,8 @@ func TestComputeSplitKeyTableIDs(t *testing.T) { cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) for tcNum, tc := range testCases { cfg.Values = tc.values - splitKey := cfg.ComputeSplitKey(context.Background(), tc.start, tc.end) + splitKey, err := cfg.ComputeSplitKey(context.Background(), tc.start, tc.end) + require.NoError(t, err) if !splitKey.Equal(tc.split) { t.Errorf("#%d: bad split:\ngot: %v\nexpected: %v", tcNum, splitKey, tc.split) } @@ -530,7 +532,8 @@ func TestComputeSplitKeyTenantBoundaries(t *testing.T) { cfg := config.NewSystemConfig(zonepb.DefaultZoneConfigRef()) for tcNum, tc := range testCases { cfg.Values = tc.values - splitKey := cfg.ComputeSplitKey(context.Background(), tc.start, tc.end) + splitKey, err := cfg.ComputeSplitKey(context.Background(), tc.start, tc.end) + require.NoError(t, err) if !splitKey.Equal(tc.split) { t.Errorf("#%d: bad split:\ngot: %v\nexpected: %v", tcNum, splitKey, tc.split) } diff --git a/pkg/jobs/job_info_storage.go b/pkg/jobs/job_info_storage.go index 07c27ae526f8..6de5509bb2cb 100644 --- a/pkg/jobs/job_info_storage.go +++ b/pkg/jobs/job_info_storage.go @@ -66,7 +66,7 @@ func (i InfoStorage) checkClaimSession(ctx context.Context) error { return nil } -func (i InfoStorage) get(ctx context.Context, infoKey []byte) ([]byte, bool, error) { +func (i InfoStorage) get(ctx context.Context, infoKey string) ([]byte, bool, error) { if i.txn == nil { return nil, false, errors.New("cannot access the job info table without an associated txn") } @@ -82,7 +82,7 @@ func (i InfoStorage) get(ctx context.Context, infoKey []byte) ([]byte, bool, err row, err := i.txn.QueryRowEx( ctx, "job-info-get", i.txn.KV(), sessiondata.NodeUserSessionDataOverride, - "SELECT value FROM system.job_info WHERE job_id = $1 AND info_key = $2 ORDER BY written LIMIT 1", + "SELECT value FROM system.job_info WHERE job_id = $1 AND info_key::string = $2 ORDER BY written LIMIT 1", j.ID(), infoKey, ) @@ -102,13 +102,13 @@ func (i InfoStorage) get(ctx context.Context, infoKey []byte) ([]byte, bool, err return []byte(*value), true, nil } -func (i InfoStorage) write(ctx context.Context, infoKey, value []byte) error { +func (i InfoStorage) write(ctx context.Context, infoKey string, value []byte) error { return i.doWrite(ctx, func(ctx context.Context, j *Job, txn isql.Txn) error { // First clear out any older revisions of this info. _, err := txn.ExecEx( ctx, "write-job-info-delete", txn.KV(), sessiondata.NodeUserSessionDataOverride, - "DELETE FROM system.job_info WHERE job_id = $1 AND info_key = $2", + "DELETE FROM system.job_info WHERE job_id = $1 AND info_key::string = $2", j.ID(), infoKey, ) if err != nil { @@ -156,8 +156,8 @@ func (i InfoStorage) doWrite( func (i InfoStorage) iterate( ctx context.Context, iterMode iterateMode, - infoPrefix []byte, - fn func(infoKey, value []byte) error, + infoPrefix string, + fn func(infoKey string, value []byte) error, ) (retErr error) { if i.txn == nil { return errors.New("cannot iterate over the job info table without an associated txn") @@ -184,14 +184,14 @@ func (i InfoStorage) iterate( FROM system.job_info WHERE job_id = $1 AND info_key >= $2 AND info_key < $3 `+iterConfig, - i.j.ID(), infoPrefix, roachpb.Key(infoPrefix).PrefixEnd(), + i.j.ID(), infoPrefix, string(roachpb.Key(infoPrefix).PrefixEnd()), ) if err != nil { return err } defer func(it isql.Rows) { retErr = errors.CombineErrors(retErr, it.Close()) }(rows) - var prevKey []byte + var prevKey string var ok bool for ok, err = rows.Next(ctx); ok; ok, err = rows.Next(ctx) { if err != nil { @@ -199,16 +199,16 @@ func (i InfoStorage) iterate( } row := rows.Cur() - key, ok := row[0].(*tree.DBytes) + key, ok := row[0].(*tree.DString) if !ok { - return errors.AssertionFailedf("job info: expected info_key to be DBytes (was %T)", row[0]) + return errors.AssertionFailedf("job info: expected info_key to be string (was %T)", row[0]) } - infoKey := []byte(*key) + infoKey := string(*key) - if bytes.Equal(infoKey, prevKey) { + if infoKey == prevKey { continue } - prevKey = append(prevKey[:0], infoKey...) + prevKey = infoKey value, ok := row[1].(*tree.DBytes) if !ok { @@ -223,7 +223,7 @@ func (i InfoStorage) iterate( } // Get fetches the latest info record for the given job and infoKey. -func (i InfoStorage) Get(ctx context.Context, infoKey []byte) ([]byte, bool, error) { +func (i InfoStorage) Get(ctx context.Context, infoKey string) ([]byte, bool, error) { return i.get(ctx, infoKey) } @@ -231,21 +231,21 @@ func (i InfoStorage) Get(ctx context.Context, infoKey []byte) ([]byte, bool, err // infoKey after removing any existing info records for that job and infoKey // using the same transaction, effectively replacing any older row with a row // with the new value. -func (i InfoStorage) Write(ctx context.Context, infoKey, value []byte) error { +func (i InfoStorage) Write(ctx context.Context, infoKey string, value []byte) error { if value == nil { - return errors.AssertionFailedf("missing value (infoKey %q)", string(infoKey)) + return errors.AssertionFailedf("missing value (infoKey %q)", infoKey) } return i.write(ctx, infoKey, value) } // Delete removes the info record for the provided infoKey. -func (i InfoStorage) Delete(ctx context.Context, infoKey []byte) error { +func (i InfoStorage) Delete(ctx context.Context, infoKey string) error { return i.write(ctx, infoKey, nil /* value */) } // DeleteRange removes the info records between the provided // start key (inclusive) and end key (exclusive). -func (i InfoStorage) DeleteRange(ctx context.Context, startInfoKey, endInfoKey []byte) error { +func (i InfoStorage) DeleteRange(ctx context.Context, startInfoKey, endInfoKey string) error { return i.doWrite(ctx, func(ctx context.Context, j *Job, txn isql.Txn) error { _, err := txn.ExecEx( ctx, "write-job-info-delete", txn.KV(), @@ -259,7 +259,7 @@ func (i InfoStorage) DeleteRange(ctx context.Context, startInfoKey, endInfoKey [ // Iterate iterates though the info records for a given job and info key prefix. func (i InfoStorage) Iterate( - ctx context.Context, infoPrefix []byte, fn func(infoKey, value []byte) error, + ctx context.Context, infoPrefix string, fn func(infoKey string, value []byte) error, ) (retErr error) { return i.iterate(ctx, iterateAll, infoPrefix, fn) } @@ -267,7 +267,7 @@ func (i InfoStorage) Iterate( // GetLast calls fn on the last info record whose key matches the // given prefix. func (i InfoStorage) GetLast( - ctx context.Context, infoPrefix []byte, fn func(infoKey, value []byte) error, + ctx context.Context, infoPrefix string, fn func(infoKey string, value []byte) error, ) (retErr error) { return i.iterate(ctx, getLast, infoPrefix, fn) } @@ -286,32 +286,32 @@ const ( // GetLegacyPayloadKey returns the info_key whose value is the jobspb.Payload of // the job. -func GetLegacyPayloadKey() []byte { - return []byte(legacyPayloadKey) +func GetLegacyPayloadKey() string { + return legacyPayloadKey } // GetLegacyProgressKey returns the info_key whose value is the jobspb.Progress // of the job. -func GetLegacyProgressKey() []byte { - return []byte(legacyProgressKey) +func GetLegacyProgressKey() string { + return legacyProgressKey } // GetLegacyPayload returns the job's Payload from the system.jobs_info table. func (i InfoStorage) GetLegacyPayload(ctx context.Context) ([]byte, bool, error) { - return i.Get(ctx, []byte(legacyPayloadKey)) + return i.Get(ctx, legacyPayloadKey) } // WriteLegacyPayload writes the job's Payload to the system.jobs_info table. func (i InfoStorage) WriteLegacyPayload(ctx context.Context, payload []byte) error { - return i.Write(ctx, []byte(legacyPayloadKey), payload) + return i.Write(ctx, legacyPayloadKey, payload) } // GetLegacyProgress returns the job's Progress from the system.jobs_info table. func (i InfoStorage) GetLegacyProgress(ctx context.Context) ([]byte, bool, error) { - return i.Get(ctx, []byte(legacyProgressKey)) + return i.Get(ctx, legacyProgressKey) } // WriteLegacyProgress writes the job's Progress to the system.jobs_info table. func (i InfoStorage) WriteLegacyProgress(ctx context.Context, progress []byte) error { - return i.Write(ctx, []byte(legacyProgressKey), progress) + return i.Write(ctx, legacyProgressKey, progress) } diff --git a/pkg/jobs/job_info_storage_test.go b/pkg/jobs/job_info_storage_test.go index 475e4a19e682..f3fa6bd07eba 100644 --- a/pkg/jobs/job_info_storage_test.go +++ b/pkg/jobs/job_info_storage_test.go @@ -63,11 +63,11 @@ func TestJobInfoAccessors(t *testing.T) { job1 := createJob(1) job2 := createJob(2) job3 := createJob(3) - kPrefix, kA, kB, kC, kD := []byte("🔑"), []byte("🔑A"), []byte("🔑B"), []byte("🔑C"), []byte("🔑D") + kPrefix, kA, kB, kC, kD := "🔑", "🔑A", "🔑B", "🔑C", "🔑D" v1, v2, v3 := []byte("val1"), []byte("val2"), []byte("val3") // Key doesn't exist yet. - getJobInfo := func(j *jobs.Job, key []byte) (v []byte, ok bool, err error) { + getJobInfo := func(j *jobs.Job, key string) (v []byte, ok bool, err error) { err = idb.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := j.InfoStorage(txn) v, ok, err = infoStorage.Get(ctx, key) @@ -145,7 +145,7 @@ func TestJobInfoAccessors(t *testing.T) { var i int require.NoError(t, idb.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := job2.InfoStorage(txn) - return infoStorage.Iterate(ctx, kPrefix, func(key, value []byte) error { + return infoStorage.Iterate(ctx, kPrefix, func(key string, value []byte) error { i++ switch i { case 1: @@ -169,7 +169,7 @@ func TestJobInfoAccessors(t *testing.T) { i = 0 require.NoError(t, idb.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := job2.InfoStorage(txn) - return infoStorage.GetLast(ctx, kPrefix, func(key, value []byte) error { + return infoStorage.GetLast(ctx, kPrefix, func(key string, value []byte) error { i++ require.Equal(t, key, kC) require.Equal(t, v3, value) @@ -182,7 +182,7 @@ func TestJobInfoAccessors(t *testing.T) { found := false require.NoError(t, idb.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := job2.InfoStorage(txn) - return infoStorage.Iterate(ctx, kA, func(key, value []byte) error { + return infoStorage.Iterate(ctx, kA, func(key string, value []byte) error { require.Equal(t, kA, key) require.Equal(t, v2, value) found = true @@ -200,7 +200,7 @@ func TestJobInfoAccessors(t *testing.T) { i = 0 require.NoError(t, idb.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := job2.InfoStorage(txn) - return infoStorage.Iterate(ctx, kPrefix, func(key, value []byte) error { + return infoStorage.Iterate(ctx, kPrefix, func(key string, value []byte) error { i++ require.Equal(t, key, kC) return nil @@ -211,7 +211,7 @@ func TestJobInfoAccessors(t *testing.T) { // Iterate a different job. require.NoError(t, idb.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := job3.InfoStorage(txn) - return infoStorage.Iterate(ctx, kPrefix, func(key, value []byte) error { + return infoStorage.Iterate(ctx, kPrefix, func(key string, value []byte) error { t.Fatalf("unexpected record for job 3: %v = %v", key, value) return nil }) @@ -259,7 +259,7 @@ func TestAccessorsWithWrongSQLLivenessSession(t *testing.T) { require.NoError(t, err) require.NoError(t, ief.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := job.InfoStorage(txn) - return infoStorage.Write(ctx, []byte("foo"), []byte("baz")) + return infoStorage.Write(ctx, "foo", []byte("baz")) })) require.NoError(t, ief.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { @@ -272,14 +272,14 @@ func TestAccessorsWithWrongSQLLivenessSession(t *testing.T) { err = ief.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := job.InfoStorage(txn) - return infoStorage.Write(ctx, []byte("foo"), []byte("bar")) + return infoStorage.Write(ctx, "foo", []byte("bar")) }) require.True(t, testutils.IsError(err, "expected session.*but found")) // A Get should still succeed even with an invalid session id. require.NoError(t, ief.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := job.InfoStorage(txn) - val, exists, err := infoStorage.Get(ctx, []byte("foo")) + val, exists, err := infoStorage.Get(ctx, "foo") if err != nil { return err } @@ -291,7 +291,7 @@ func TestAccessorsWithWrongSQLLivenessSession(t *testing.T) { // Iterate should still succeed even with an invalid session id. require.NoError(t, ief.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := job.InfoStorage(txn) - return infoStorage.Iterate(ctx, []byte("foo"), func(infoKey, value []byte) error { + return infoStorage.Iterate(ctx, "foo", func(infoKey string, value []byte) error { require.Equal(t, value, []byte("baz")) return nil }) diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index 7f4067d35700..44201680c664 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -2188,13 +2188,13 @@ SELECT id, payload, progress FROM "".crdb_internal.system_jobs ORDER BY id DESC } if _, err := sqlDB.Exec(` -- Create a payload field from the most recent row. - INSERT INTO system.job_info(job_id, info_key, value) SELECT job_id+2, info_key, value FROM system.job_info WHERE job_id = $1 AND info_key = 'legacy_payload'::BYTES; + INSERT INTO system.job_info(job_id, info_key, value) SELECT job_id+2, info_key, value FROM system.job_info WHERE job_id = $1 AND info_key = 'legacy_payload'; `, jobID); err != nil { t.Fatal(err) } if _, err := sqlDB.Exec(` -- Create a corrupted progress field. - INSERT INTO system.job_info(job_id, info_key, value) SELECT job_id+2, info_key, '\xaaaa'::BYTES FROM system.job_info WHERE job_id = $1 AND info_key = 'legacy_progress'::BYTES; + INSERT INTO system.job_info(job_id, info_key, value) SELECT job_id+2, info_key, '\xaaaa'::BYTES FROM system.job_info WHERE job_id = $1 AND info_key = 'legacy_progress'; `, jobID); err != nil { t.Fatal(err) } @@ -2207,12 +2207,12 @@ SELECT id, payload, progress FROM "".crdb_internal.system_jobs ORDER BY id DESC } if _, err := sqlDB.Exec(` -- Create a payload field from the most recent row. - INSERT INTO system.job_info(job_id, info_key, value) SELECT job_id+4, info_key, value FROM system.job_info WHERE job_id = $1 AND info_key = 'legacy_payload'::BYTES; + INSERT INTO system.job_info(job_id, info_key, value) SELECT job_id+4, info_key, value FROM system.job_info WHERE job_id = $1 AND info_key = 'legacy_payload'; `, jobID); err != nil { t.Fatal(err) } if _, err := sqlDB.Exec(` - INSERT INTO system.job_info(job_id, info_key, value) SELECT job_id+4, info_key, NULL::BYTES FROM system.job_info WHERE job_id = $1 AND info_key = 'legacy_progress'::BYTES; + INSERT INTO system.job_info(job_id, info_key, value) SELECT job_id+4, info_key, NULL::BYTES FROM system.job_info WHERE job_id = $1 AND info_key = 'legacy_progress'; `, jobID); err != nil { t.Fatal(err) } diff --git a/pkg/jobs/jobsprofiler/profiler.go b/pkg/jobs/jobsprofiler/profiler.go index ac4f63af7a75..a0145961736f 100644 --- a/pkg/jobs/jobsprofiler/profiler.go +++ b/pkg/jobs/jobsprofiler/profiler.go @@ -44,7 +44,7 @@ func StorePlanDiagram( const infoKey = "dsp-diag-url-%d" infoStorage := jobs.InfoStorageForJob(txn, jobID) - return infoStorage.Write(ctx, []byte(fmt.Sprintf(infoKey, timeutil.Now().UnixNano())), + return infoStorage.Write(ctx, fmt.Sprintf(infoKey, timeutil.Now().UnixNano()), []byte(diagURL.String())) }) if err != nil { diff --git a/pkg/jobs/jobsprofiler/profiler_test.go b/pkg/jobs/jobsprofiler/profiler_test.go index 6fbccee7250f..ae7d81b151a8 100644 --- a/pkg/jobs/jobsprofiler/profiler_test.go +++ b/pkg/jobs/jobsprofiler/profiler_test.go @@ -83,7 +83,7 @@ func TestProfilerStorePlanDiagram(t *testing.T) { var count int err = execCfg.InternalDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { infoStorage := jobs.InfoStorageForJob(txn, jobID) - return infoStorage.Iterate(ctx, []byte("dsp-diag-url"), func(infoKey, value []byte) error { + return infoStorage.Iterate(ctx, "dsp-diag-url", func(infoKey string, value []byte) error { count++ return nil }) diff --git a/pkg/jobs/registry_test.go b/pkg/jobs/registry_test.go index e0f8c6310f83..7ac0f2e80f09 100644 --- a/pkg/jobs/registry_test.go +++ b/pkg/jobs/registry_test.go @@ -360,7 +360,7 @@ func TestCreateJobWritesToJobInfo(t *testing.T) { // Verify the payload in the system.job_info is the same as what we read // from system.jobs. - require.NoError(t, infoStorage.Iterate(ctx, []byte(legacyPayloadKey), func(infoKey, value []byte) error { + require.NoError(t, infoStorage.Iterate(ctx, legacyPayloadKey, func(infoKey string, value []byte) error { data, err := protoutil.Marshal(&expectedPayload) if err != nil { panic(err) @@ -371,7 +371,7 @@ func TestCreateJobWritesToJobInfo(t *testing.T) { // Verify the progress in the system.job_info is the same as what we read // from system.jobs. - require.NoError(t, infoStorage.Iterate(ctx, []byte(legacyProgressKey), func(infoKey, value []byte) error { + require.NoError(t, infoStorage.Iterate(ctx, legacyProgressKey, func(infoKey string, value []byte) error { data, err := protoutil.Marshal(&expectedProgress) if err != nil { panic(err) @@ -529,9 +529,9 @@ func TestBatchJobsCreation(t *testing.T) { [][]string{{fmt.Sprintf("%d", test.batchSize)}}) // Ensure that we are also writing the payload and progress to the job_info table. - tdb.CheckQueryResults(t, fmt.Sprintf(`SELECT count(*) FROM system.job_info WHERE info_key = 'legacy_payload'::BYTES AND job_id IN (%s)`, jobIdsClause), + tdb.CheckQueryResults(t, fmt.Sprintf(`SELECT count(*) FROM system.job_info WHERE info_key = 'legacy_payload' AND job_id IN (%s)`, jobIdsClause), [][]string{{fmt.Sprintf("%d", test.batchSize)}}) - tdb.CheckQueryResults(t, fmt.Sprintf(`SELECT count(*) FROM system.job_info WHERE info_key = 'legacy_progress'::BYTES AND job_id IN (%s)`, jobIdsClause), + tdb.CheckQueryResults(t, fmt.Sprintf(`SELECT count(*) FROM system.job_info WHERE info_key = 'legacy_progress' AND job_id IN (%s)`, jobIdsClause), [][]string{{fmt.Sprintf("%d", test.batchSize)}}) } }) diff --git a/pkg/jobs/update.go b/pkg/jobs/update.go index 768089e1a5a5..6c544c42d750 100644 --- a/pkg/jobs/update.go +++ b/pkg/jobs/update.go @@ -389,8 +389,8 @@ func getSelectStmtForJobUpdate( const ( selectWithoutSession = ` WITH - latestpayload AS (SELECT job_id, value FROM system.job_info AS payload WHERE info_key = 'legacy_payload'::BYTES), - latestprogress AS (SELECT job_id, value FROM system.job_info AS progress WHERE info_key = 'legacy_progress'::BYTES) + latestpayload AS (SELECT job_id, value FROM system.job_info AS payload WHERE info_key = 'legacy_payload'), + latestprogress AS (SELECT job_id, value FROM system.job_info AS progress WHERE info_key = 'legacy_progress') SELECT status, payload.value AS payload, progress.value AS progress` selectWithSession = selectWithoutSession + `, claim_session_id` diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 297bad31df8e..3859d9c22f09 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -406,6 +406,7 @@ go_test( "//pkg/settings", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigbounds", "//pkg/spanconfig/spanconfigptsreader", "//pkg/spanconfig/spanconfigstore", "//pkg/sql", diff --git a/pkg/kv/kvserver/batcheval/.gitignore b/pkg/kv/kvserver/batcheval/.gitignore index 7c3b30f2765e..90c026977570 100644 --- a/pkg/kv/kvserver/batcheval/.gitignore +++ b/pkg/kv/kvserver/batcheval/.gitignore @@ -1 +1,2 @@ +# Old benchmark data. refresh_range_bench_data_* diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 7d0d1155d4b6..07f65e44a8ad 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -166,6 +166,7 @@ go_test( "//pkg/testutils/sqlutils", "//pkg/testutils/storageutils", "//pkg/testutils/testcluster", + "//pkg/testutils/testfixtures", "//pkg/util", "//pkg/util/admission", "//pkg/util/encoding", @@ -179,7 +180,6 @@ go_test( "//pkg/util/uint128", "//pkg/util/uuid", "@com_github_cockroachdb_errors//:errors", - "@com_github_cockroachdb_errors//oserror", "@com_github_cockroachdb_pebble//vfs", "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go index 1712aa96e351..6b5945612522 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range_bench_test.go @@ -14,7 +14,6 @@ import ( "context" "fmt" "math/rand" - "os" "path/filepath" "testing" @@ -27,11 +26,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testfixtures" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" - "github.com/cockroachdb/errors/oserror" "github.com/cockroachdb/pebble/vfs" "github.com/stretchr/testify/require" ) @@ -202,13 +201,13 @@ func setupMVCCPebble(b testing.TB, dir string, lBaseMaxBytes int64, readOnly boo // and continuing to t=5ns*(numKeys+1). The goal of this is to // approximate an append-only type workload. // -// A read-only engine can be returned if opts.readOnlyEngine is -// set. The goal of this is to prevent read-triggered compactions that -// might change the distribution of data across levels. +// A read-only engine is returned if opts.readOnlyEngine is set. The goal of +// this is to prevent read-triggered compactions that might change the +// distribution of data across levels. // -// The creation of the database is time consuming, especially for -// larger numbers of versions. The database is persisted between runs -// and stored in the current directory. +// The creation of the database is time-consuming, especially for larger numbers +// of versions. The database is persisted between runs using +// testfixtures.ReuseOrGenerate. func setupData( ctx context.Context, b *testing.B, emk engineMaker, opts benchDataOptions, ) (storage.Engine, string) { @@ -223,88 +222,70 @@ func setupData( if opts.readOnlyEngine { readOnlyStr = "_readonly" } - loc := fmt.Sprintf("refresh_range_bench_data_%s_%s%s_%d_%d_%d", + name := fmt.Sprintf("refresh_range_bench_data_%s_%s%s_%d_%d_%d", verStr, orderStr, readOnlyStr, opts.numKeys, opts.valueBytes, opts.lBaseMaxBytes) - exists := true - if _, err := os.Stat(loc); oserror.IsNotExist(err) { - exists = false - } else if err != nil { - b.Fatal(err) - } - - absPath, err := filepath.Abs(loc) - if err != nil { - absPath = loc - } - - if exists { - log.Infof(ctx, "using existing refresh range benchmark data: %s", absPath) - testutils.ReadAllFiles(filepath.Join(loc, "*")) - return emk(b, loc, opts.lBaseMaxBytes, opts.readOnlyEngine), loc - } - - eng := emk(b, loc, opts.lBaseMaxBytes, false) - log.Infof(ctx, "creating refresh range benchmark data: %s", absPath) - - // Generate the same data every time. - rng := rand.New(rand.NewSource(1449168817)) + dir := testfixtures.ReuseOrGenerate(b, name, func(dir string) { + eng := emk(b, dir, opts.lBaseMaxBytes, false) + log.Infof(ctx, "creating refresh range benchmark data: %s", dir) - keys := make([]roachpb.Key, opts.numKeys) - order := make([]int, 0, opts.numKeys) - for i := 0; i < opts.numKeys; i++ { - keys[i] = encoding.EncodeUvarintAscending([]byte("key-"), uint64(i)) - order = append(order, i) - } + // Generate the same data every time. + rng := rand.New(rand.NewSource(1449168817)) - if opts.randomKeyOrder { - rng.Shuffle(len(order), func(i, j int) { - order[i], order[j] = order[j], order[i] - }) - } + keys := make([]roachpb.Key, opts.numKeys) + order := make([]int, 0, opts.numKeys) + for i := 0; i < opts.numKeys; i++ { + keys[i] = encoding.EncodeUvarintAscending([]byte("key-"), uint64(i)) + order = append(order, i) + } - writeKey := func(batch storage.Batch, idx int, pos int) { - key := keys[idx] - value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, opts.valueBytes)) - value.InitChecksum(key) - ts := hlc.Timestamp{WallTime: int64((pos + 1) * 5)} - if err := storage.MVCCPut(ctx, batch, nil /* ms */, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { - b.Fatal(err) + if opts.randomKeyOrder { + rng.Shuffle(len(order), func(i, j int) { + order[i], order[j] = order[j], order[i] + }) } - } - batch := eng.NewBatch() - for i, idx := range order { - // Output the keys in ~20 batches. If we used a single batch to output all - // of the keys rocksdb would create a single sstable. We want multiple - // sstables in order to exercise filtering of which sstables are examined - // during iterator seeking. We fix the number of batches we output so that - // optimizations which change the data size result in the same number of - // sstables. - if scaled := len(order) / 20; i > 0 && (i%scaled) == 0 { - log.Infof(ctx, "committing (%d/~%d) (%d/%d)", i/scaled, 20, i, len(order)) - if err := batch.Commit(false /* sync */); err != nil { - b.Fatal(err) - } - batch.Close() - batch = eng.NewBatch() - if err := eng.Flush(); err != nil { + writeKey := func(batch storage.Batch, idx int, pos int) { + key := keys[idx] + value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, opts.valueBytes)) + value.InitChecksum(key) + ts := hlc.Timestamp{WallTime: int64((pos + 1) * 5)} + if err := storage.MVCCPut(ctx, batch, nil /* ms */, key, ts, hlc.ClockTimestamp{}, value, nil); err != nil { b.Fatal(err) } } - writeKey(batch, idx, i) - } - if err := batch.Commit(false /* sync */); err != nil { - b.Fatal(err) - } - batch.Close() - if err := eng.Flush(); err != nil { - b.Fatal(err) - } - if opts.readOnlyEngine { + batch := eng.NewBatch() + for i, idx := range order { + // Output the keys in ~20 batches. If we used a single batch to output all + // of the keys rocksdb would create a single sstable. We want multiple + // sstables in order to exercise filtering of which sstables are examined + // during iterator seeking. We fix the number of batches we output so that + // optimizations which change the data size result in the same number of + // sstables. + if scaled := len(order) / 20; i > 0 && (i%scaled) == 0 { + log.Infof(ctx, "committing (%d/~%d) (%d/%d)", i/scaled, 20, i, len(order)) + if err := batch.Commit(false /* sync */); err != nil { + b.Fatal(err) + } + batch.Close() + batch = eng.NewBatch() + if err := eng.Flush(); err != nil { + b.Fatal(err) + } + } + writeKey(batch, idx, i) + } + if err := batch.Commit(false /* sync */); err != nil { + b.Fatal(err) + } + batch.Close() + if err := eng.Flush(); err != nil { + b.Fatal(err) + } eng.Close() - eng = emk(b, loc, opts.lBaseMaxBytes, opts.readOnlyEngine) - } - return eng, loc + }) + + testutils.ReadAllFiles(filepath.Join(dir, "*")) + return emk(b, dir, opts.lBaseMaxBytes, opts.readOnlyEngine), dir } diff --git a/pkg/kv/kvserver/client_spanconfigs_test.go b/pkg/kv/kvserver/client_spanconfigs_test.go index 4c0e9d5e73c3..58ade2858008 100644 --- a/pkg/kv/kvserver/client_spanconfigs_test.go +++ b/pkg/kv/kvserver/client_spanconfigs_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigstore" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -41,6 +42,7 @@ func TestSpanConfigUpdateAppliedToReplica(t *testing.T) { spanConfigStore := spanconfigstore.New( roachpb.TestingDefaultSpanConfig(), cluster.MakeTestingClusterSettings(), + spanconfigbounds.NewEmptyReader(), nil, ) var t0 = time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC) @@ -107,7 +109,9 @@ func TestFallbackSpanConfigOverride(t *testing.T) { defer leaktest.AfterTest(t)() st := cluster.MakeTestingClusterSettings() - spanConfigStore := spanconfigstore.New(roachpb.TestingDefaultSpanConfig(), st, nil) + spanConfigStore := spanconfigstore.New( + roachpb.TestingDefaultSpanConfig(), st, spanconfigbounds.NewEmptyReader(), nil, + ) var t0 = time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC) mockSubscriber := newMockSpanConfigSubscriber(t0, spanConfigStore) @@ -171,13 +175,15 @@ func newMockSpanConfigSubscriber( } } -func (m *mockSpanConfigSubscriber) NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool { +func (m *mockSpanConfigSubscriber) NeedsSplit( + ctx context.Context, start, end roachpb.RKey, +) (bool, error) { return m.Store.NeedsSplit(ctx, start, end) } func (m *mockSpanConfigSubscriber) ComputeSplitKey( ctx context.Context, start, end roachpb.RKey, -) roachpb.RKey { +) (roachpb.RKey, error) { return m.Store.ComputeSplitKey(ctx, start, end) } diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index c5c3e23207e6..b986b5fa7cdd 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -147,7 +147,17 @@ func (mq *mergeQueue) shouldQueue( return false, 0 } - if confReader.NeedsSplit(ctx, desc.StartKey, desc.EndKey.Next()) { + needsSplit, err := confReader.NeedsSplit(ctx, desc.StartKey, desc.EndKey.Next()) + if err != nil { + log.Warningf( + ctx, + "could not compute if extending range would result in a split (err=%v); skipping merge for range %s", + err, + desc.RangeID, + ) + return false, 0 + } + if needsSplit { // This range would need to be split if it extended just one key further. // There is thus no possible right-hand neighbor that it could be merged // with. diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index fbf6f2429ac3..79aa86a46011 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -666,13 +666,20 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. repl.maybeInitializeRaftGroup(ctx) } - if !bq.acceptsUnsplitRanges && confReader.NeedsSplit(ctx, repl.Desc().StartKey, repl.Desc().EndKey) { - // Range needs to be split due to span configs, but queue does not - // accept unsplit ranges. - if log.V(1) { - log.Infof(ctx, "split needed; not adding") + if !bq.acceptsUnsplitRanges { + // Queue does not accept unsplit ranges. Check to see if the range needs to + // be split because of spanconfigs. + needsSplit, err := confReader.NeedsSplit(ctx, repl.Desc().StartKey, repl.Desc().EndKey) + if err != nil { + log.Warningf(ctx, "unable to compute whether split is needed; not adding") + return + } + if needsSplit { + if log.V(1) { + log.Infof(ctx, "split needed; not adding") + } + return } - return } if bq.needsLease { @@ -945,11 +952,18 @@ func (bq *baseQueue) processReplica(ctx context.Context, repl replicaInQueue) er } } - if !bq.acceptsUnsplitRanges && confReader.NeedsSplit(ctx, repl.Desc().StartKey, repl.Desc().EndKey) { - // Range needs to be split due to zone configs, but queue does - // not accept unsplit ranges. - log.VEventf(ctx, 3, "split needed; skipping") - return nil + if !bq.acceptsUnsplitRanges { + // Queue does not accept unsplit ranges. Check to see if the range needs to + // be spilt because of a span config. + needsSplit, err := confReader.NeedsSplit(ctx, repl.Desc().StartKey, repl.Desc().EndKey) + if err != nil { + log.Warningf(ctx, "unable to compute NeedsSplit, skipping: %v", err) + return nil + } + if needsSplit { + log.VEventf(ctx, 3, "split needed; skipping") + return nil + } } ctx, span := tracing.EnsureChildSpan(ctx, bq.Tracer, bq.processOpName()) diff --git a/pkg/kv/kvserver/queue_test.go b/pkg/kv/kvserver/queue_test.go index 9375e2fdd17e..fe1208ca0305 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -712,11 +712,15 @@ func TestAcceptsUnsplitRanges(t *testing.T) { return nil }) neverSplitsDesc := neverSplits.Desc() - if cfg.NeedsSplit(ctx, neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) { + needsSplit, err := cfg.NeedsSplit(ctx, neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) + require.NoError(t, err) + if needsSplit { t.Fatal("System config says range needs to be split") } willSplitDesc := willSplit.Desc() - if cfg.NeedsSplit(ctx, willSplitDesc.StartKey, willSplitDesc.EndKey) { + needsSplit, err = cfg.NeedsSplit(ctx, willSplitDesc.StartKey, willSplitDesc.EndKey) + require.NoError(t, err) + if needsSplit { t.Fatal("System config says range needs to be split") } @@ -748,11 +752,15 @@ func TestAcceptsUnsplitRanges(t *testing.T) { // Check our config. neverSplitsDesc = neverSplits.Desc() - if cfg.NeedsSplit(ctx, neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) { + needsSplit, err = cfg.NeedsSplit(ctx, neverSplitsDesc.StartKey, neverSplitsDesc.EndKey) + require.NoError(t, err) + if needsSplit { t.Fatal("System config says range needs to be split") } willSplitDesc = willSplit.Desc() - if !cfg.NeedsSplit(ctx, willSplitDesc.StartKey, willSplitDesc.EndKey) { + needsSplit, err = cfg.NeedsSplit(ctx, willSplitDesc.StartKey, willSplitDesc.EndKey) + require.NoError(t, err) + if !needsSplit { t.Fatal("System config says range does not need to be split") } diff --git a/pkg/kv/kvserver/split_queue.go b/pkg/kv/kvserver/split_queue.go index 3165c6d2fb48..5b718e9c043e 100644 --- a/pkg/kv/kvserver/split_queue.go +++ b/pkg/kv/kvserver/split_queue.go @@ -139,7 +139,12 @@ func shouldSplitRange( shouldBackpressureWrites bool, confReader spanconfig.StoreReader, ) (shouldQ bool, priority float64) { - if confReader.NeedsSplit(ctx, desc.StartKey, desc.EndKey) { + needsSplit, err := confReader.NeedsSplit(ctx, desc.StartKey, desc.EndKey) + if err != nil { + log.Warningf(ctx, "unable to compute NeedsSpilt (%v); skipping range %s", err, desc.RangeID) + return false, 0 + } + if needsSplit { // Set priority to 1 in the event the range is split by zone configs. priority = 1 shouldQ = true @@ -236,7 +241,11 @@ func (sq *splitQueue) processAttempt( ) (processed bool, err error) { desc := r.Desc() // First handle the case of splitting due to span config maps. - if splitKey := confReader.ComputeSplitKey(ctx, desc.StartKey, desc.EndKey); splitKey != nil { + splitKey, err := confReader.ComputeSplitKey(ctx, desc.StartKey, desc.EndKey) + if err != nil { + return false, errors.Wrapf(err, "unable to compute split key") + } + if splitKey != nil { if _, err := r.adminSplitWithDescriptor( ctx, kvpb.AdminSplitRequest{ diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 790417acdadc..cc68945e73bf 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -3411,13 +3411,15 @@ type mockSpanConfigReader struct { overrides map[string]roachpb.SpanConfig } -func (m *mockSpanConfigReader) NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool { +func (m *mockSpanConfigReader) NeedsSplit( + ctx context.Context, start, end roachpb.RKey, +) (bool, error) { panic("unimplemented") } func (m *mockSpanConfigReader) ComputeSplitKey( ctx context.Context, start, end roachpb.RKey, -) roachpb.RKey { +) (roachpb.RKey, error) { panic("unimplemented") } diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb/capabilities.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb/capabilities.go index d82dd5682052..4eda2d707a15 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb/capabilities.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb/capabilities.go @@ -29,9 +29,15 @@ func (b boolCapValue) SafeFormat(p redact.SafePrinter, verb rune) { p.Print(redact.Safe(bool(b))) } -// Unwrap implements the tenantcapabilities.Value interface. func (b boolCapValue) Unwrap() interface{} { return bool(b) } +// Unwrap implements the tenantcapabilities.Value interface. +func (m *SpanConfigBounds) Unwrap() interface{} { return m } + +func (m *SpanConfigBounds) SafeFormat(p redact.SafePrinter, verb rune) { + p.Print(redact.SafeString(m.String())) +} + // boolCap is an accessor struct for boolean capabilities. type boolCap struct { cap *bool @@ -73,6 +79,26 @@ func (i invertedBoolCap) Set(val interface{}) { *i.cap = !bval } +// spanConfigBoundsCap is an accessor struct for SpanConfigBounds that are +// stored on the underlying TenantCapabilities proto. +type spanConfigBoundsCap struct { + cap *SpanConfigBounds +} + +// Get implements the tenantcapabilities.Capability interface. +func (s spanConfigBoundsCap) Get() tenantcapabilities.Value { + return s.cap +} + +// Set implements the tenantcapabilities.Capability interface. +func (s spanConfigBoundsCap) Set(val interface{}) { + scfgBoundsVal, ok := val.(SpanConfigBounds) + if !ok { + panic(errors.AssertionFailedf("invalid value type: %T", val)) + } + *s.cap = scfgBoundsVal +} + // Cap implements the tenantcapabilities.TenantCapabilities interface. func (t *TenantCapabilities) Cap( capabilityID tenantcapabilities.CapabilityID, @@ -92,6 +118,8 @@ func (t *TenantCapabilities) Cap( return boolCap{&t.CanViewTSDBMetrics} case tenantcapabilities.ExemptFromRateLimiting: return boolCap{&t.ExemptFromRateLimiting} + case tenantcapabilities.TenantSpanConfigBounds: + return spanConfigBoundsCap{t.SpanConfigBounds} default: panic(errors.AssertionFailedf("unknown capability: %q", capabilityID.String())) diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index a425cbc8aa79..318a14533226 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -178,6 +178,7 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigbounds", "//pkg/spanconfig/spanconfigjob", "//pkg/spanconfig/spanconfigkvaccessor", "//pkg/spanconfig/spanconfigkvsubscriber", diff --git a/pkg/server/server.go b/pkg/server/server.go index d3ad9fcc9e98..b2e95b8bf3de 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -71,6 +71,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/tenantsettingswatcher" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" _ "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigjob" // register jobs declared outside of pkg/sql "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvsubscriber" @@ -623,6 +624,15 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { keys.SystemSQLCodec, clock, rangeFeedFactory, &cfg.DefaultZoneConfig, ) + tenantCapabilitiesWatcher := tenantcapabilitieswatcher.New( + clock, + rangeFeedFactory, + keys.TenantsTableID, + stopper, + 1<<20, /* 1 MB */ + tenantCapabilitiesTestingKnobs, + ) + var spanConfig struct { // kvAccessor powers the span configuration RPCs and the host tenant's // reconciliation job. @@ -668,6 +678,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { 1<<20, /* 1 MB */ fallbackConf, cfg.Settings, + spanconfigbounds.NewReader(tenantCapabilitiesWatcher), spanConfigKnobs, registry, ) @@ -791,15 +802,6 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { clock, rangeFeedFactory, stopper, st, ) - tenantCapabilitiesWatcher := tenantcapabilitieswatcher.New( - clock, - rangeFeedFactory, - keys.TenantsTableID, - stopper, - 1<<20, /* 1 MB */ - tenantCapabilitiesTestingKnobs, - ) - node := NewNode( storeCfg, recorder, diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index 7c987c863b09..7d9299b712e1 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -274,8 +274,8 @@ type StoreWriter interface { // StoreReader is the read-only portion of the Store interface. It doubles as an // adaptor interface for config.SystemConfig. type StoreReader interface { - NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool - ComputeSplitKey(ctx context.Context, start, end roachpb.RKey) roachpb.RKey + NeedsSplit(ctx context.Context, start, end roachpb.RKey) (bool, error) + ComputeSplitKey(ctx context.Context, start, end roachpb.RKey) (roachpb.RKey, error) GetSpanConfigForKey(ctx context.Context, key roachpb.RKey) (roachpb.SpanConfig, error) } diff --git a/pkg/spanconfig/spanconfigbounds/BUILD.bazel b/pkg/spanconfig/spanconfigbounds/BUILD.bazel index a7891683fcb6..a46df79654ab 100644 --- a/pkg/spanconfig/spanconfigbounds/BUILD.bazel +++ b/pkg/spanconfig/spanconfigbounds/BUILD.bazel @@ -6,6 +6,7 @@ go_library( srcs = [ "bool_field.go", "bounds.go", + "bounds_reader.go", "constraints_field.go", "doc.go", "fields.go", @@ -21,6 +22,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/config", + "//pkg/multitenant/tenantcapabilities", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", "//pkg/roachpb", "//pkg/util/protoutil", diff --git a/pkg/spanconfig/spanconfigbounds/bounds_reader.go b/pkg/spanconfig/spanconfigbounds/bounds_reader.go new file mode 100644 index 000000000000..2d898ab8598e --- /dev/null +++ b/pkg/spanconfig/spanconfigbounds/bounds_reader.go @@ -0,0 +1,66 @@ +// 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 spanconfigbounds + +import ( + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +// Reader maintains an in-memory view of the global SpanConfigBounds state. +// +// SpanConfigBounds are stored as tenant capabilities, the state of which is +// surfaced by the tenantcapabilities.Reader. BoundsReader serves as a narrow, +// adapter interface for the same. +type Reader interface { + // Bounds returns span config bounds set for a given tenant. If no bounds have + // been configured for the given tenant, found returns false. + Bounds(tenID roachpb.TenantID) (_ Bounds, found bool) +} + +type boundsReader struct { + capabilitiesReader tenantcapabilities.Reader +} + +// NewReader constructs and returns a new Reader. +func NewReader(reader tenantcapabilities.Reader) Reader { + return &boundsReader{ + capabilitiesReader: reader, + } +} + +// Bounds implements the BoundsReader interface. +func (r *boundsReader) Bounds(tenID roachpb.TenantID) (_ Bounds, found bool) { + capabilities, found := r.capabilitiesReader.GetCapabilities(tenID) + if !found { + return Bounds{}, false + } + + boundspb := capabilities.Cap(tenantcapabilities.TenantSpanConfigBounds).Get().Unwrap().(*tenantcapabilitiespb.SpanConfigBounds) + if boundspb == nil { + return Bounds{}, false + } + return MakeBounds(boundspb), true +} + +// NewEmptyReader returns a new Reader which corresponds to an empty span config +// bounds state. It's only intended for testing. +func NewEmptyReader() Reader { + return emptyReader(true) +} + +type emptyReader bool + +// Bounds implements the Reader interface. +func (emptyReader) Bounds(roachpb.TenantID) (Bounds, bool) { + return Bounds{}, false +} diff --git a/pkg/spanconfig/spanconfigbounds/int32field.go b/pkg/spanconfig/spanconfigbounds/int32field.go index d0310539fdd0..e35f30c0a61c 100644 --- a/pkg/spanconfig/spanconfigbounds/int32field.go +++ b/pkg/spanconfig/spanconfigbounds/int32field.go @@ -30,9 +30,6 @@ func (f int32Field) SafeFormat(s redact.SafePrinter, verb rune) { func (f int32Field) FieldBound(b Bounds) ValueBounds { getBound := func() *tenantcapabilitiespb.SpanConfigBounds_Int32Range { - if b.b.ConstraintBounds == nil { - return nil - } switch f { case numReplicas: return b.b.NumReplicas diff --git a/pkg/spanconfig/spanconfigbounds/span_config_bounds.go b/pkg/spanconfig/spanconfigbounds/span_config_bounds.go index a9cd006532cc..4415a91493eb 100644 --- a/pkg/spanconfig/spanconfigbounds/span_config_bounds.go +++ b/pkg/spanconfig/spanconfigbounds/span_config_bounds.go @@ -77,7 +77,7 @@ func (b Bounds) Check(c *roachpb.SpanConfig) Violations { func (b Bounds) clamp(c *roachpb.SpanConfig, reporter func(Field)) (changed bool) { for _, f := range fields { - if b := f.FieldBound(b); !b.clamp(c, f) { + if bb := f.FieldBound(b); !bb.clamp(c, f) { continue } changed = true diff --git a/pkg/spanconfig/spanconfigbounds/testdata/basic b/pkg/spanconfig/spanconfigbounds/testdata/basic index c3a52ad58d89..f89ab542d16b 100644 --- a/pkg/spanconfig/spanconfigbounds/testdata/basic +++ b/pkg/spanconfig/spanconfigbounds/testdata/basic @@ -1,12 +1,16 @@ bounds name=foo gc_ttl_seconds: range_max_bytes: +num_voters: +num_replicas: ---- config name=bar gc_policy: range_min_bytes: 5 range_max_bytes: 5 +num_voters: 7 +num_replicas: 7 ---- conforms bounds=foo config=bar @@ -15,23 +19,29 @@ false check bounds=foo config=bar ---- -span config bounds violated for fields: range_max_bytes -span config bounds violated for fields: range_max_bytes -(1) span config bounds violated for fields: range_max_bytes +span config bounds violated for fields: range_max_bytes, num_voters, gc.ttlseconds +span config bounds violated for fields: range_max_bytes, num_voters, gc.ttlseconds +(1) span config bounds violated for fields: range_max_bytes, num_voters, gc.ttlseconds | range_max_bytes: 5 does not conform to [10, 20], will be clamped to 10 + | num_voters: 7 does not conform to [3, 5], will be clamped to 5 + | gc.ttlseconds: 122 does not conform to [123, 7000], will be clamped to 123 Error types: (1) *spanconfigbounds.ViolationError clamp bounds=foo config=bar ---- ---- -@@ -1,6 +1,6 @@ +@@ -1,8 +1,8 @@ range_min_bytes: 5 -range_max_bytes: 5 +range_max_bytes: 10 gc_policy: < - ttl_seconds: 122 +- ttl_seconds: 122 ++ ttl_seconds: 123 > + num_replicas: 7 +-num_voters: 7 ++num_voters: 5 ---- ---- diff --git a/pkg/spanconfig/spanconfigkvsubscriber/BUILD.bazel b/pkg/spanconfig/spanconfigkvsubscriber/BUILD.bazel index 4f7f7c4c6fab..39f92766835d 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/BUILD.bazel +++ b/pkg/spanconfig/spanconfigkvsubscriber/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigbounds", "//pkg/spanconfig/spanconfigstore", "//pkg/sql/catalog", "//pkg/sql/catalog/systemschema", @@ -63,6 +64,7 @@ go_test( "//pkg/server", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigbounds", "//pkg/spanconfig/spanconfigkvaccessor", "//pkg/spanconfig/spanconfigtestutils", "//pkg/sql/isql", diff --git a/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go b/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go index d3383bf71a4b..2c67fd13d027 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvaccessor" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvsubscriber" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" @@ -100,7 +101,6 @@ import ( // Text of the form [a,b) and [a,b):C correspond to spans and span config // records; see spanconfigtestutils.Parse{Span,Config,SpanConfigRecord} for more // details. -// TODO(arul): Add ability to express tenant spans to this datadriven test. func TestDataDriven(t *testing.T) { defer leaktest.AfterTest(t)() @@ -148,6 +148,7 @@ func TestDataDriven(t *testing.T) { 10<<20, /* 10 MB */ spanconfigtestutils.ParseConfig(t, "FALLBACK"), tc.Server(0).ClusterSettings(), + spanconfigbounds.NewEmptyReader(), &spanconfig.TestingKnobs{ KVSubscriberRangeFeedKnobs: &rangefeedcache.TestingKnobs{ OnTimestampAdvance: func(ts hlc.Timestamp) { @@ -284,7 +285,8 @@ func TestDataDriven(t *testing.T) { d.ScanArgs(t, cmdArg.Key, &spanStr) span := spanconfigtestutils.ParseSpan(t, spanStr) start, end := roachpb.RKey(span.Key), roachpb.RKey(span.EndKey) - splitKey := kvSubscriber.ComputeSplitKey(ctx, start, end) + splitKey, err := kvSubscriber.ComputeSplitKey(ctx, start, end) + require.NoError(t, err) return string(splitKey) case "needs-split": @@ -292,7 +294,8 @@ func TestDataDriven(t *testing.T) { d.ScanArgs(t, cmdArg.Key, &spanStr) span := spanconfigtestutils.ParseSpan(t, spanStr) start, end := roachpb.RKey(span.Key), roachpb.RKey(span.EndKey) - result := kvSubscriber.NeedsSplit(ctx, start, end) + result, err := kvSubscriber.NeedsSplit(ctx, start, end) + require.NoError(t, err) return fmt.Sprintf("%t", result) default: diff --git a/pkg/spanconfig/spanconfigkvsubscriber/dummy.go b/pkg/spanconfig/spanconfigkvsubscriber/dummy.go index dbeebf81156e..7649861a55b3 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/dummy.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/dummy.go @@ -43,15 +43,15 @@ func (n *noopKVSubscriber) LastUpdated() hlc.Timestamp { } // NeedsSplit is part of the spanconfig.KVSubscriber interface. -func (n *noopKVSubscriber) NeedsSplit(context.Context, roachpb.RKey, roachpb.RKey) bool { - return false +func (n *noopKVSubscriber) NeedsSplit(context.Context, roachpb.RKey, roachpb.RKey) (bool, error) { + return false, nil } // ComputeSplitKey is part of the spanconfig.KVSubscriber interface. func (n *noopKVSubscriber) ComputeSplitKey( context.Context, roachpb.RKey, roachpb.RKey, -) roachpb.RKey { - return roachpb.RKey{} +) (roachpb.RKey, error) { + return roachpb.RKey{}, nil } // GetSpanConfigForKey is part of the spanconfig.KVSubscriber interface. diff --git a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go index 285f07b37622..d6ef8a102620 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigstore" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -139,6 +140,9 @@ type KVSubscriber struct { clock *hlc.Clock metrics *Metrics + + // boundsReader provides a handle to the global SpanConfigBounds state. + boundsReader spanconfigbounds.Reader } var _ spanconfig.KVSubscriber = &KVSubscriber{} @@ -186,6 +190,7 @@ func New( bufferMemLimit int64, fallback roachpb.SpanConfig, settings *cluster.Settings, + boundsReader spanconfigbounds.Reader, knobs *spanconfig.TestingKnobs, registry *metric.Registry, ) *KVSubscriber { @@ -200,12 +205,13 @@ func New( Key: spanConfigTableStart, EndKey: spanConfigTableStart.PrefixEnd(), } - spanConfigStore := spanconfigstore.New(fallback, settings, knobs) + spanConfigStore := spanconfigstore.New(fallback, settings, boundsReader, knobs) s := &KVSubscriber{ - fallback: fallback, - knobs: knobs, - settings: settings, - clock: clock, + fallback: fallback, + knobs: knobs, + settings: settings, + clock: clock, + boundsReader: boundsReader, } var rfCacheKnobs *rangefeedcache.TestingKnobs if knobs != nil { @@ -331,7 +337,7 @@ func (s *KVSubscriber) LastUpdated() hlc.Timestamp { } // NeedsSplit is part of the spanconfig.KVSubscriber interface. -func (s *KVSubscriber) NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool { +func (s *KVSubscriber) NeedsSplit(ctx context.Context, start, end roachpb.RKey) (bool, error) { s.mu.RLock() defer s.mu.RUnlock() @@ -339,7 +345,9 @@ func (s *KVSubscriber) NeedsSplit(ctx context.Context, start, end roachpb.RKey) } // ComputeSplitKey is part of the spanconfig.KVSubscriber interface. -func (s *KVSubscriber) ComputeSplitKey(ctx context.Context, start, end roachpb.RKey) roachpb.RKey { +func (s *KVSubscriber) ComputeSplitKey( + ctx context.Context, start, end roachpb.RKey, +) (roachpb.RKey, error) { s.mu.RLock() defer s.mu.RUnlock() @@ -395,7 +403,7 @@ func (s *KVSubscriber) handleUpdate(ctx context.Context, u rangefeedcache.Update func (s *KVSubscriber) handleCompleteUpdate( ctx context.Context, ts hlc.Timestamp, events []rangefeedbuffer.Event, ) { - freshStore := spanconfigstore.New(s.fallback, s.settings, s.knobs) + freshStore := spanconfigstore.New(s.fallback, s.settings, s.boundsReader, s.knobs) for _, ev := range events { freshStore.Apply(ctx, false /* dryrun */, ev.(*bufferEvent).Update) } diff --git a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber_test.go b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber_test.go index 8bc1f7842b5f..89e61a69b431 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber_test.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -91,6 +92,7 @@ func TestGetProtectionTimestamps(t *testing.T) { 1<<20, /* 1 MB */ roachpb.SpanConfig{}, cluster.MakeTestingClusterSettings(), + spanconfigbounds.NewEmptyReader(), nil, nil, ) @@ -171,12 +173,14 @@ func (m *manualStore) Apply( } // NeedsSplit implements the spanconfig.Store interface. -func (m *manualStore) NeedsSplit(context.Context, roachpb.RKey, roachpb.RKey) bool { +func (m *manualStore) NeedsSplit(context.Context, roachpb.RKey, roachpb.RKey) (bool, error) { panic("unimplemented") } // ComputeSplitKey implements the spanconfig.Store interface. -func (m *manualStore) ComputeSplitKey(context.Context, roachpb.RKey, roachpb.RKey) roachpb.RKey { +func (m *manualStore) ComputeSplitKey( + context.Context, roachpb.RKey, roachpb.RKey, +) (roachpb.RKey, error) { panic("unimplemented") } diff --git a/pkg/spanconfig/spanconfigptsreader/adapter_test.go b/pkg/spanconfig/spanconfigptsreader/adapter_test.go index ff008f0fb622..faa9f77b50dc 100644 --- a/pkg/spanconfig/spanconfigptsreader/adapter_test.go +++ b/pkg/spanconfig/spanconfigptsreader/adapter_test.go @@ -115,13 +115,13 @@ func (m *manualSubscriber) Start(context.Context, *stop.Stopper) error { panic("unimplemented") } -func (m *manualSubscriber) NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool { +func (m *manualSubscriber) NeedsSplit(ctx context.Context, start, end roachpb.RKey) (bool, error) { panic("unimplemented") } func (m *manualSubscriber) ComputeSplitKey( context.Context, roachpb.RKey, roachpb.RKey, -) roachpb.RKey { +) (roachpb.RKey, error) { panic("unimplemented") } diff --git a/pkg/spanconfig/spanconfigreconciler/BUILD.bazel b/pkg/spanconfig/spanconfigreconciler/BUILD.bazel index cfccd98cf5d4..f1590af59357 100644 --- a/pkg/spanconfig/spanconfigreconciler/BUILD.bazel +++ b/pkg/spanconfig/spanconfigreconciler/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigbounds", "//pkg/spanconfig/spanconfigsqltranslator", "//pkg/spanconfig/spanconfigstore", "//pkg/sql", diff --git a/pkg/spanconfig/spanconfigreconciler/reconciler.go b/pkg/spanconfig/spanconfigreconciler/reconciler.go index 052576843e19..fd2975ca8371 100644 --- a/pkg/spanconfig/spanconfigreconciler/reconciler.go +++ b/pkg/spanconfig/spanconfigreconciler/reconciler.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigsqltranslator" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigstore" "github.com/cockroachdb/cockroach/pkg/sql" @@ -384,7 +385,9 @@ func (f *fullReconciler) fetchExistingSpanConfigs( targets = append(targets, spanconfig.MakeTargetFromSystemTarget(spanconfig.MakeAllTenantKeyspaceTargetsSet(f.tenID))) } - store := spanconfigstore.New(roachpb.SpanConfig{}, f.settings, f.knobs) + // The reconciler doesn't do any bounds checks or clamping, so it shouldn't + // need access to tenant capabilities (and by extension span config bounds). + store := spanconfigstore.New(roachpb.SpanConfig{}, f.settings, spanconfigbounds.NewEmptyReader(), f.knobs) { // Fully populate the store with KVAccessor contents. records, err := f.kvAccessor.GetSpanConfigRecords(ctx, targets) diff --git a/pkg/spanconfig/spanconfigreporter/BUILD.bazel b/pkg/spanconfig/spanconfigreporter/BUILD.bazel index d463555d1fb3..1f75c9d87cc3 100644 --- a/pkg/spanconfig/spanconfigreporter/BUILD.bazel +++ b/pkg/spanconfig/spanconfigreporter/BUILD.bazel @@ -41,6 +41,7 @@ go_test( "//pkg/server", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigbounds", "//pkg/spanconfig/spanconfigstore", "//pkg/spanconfig/spanconfigtestutils", "//pkg/testutils/datapathutils", diff --git a/pkg/spanconfig/spanconfigreporter/datadriven_test.go b/pkg/spanconfig/spanconfigreporter/datadriven_test.go index feddfd9092c5..79074ee00a14 100644 --- a/pkg/spanconfig/spanconfigreporter/datadriven_test.go +++ b/pkg/spanconfig/spanconfigreporter/datadriven_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" clustersettings "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigreporter" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigstore" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" @@ -233,7 +234,9 @@ func newMockCluster( nodes: make(map[roachpb.NodeID]roachpb.NodeDescriptor), ranges: make(map[roachpb.RangeID]roachpb.RangeDescriptor), liveness: make(map[roachpb.NodeID]bool), - store: spanconfigstore.New(roachpb.TestingDefaultSpanConfig(), st, scKnobs), + store: spanconfigstore.New( + roachpb.TestingDefaultSpanConfig(), st, spanconfigbounds.NewEmptyReader(), scKnobs, + ), } } @@ -274,12 +277,14 @@ func (s *mockCluster) Scan( } // NeedsSplit implements spanconfig.StoreReader. -func (s *mockCluster) NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool { +func (s *mockCluster) NeedsSplit(ctx context.Context, start, end roachpb.RKey) (bool, error) { return s.store.NeedsSplit(ctx, start, end) } // ComputeSplitKey implements spanconfig.StoreReader. -func (s *mockCluster) ComputeSplitKey(ctx context.Context, start, end roachpb.RKey) roachpb.RKey { +func (s *mockCluster) ComputeSplitKey( + ctx context.Context, start, end roachpb.RKey, +) (roachpb.RKey, error) { return s.store.ComputeSplitKey(ctx, start, end) } diff --git a/pkg/spanconfig/spanconfigstore/BUILD.bazel b/pkg/spanconfig/spanconfigstore/BUILD.bazel index a755fa9a3093..238d99947d9c 100644 --- a/pkg/spanconfig/spanconfigstore/BUILD.bazel +++ b/pkg/spanconfig/spanconfigstore/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigbounds", "//pkg/util/iterutil", "//pkg/util/log", "//pkg/util/protoutil", @@ -44,6 +45,7 @@ go_test( "//pkg/roachpb", "//pkg/settings/cluster", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigbounds", "//pkg/spanconfig/spanconfigtestutils", "//pkg/testutils/datapathutils", "//pkg/util/hlc", diff --git a/pkg/spanconfig/spanconfigstore/store.go b/pkg/spanconfig/spanconfigstore/store.go index 1523f6129ce8..0eda3a8e32c4 100644 --- a/pkg/spanconfig/spanconfigstore/store.go +++ b/pkg/spanconfig/spanconfigstore/store.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" @@ -46,6 +47,16 @@ var FallbackConfigOverride = settings.RegisterProtobufSetting( &roachpb.SpanConfig{}, ) +// BoundsEnabled is a hidden cluster setting which controls whether +// SpanConfigBounds should be consulted (to perform clamping of secondary tenant +// span configurations) before serving span configs. +var boundsEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "spanconfig.bounds.enabled", + "dictates whether span config bounds are consulted when serving span configs for secondary tenants", + false, +) + // Store is an in-memory data structure to store, retrieve, and incrementally // update the span configuration state. Internally, it makes use of an interval // btree based spanConfigStore to store non-overlapping span configurations that @@ -74,21 +85,28 @@ type Store struct { fallback roachpb.SpanConfig knobs *spanconfig.TestingKnobs + + // boundsReader provides a handle to the global SpanConfigBounds state. + boundsReader spanconfigbounds.Reader } var _ spanconfig.Store = &Store{} // New instantiates a span config store with the given fallback. func New( - fallback roachpb.SpanConfig, settings *cluster.Settings, knobs *spanconfig.TestingKnobs, + fallback roachpb.SpanConfig, + settings *cluster.Settings, + boundsReader spanconfigbounds.Reader, + knobs *spanconfig.TestingKnobs, ) *Store { if knobs == nil { knobs = &spanconfig.TestingKnobs{} } s := &Store{ - settings: settings, - fallback: fallback, - knobs: knobs, + settings: settings, + fallback: fallback, + boundsReader: boundsReader, + knobs: knobs, } s.mu.spanConfigStore = newSpanConfigStore(settings, s.knobs) s.mu.systemSpanConfigStore = newSystemSpanConfigStore() @@ -96,20 +114,21 @@ func New( } // NeedsSplit is part of the spanconfig.StoreReader interface. -func (s *Store) NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool { - return len(s.ComputeSplitKey(ctx, start, end)) > 0 +func (s *Store) NeedsSplit(ctx context.Context, start, end roachpb.RKey) (bool, error) { + splits, err := s.ComputeSplitKey(ctx, start, end) + if err != nil { + return false, err + } + + return len(splits) > 0, nil } // ComputeSplitKey is part of the spanconfig.StoreReader interface. -func (s *Store) ComputeSplitKey(ctx context.Context, start, end roachpb.RKey) roachpb.RKey { +func (s *Store) ComputeSplitKey(_ context.Context, start, end roachpb.RKey) (roachpb.RKey, error) { s.mu.RLock() defer s.mu.RUnlock() - splitKey, err := s.mu.spanConfigStore.computeSplitKey(start, end) - if err != nil { - log.FatalfDepth(ctx, 3, "unable to compute split key: %v", err) - } - return splitKey + return s.mu.spanConfigStore.computeSplitKey(start, end) } // GetSpanConfigForKey is part of the spanconfig.StoreReader interface. @@ -131,7 +150,37 @@ func (s *Store) getSpanConfigForKeyRLocked( if !found { conf = s.getFallbackConfig() } - return s.mu.systemSpanConfigStore.combine(key, conf) + var err error + conf, err = s.mu.systemSpanConfigStore.combine(key, conf) + if err != nil { + return roachpb.SpanConfig{}, err + } + + // No need to perform clamping if SpanConfigBounds are not enabled. + if !boundsEnabled.Get(&s.settings.SV) { + return conf, nil + } + + _, tenID, err := keys.DecodeTenantPrefix(roachpb.Key(key)) + if err != nil { + return roachpb.SpanConfig{}, err + } + if tenID.IsSystem() { + // SpanConfig bounds do not apply to the system tenant. + return conf, nil + } + + bounds, found := s.boundsReader.Bounds(tenID) + if !found { + return conf, nil + } + + clamped := bounds.Clamp(&conf) + + if clamped { + log.VInfof(ctx, 3, "span config for tenant clamped to %v", conf) + } + return conf, nil } func (s *Store) getFallbackConfig() roachpb.SpanConfig { @@ -172,7 +221,7 @@ func (s *Store) Clone() *Store { s.mu.Lock() defer s.mu.Unlock() - clone := New(s.fallback, s.settings, s.knobs) + clone := New(s.fallback, s.settings, s.boundsReader, s.knobs) clone.mu.spanConfigStore = s.mu.spanConfigStore.clone() clone.mu.systemSpanConfigStore = s.mu.systemSpanConfigStore.clone() return clone diff --git a/pkg/spanconfig/spanconfigstore/store_test.go b/pkg/spanconfig/spanconfigstore/store_test.go index 74fdd9596984..d17491b06306 100644 --- a/pkg/spanconfig/spanconfigstore/store_test.go +++ b/pkg/spanconfig/spanconfigstore/store_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -104,6 +105,11 @@ func (s *spanConfigStore) TestingSplitKeys(tb testing.TB, start, end roachpb.RKe // A (refs = 2) // B (refs = 1) // +// declare-bounds +// set /Tenant/20:{GC.ttl_start=15, GC.ttl_end=30} +// delete /Tenant/10 +// ---- +// // Text of the form [a,b), {entire-keyspace}, {source=1,target=20}, and [a,b):C // correspond to targets {spans, system targets} and span config records; see // spanconfigtestutils.Parse{Target,Config,SpanConfigRecord} for more details. @@ -111,10 +117,14 @@ func TestDataDriven(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() + boundsReader := newMockBoundsReader() + settings := cluster.MakeTestingClusterSettings() + boundsEnabled.Override(ctx, &settings.SV, true) datadriven.Walk(t, datapathutils.TestDataPath(t), func(t *testing.T, path string) { store := New( spanconfigtestutils.ParseConfig(t, "FALLBACK"), - cluster.MakeTestingClusterSettings(), + settings, + boundsReader, &spanconfig.TestingKnobs{ StoreIgnoreCoalesceAdjacentExceptions: true, StoreInternConfigsInDryRuns: true, @@ -147,7 +157,8 @@ func TestDataDriven(t *testing.T) { case "get": d.ScanArgs(t, "key", &keyStr) - config, err := store.GetSpanConfigForKey(ctx, roachpb.RKey(keyStr)) + key, _ := spanconfigtestutils.ParseKey(t, keyStr) + config, err := store.GetSpanConfigForKey(ctx, roachpb.RKey(key)) require.NoError(t, err) return fmt.Sprintf("conf=%s", spanconfigtestutils.PrintSpanConfig(config)) @@ -155,14 +166,16 @@ func TestDataDriven(t *testing.T) { d.ScanArgs(t, "span", &spanStr) span := spanconfigtestutils.ParseSpan(t, spanStr) start, end := roachpb.RKey(span.Key), roachpb.RKey(span.EndKey) - result := store.NeedsSplit(ctx, start, end) + result, err := store.NeedsSplit(ctx, start, end) + require.NoError(t, err) return fmt.Sprintf("%t", result) case "compute-split": d.ScanArgs(t, "span", &spanStr) span := spanconfigtestutils.ParseSpan(t, spanStr) start, end := roachpb.RKey(span.Key), roachpb.RKey(span.EndKey) - splitKey := store.ComputeSplitKey(ctx, start, end) + splitKey, err := store.ComputeSplitKey(ctx, start, end) + require.NoError(t, err) if splitKey == nil { return "n/a" } @@ -205,6 +218,9 @@ func TestDataDriven(t *testing.T) { } return b.String() + case "declare-bounds": + updates := spanconfigtestutils.ParseDeclareBoundsArguments(t, d.Input) + boundsReader.apply(updates) default: t.Fatalf("unknown command: %s", d.Cmd) } @@ -214,6 +230,34 @@ func TestDataDriven(t *testing.T) { }) } +type mockBoundsReader struct { + bounds map[roachpb.TenantID]spanconfigbounds.Bounds +} + +func newMockBoundsReader() *mockBoundsReader { + m := mockBoundsReader{ + bounds: make(map[roachpb.TenantID]spanconfigbounds.Bounds), + } + return &m +} + +// Bounds implements the spanconfigbounds.Reader interface. +func (m *mockBoundsReader) Bounds(id roachpb.TenantID) (spanconfigbounds.Bounds, bool) { + bounds, found := m.bounds[id] + return bounds, found +} + +func (m *mockBoundsReader) apply(updates []spanconfigtestutils.BoundsUpdate) { + for _, update := range updates { + if update.Deleted { + delete(m.bounds, update.TenantID) + continue + } + + m.bounds[update.TenantID] = update.Bounds + } +} + // TestStoreClone verifies that a cloned store contains the same contents as the // original. func TestStoreClone(t *testing.T) { @@ -257,7 +301,12 @@ func TestStoreClone(t *testing.T) { ), } - original := New(roachpb.TestingDefaultSpanConfig(), cluster.MakeClusterSettings(), nil) + original := New( + roachpb.TestingDefaultSpanConfig(), + cluster.MakeClusterSettings(), + spanconfigbounds.NewEmptyReader(), + nil, + ) original.Apply(ctx, false, updates...) clone := original.Clone() @@ -293,6 +342,7 @@ func BenchmarkStoreComputeSplitKey(b *testing.B) { store := New( roachpb.SpanConfig{}, cluster.MakeClusterSettings(), + spanconfigbounds.NewEmptyReader(), &spanconfig.TestingKnobs{ StoreIgnoreCoalesceAdjacentExceptions: true, }, @@ -320,7 +370,8 @@ func BenchmarkStoreComputeSplitKey(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - _ = store.ComputeSplitKey(ctx, roachpb.RKey(query.Key), roachpb.RKey(query.EndKey)) + _, err := store.ComputeSplitKey(ctx, roachpb.RKey(query.Key), roachpb.RKey(query.EndKey)) + require.NoError(b, err) } }) } diff --git a/pkg/spanconfig/spanconfigstore/testdata/bounds/basic b/pkg/spanconfig/spanconfigstore/testdata/bounds/basic new file mode 100644 index 000000000000..109e3f169d56 --- /dev/null +++ b/pkg/spanconfig/spanconfigstore/testdata/bounds/basic @@ -0,0 +1,72 @@ +apply +set [/Tenant/10/a,/Tenant/10/b):GC.ttl=10 +set [/Tenant/10/f,/Tenant/10/h):GC.ttl=20 +set [/Tenant/10/h,/Tenant/10/j):GC.ttl=35 +---- +added [/Tenant/10/a,/Tenant/10/b):GC.ttl=10 +added [/Tenant/10/f,/Tenant/10/h):GC.ttl=20 +added [/Tenant/10/h,/Tenant/10/j):GC.ttl=35 + +apply +set [/Tenant/20/a,/Tenant/20/b):GC.ttl=55 +set [/Tenant/20/c,/Tenant/20/d):GC.ttl=25 +set [/Tenant/20/e,/Tenant/20/f):GC.ttl=3 +---- +added [/Tenant/20/a,/Tenant/20/b):GC.ttl=55 +added [/Tenant/20/c,/Tenant/20/d):GC.ttl=25 +added [/Tenant/20/e,/Tenant/20/f):GC.ttl=3 + +declare-bounds +set /Tenant/10:{GC.ttl_start=15, GC.ttl_end=30} +set /Tenant/20:{GC.ttl_start=5, GC.ttl_end=50} +---- + +# Ensure GC TTL is clamped correctly for tenant 10. +overlapping span=[/Tenant/10/a,/Tenant/10/j) +---- +[/Tenant/10/a,/Tenant/10/b):GC.ttl=15 +[/Tenant/10/f,/Tenant/10/h):GC.ttl=20 +[/Tenant/10/h,/Tenant/10/j):GC.ttl=30 + +get key=/Tenant/10/a +---- +conf=GC.ttl=15 + + +# Ditto for tenant 20. +overlapping span=[/Tenant/20/a,/Tenant/20/f) +---- +[/Tenant/20/a,/Tenant/20/b):GC.ttl=50 +[/Tenant/20/c,/Tenant/20/d):GC.ttl=25 +[/Tenant/20/e,/Tenant/20/f):GC.ttl=5 + +get key=/Tenant/20/e +---- +conf=GC.ttl=5 + +# Delete bounds for tenant 10. +declare-bounds +delete /Tenant/10 +---- + +# Ensure GC TTL values are no longer clamped for tenant 10. +overlapping span=[/Tenant/10/a,/Tenant/10/j) +---- +[/Tenant/10/a,/Tenant/10/b):GC.ttl=10 +[/Tenant/10/f,/Tenant/10/h):GC.ttl=20 +[/Tenant/10/h,/Tenant/10/j):GC.ttl=35 + +get key=/Tenant/10/a +---- +conf=GC.ttl=10 + +# However, this shouldn't affect tenant 20. +overlapping span=[/Tenant/20/a,/Tenant/20/f) +---- +[/Tenant/20/a,/Tenant/20/b):GC.ttl=50 +[/Tenant/20/c,/Tenant/20/d):GC.ttl=25 +[/Tenant/20/e,/Tenant/20/f):GC.ttl=5 + +get key=/Tenant/20/e +---- +conf=GC.ttl=5 diff --git a/pkg/spanconfig/spanconfigtestutils/BUILD.bazel b/pkg/spanconfig/spanconfigtestutils/BUILD.bazel index ccd602dbb69f..773614f5cbc1 100644 --- a/pkg/spanconfig/spanconfigtestutils/BUILD.bazel +++ b/pkg/spanconfig/spanconfigtestutils/BUILD.bazel @@ -11,10 +11,13 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/config/zonepb", + "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver/protectedts/ptpb", + "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", "//pkg/roachpb", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigbounds", "//pkg/sql/catalog/descpb", "//pkg/util/hlc", "//pkg/util/syncutil", diff --git a/pkg/spanconfig/spanconfigtestutils/utils.go b/pkg/spanconfig/spanconfigtestutils/utils.go index 527592bac371..7400cada03ae 100644 --- a/pkg/spanconfig/spanconfigtestutils/utils.go +++ b/pkg/spanconfig/spanconfigtestutils/utils.go @@ -11,7 +11,6 @@ package spanconfigtestutils import ( - "context" "fmt" "reflect" "regexp" @@ -21,9 +20,12 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" + "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/datadriven" @@ -33,7 +35,7 @@ import ( // spanRe matches strings of the form "[start, end)", capturing both the "start" // and "end" keys. -var spanRe = regexp.MustCompile(`^\[(\w+),\s??(\w+)\)$`) +var spanRe = regexp.MustCompile(`^\[((/Tenant/\d*/)?\w+),\s??((/Tenant/\d*/)?\w+)\)$`) // systemTargetRe matches strings of the form // "{entire-keyspace|source=,(target=|all-tenant-keyspace-targets-set)}". @@ -41,9 +43,21 @@ var systemTargetRe = regexp.MustCompile( `^{(entire-keyspace)|(source=(\d*),\s??((target=(\d*))|all-tenant-keyspace-targets-set))}$`, ) -// configRe matches either FALLBACK (for readability) or a single letter. It's a -// shorthand for declaring a unique tagged config. -var configRe = regexp.MustCompile(`^(FALLBACK)|(^\w)$`) +// configRe matches either FALLBACK (for readability), a single letter, or a +// specified GC TTL value. It's a shorthand for declaring a unique tagged config. +var configRe = regexp.MustCompile(`^(FALLBACK)|(^GC\.ttl=(\d*))|(^\w)$`) + +// boundsRe matches a string of the form "{GC.ttl_start=,GC.ttl_end=}". +// It translates to an upper/lower bound expressed through SpanConfigBounds. +var boundsRe = regexp.MustCompile(`{GC\.ttl_start=(\d*),\s??GC.ttl_end=(\d*)}`) + +// tenantRe matches a string of the form "/Tenant/". +var tenantRe = regexp.MustCompile(`/Tenant/(\d*)`) + +// keyRe matches a string of the form "a", "Tenant/10/a". An optional tenant +// prefix may be used to specify a key inside a secondary tenant's keyspace; +// otherwise, the key is within the system tenant's keyspace. +var keyRe = regexp.MustCompile(`(Tenant/\d*/)?(\w+)`) // ParseRangeID is helper function that constructs a roachpb.RangeID from a // string of the form "r". @@ -145,11 +159,48 @@ func ParseSpan(t testing.TB, sp string) roachpb.Span { } matches := spanRe.FindStringSubmatch(sp) - start, end := matches[1], matches[2] + startStr, endStr := matches[1], matches[3] + start, tenStart := ParseKey(t, startStr) + end, tenEnd := ParseKey(t, endStr) + + // Sanity check keys don't straddle tenant boundaries. + require.Equal(t, tenStart, tenEnd) + return roachpb.Span{ - Key: roachpb.Key(start), - EndKey: roachpb.Key(end), + Key: start, + EndKey: end, + } +} + +// ParseKey constructs a roachpb.Key from the supplied input. The key may be +// optionally prefixed with a "/Tenant/ID/" prefix; doing so ensures the key +// belongs to the specified tenant's keyspace. Otherwise, the key lies in the +// system tenant's keyspace. +// +// In addition to the key, the tenant ID is also returned. +func ParseKey(t testing.TB, key string) (roachpb.Key, roachpb.TenantID) { + require.True(t, keyRe.MatchString(key)) + + matches := keyRe.FindStringSubmatch(key) + tenantID := roachpb.SystemTenantID + if matches[1] != "" { + tenantID = parseTenant(t, key) } + + codec := keys.MakeSQLCodec(tenantID) + return append(codec.TenantPrefix(), roachpb.Key(matches[2])...), tenantID +} + +// parseTenant expects a string of the form "ten-" and returns the +// tenant ID. +func parseTenant(t testing.TB, input string) roachpb.TenantID { + require.True(t, tenantRe.MatchString(input), input) + + matches := tenantRe.FindStringSubmatch(input) + tenID := matches[1] + tenIDRaw, err := strconv.Atoi(tenID) + require.NoError(t, err) + return roachpb.MustMakeTenantID(uint64(tenIDRaw)) } // parseSystemTarget is a helepr function that constructs a @@ -193,8 +244,7 @@ func ParseTarget(t testing.TB, target string) spanconfig.Target { } // ParseConfig is helper function that constructs a roachpb.SpanConfig that's -// "tagged" with the given string (i.e. a constraint with the given string a -// required key). +// "tagged" with the given string. See configRe for specifics. func ParseConfig(t testing.TB, conf string) roachpb.SpanConfig { if !configRe.MatchString(conf) { t.Fatalf("expected %s to match config regex", conf) @@ -202,24 +252,106 @@ func ParseConfig(t testing.TB, conf string) roachpb.SpanConfig { matches := configRe.FindStringSubmatch(conf) var ts int64 + var gcTTL int if matches[1] == "FALLBACK" { ts = -1 + } else if matches[4] != "" { + ts = int64(matches[4][0]) } else { - ts = int64(matches[2][0]) + var err error + gcTTL, err = strconv.Atoi(matches[3]) + require.NoError(t, err) } - return roachpb.SpanConfig{ - GCPolicy: roachpb.GCPolicy{ - ProtectionPolicies: []roachpb.ProtectionPolicy{ - { - ProtectedTimestamp: hlc.Timestamp{ - WallTime: ts, - }, + + var protectionPolicies []roachpb.ProtectionPolicy + if ts != 0 { + protectionPolicies = []roachpb.ProtectionPolicy{ + { + ProtectedTimestamp: hlc.Timestamp{ + WallTime: ts, }, }, + } + } + return roachpb.SpanConfig{ + GCPolicy: roachpb.GCPolicy{ + ProtectionPolicies: protectionPolicies, + TTLSeconds: int32(gcTTL), }, } } +// BoundsUpdate .. +type BoundsUpdate struct { + TenantID roachpb.TenantID + Bounds spanconfigbounds.Bounds + Deleted bool +} + +// ParseDeclareBoundsArguments parses datadriven test input to a list of +// tenantcapabilities.Updates that can be applied to a +// tenantcapabilities.Reader. The input is of the following form: +// +// delete ten-10 +// set ten-10:{GC.ttl_start=5, GC.ttl_end=30} +func ParseDeclareBoundsArguments(t *testing.T, input string) (updates []BoundsUpdate) { + for _, line := range strings.Split(input, "\n") { + line = strings.TrimSpace(line) + if line == "" { + continue + } + + const setPrefix, deletePrefix = "set ", "delete " + + switch { + case strings.HasPrefix(line, setPrefix): + line = strings.TrimPrefix(line, line[:len(setPrefix)]) + parts := strings.Split(line, ":") + require.Len(t, parts, 2) + tenantID := parseTenant(t, parts[0]) + bounds := parseBounds(t, parts[1]) + updates = append(updates, BoundsUpdate{ + TenantID: tenantID, + Bounds: bounds, + }) + + case strings.HasPrefix(line, deletePrefix): + line = strings.TrimPrefix(line, line[:len(deletePrefix)]) + tenantID := parseTenant(t, line) + updates = append(updates, + BoundsUpdate{ + TenantID: tenantID, + Bounds: spanconfigbounds.Bounds{}, + Deleted: true, + }, + ) + default: + t.Fatalf("malformed line %q, expected to find prefix %q or %q", + line, setPrefix, deletePrefix) + } + } + return updates +} + +// parseBounds parses a string that looks like {GC.ttl_start=5, GC.ttl_end=40} +// into a spanconfigbounds.Bounds struct. +func parseBounds(t *testing.T, input string) spanconfigbounds.Bounds { + require.True(t, boundsRe.MatchString(input)) + + matches := boundsRe.FindStringSubmatch(input) + gcTTLStart, err := strconv.Atoi(matches[1]) + require.NoError(t, err) + gcTTLEnd, err := strconv.Atoi(matches[2]) + require.NoError(t, err) + + return spanconfigbounds.MakeBounds(&tenantcapabilitiespb.SpanConfigBounds{ + GCTTLSeconds: &tenantcapabilitiespb.SpanConfigBounds_Int32Range{ + Start: int32(gcTTLStart), + End: int32(gcTTLEnd), + }, + }) +} + // ParseSpanConfigRecord is helper function that constructs a // spanconfig.Target from a string of the form target:config. See // ParseTarget and ParseConfig above. @@ -347,21 +479,44 @@ func ParseStoreApplyArguments(t testing.TB, input string) (updates []spanconfig. // roundtrip; spans containing special keys that translate to pretty-printed // keys are printed as such. func PrintSpan(sp roachpb.Span) string { - s := []string{ - sp.Key.String(), - sp.EndKey.String(), - } - for i := range s { + var res []string + for _, key := range []roachpb.Key{sp.Key, sp.EndKey} { + var tenantPrefixStr string + rest, tenID, err := keys.DecodeTenantPrefix(key) + if err != nil { + panic(err) + } + + if !tenID.IsSystem() { + tenantPrefixStr = fmt.Sprintf("%s", keys.MakeSQLCodec(tenID).TenantPrefix()) + } + // Raw keys are quoted, so we unquote them. - if strings.Contains(s[i], "\"") { + restStr := roachpb.Key(rest).String() + if strings.Contains(restStr, "\"") { var err error - s[i], err = strconv.Unquote(s[i]) + restStr, err = strconv.Unquote(restStr) if err != nil { panic(err) } } + + // For keys inside a secondary tenant, we don't print the "/Min" suffix if + // the key is at the start of their keyspace. Also, we add a "/" delimiter + // after the tenant prefix. + if !tenID.IsSystem() { + if roachpb.Key(rest).Compare(keys.MinKey) == 0 { + restStr = "" + } + + if restStr != "" { + restStr = fmt.Sprintf("/%s", restStr) + } + } + + res = append(res, fmt.Sprintf("%s%s", tenantPrefixStr, restStr)) } - return fmt.Sprintf("[%s,%s)", s[0], s[1]) + return fmt.Sprintf("[%s,%s)", res[0], res[1]) } // PrintTarget is a helper function that prints a spanconfig.Target. @@ -382,6 +537,12 @@ func PrintTarget(t testing.TB, target spanconfig.Target) string { // ParseSpanConfig helper above. func PrintSpanConfig(config roachpb.SpanConfig) string { // See ParseConfig for what a "tagged" roachpb.SpanConfig translates to. + if config.GCPolicy.TTLSeconds != 0 && len(config.GCPolicy.ProtectionPolicies) != 0 { + panic("cannot have both TTL and protection policies set for tagged configs") // sanity check + } + if config.GCPolicy.TTLSeconds != 0 { + return fmt.Sprintf("GC.ttl=%d", config.GCPolicy.TTLSeconds) + } conf := make([]string, 0, len(config.GCPolicy.ProtectionPolicies)*2) for i, policy := range config.GCPolicy.ProtectionPolicies { if i > 0 { @@ -545,51 +706,6 @@ func MaybeLimitAndOffset( return strings.TrimSpace(output.String()) } -// SplitPoint is a unit of what's retrievable from a spanconfig.StoreReader. It -// captures a single split point, and the config to be applied over the -// following key span (or at least until the next such SplitPoint). -// -// TODO(irfansharif): Find a better name? -type SplitPoint struct { - RKey roachpb.RKey - Config roachpb.SpanConfig -} - -// SplitPoints is a collection of split points. -type SplitPoints []SplitPoint - -func (rs SplitPoints) String() string { - var output strings.Builder - for _, c := range rs { - output.WriteString(fmt.Sprintf("%-42s %s\n", c.RKey.String(), - PrintSpanConfigDiffedAgainstDefaults(c.Config))) - } - return output.String() -} - -// GetSplitPoints returns a list of range split points as suggested by the given -// StoreReader. -func GetSplitPoints(ctx context.Context, t testing.TB, reader spanconfig.StoreReader) SplitPoints { - var splitPoints []SplitPoint - splitKey := roachpb.RKeyMin - for { - splitKeyConf, err := reader.GetSpanConfigForKey(ctx, splitKey) - require.NoError(t, err) - - splitPoints = append(splitPoints, SplitPoint{ - RKey: splitKey, - Config: splitKeyConf, - }) - - if !reader.NeedsSplit(ctx, splitKey, roachpb.RKeyMax) { - break - } - splitKey = reader.ComputeSplitKey(ctx, splitKey, roachpb.RKeyMax) - } - - return splitPoints -} - // ParseProtectionTarget returns a ptpb.Target based on the input. This target // could either refer to a Cluster, list of Tenants or SchemaObjects. func ParseProtectionTarget(t testing.TB, input string) *ptpb.Target { diff --git a/pkg/spanconfig/spanconfigtestutils/utils_test.go b/pkg/spanconfig/spanconfigtestutils/utils_test.go index 1df3189d16f3..3c3c5ed54295 100644 --- a/pkg/spanconfig/spanconfigtestutils/utils_test.go +++ b/pkg/spanconfig/spanconfigtestutils/utils_test.go @@ -11,6 +11,7 @@ package spanconfigtestutils import ( + "strings" "testing" "github.com/stretchr/testify/require" @@ -20,16 +21,26 @@ func TestSpanRe(t *testing.T) { for _, tc := range []struct { input string expMatch bool + expTenant bool expStart, expEnd string }{ - {"[a, b)", true, "a", "b"}, - {"[acd, bfg)", true, "acd", "bfg"}, // multi character keys allowed - {"[a,b)", true, "a", "b"}, // separating space is optional - {"[ a,b) ", false, "", ""}, // extraneous spaces disallowed - {"[a,b ) ", false, "", ""}, // extraneous spaces disallowed - {"[a,, b)", false, "", ""}, // only single comma allowed - {" [a, b)", false, "", ""}, // need to start with '[' - {"[a,b)x", false, "", ""}, // need to end with ')' + {"[a, b)", true, false, "a", "b"}, + // Multi character keys allowed. + {"[acd, bfg)", true, false, "acd", "bfg"}, + // Separating space is optional. + {"[a,b)", true, false, "a", "b"}, + // Tenant span. + {"[/Tenant/10/a,/Tenant/10/b)", true, true, "/Tenant/10/a", "/Tenant/10/b"}, + // Extraneous spaces disallowed. + {"[ a,b) ", false, false, "", ""}, + // Extraneous spaces disallowed. + {"[a,b ) ", false, false, "", ""}, + // Only single comma allowed. + {"[a,, b)", false, false, "", ""}, + // Need to start with '['. + {" [a, b)", false, false, "", ""}, + // Need to end with ')'. + {"[a,b)x", false, false, "", ""}, } { require.Equalf(t, tc.expMatch, spanRe.MatchString(tc.input), "input = %s", tc.input) if !tc.expMatch { @@ -37,8 +48,17 @@ func TestSpanRe(t *testing.T) { } matches := spanRe.FindStringSubmatch(tc.input) - require.Len(t, matches, 3) - start, end := matches[1], matches[2] + require.Len(t, matches, 5) + start, end := matches[1], matches[3] + + const tenPrefix = "/Tenant/" + if tc.expTenant { + require.True(t, strings.HasPrefix(matches[2], tenPrefix)) + require.True(t, strings.HasPrefix(matches[4], tenPrefix)) + } else { + require.Equal(t, 0, len(matches[2])) + require.Equal(t, 0, len(matches[4])) + } require.Equal(t, tc.expStart, start) require.Equal(t, tc.expEnd, end) } diff --git a/pkg/sql/catalog/bootstrap/testdata/testdata b/pkg/sql/catalog/bootstrap/testdata/testdata index 316860100cd4..4a6ef3d2ef80 100644 --- a/pkg/sql/catalog/bootstrap/testdata/testdata +++ b/pkg/sql/catalog/bootstrap/testdata/testdata @@ -1,4 +1,4 @@ -system hash=463a3ab3201f211297b8ff6b8782e2ebbe0cd3fbbc77e8159ae81a5f41a5a88f +system hash=9bd17e44837cdf988000b364eeaa677ae3fbe12ca7cd87419117104243539577 ---- [{"key":"04646573632d696467656e","value":"01c801"} ,{"key":"8b"} @@ -44,7 +44,7 @@ system hash=463a3ab3201f211297b8ff6b8782e2ebbe0cd3fbbc77e8159ae81a5f41a5a88f ,{"key":"8b89ba8a89","value":"030ad7050a0f74656e616e745f73657474696e67731832200128013a00422e0a0974656e616e745f696410011a0c0801104018003000501460002000300068007000780080010088010098010042290a046e616d6510021a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510031a0c0807100018003000501960002000300068007000780080010088010098010042450a0c6c6173745f7570646174656410041a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422f0a0a76616c75655f7479706510051a0c08071000180030005019600020003000680070007800800100880100980100422b0a06726561736f6e10061a0c080710001800300050196000200130006800700078008001008801009801004807529b010a077072696d61727910011801220974656e616e745f696422046e616d652a0576616c75652a0c6c6173745f757064617465642a0a76616c75655f747970652a06726561736f6e30013002400040004a10080010001a00200028003000380040005a0070037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b20185010a3966616d5f305f74656e616e745f69645f6e616d655f76616c75655f6c6173745f757064617465645f76616c75655f747970655f726561736f6e10001a0974656e616e745f69641a046e616d651a0576616c75651a0c6c6173745f757064617465641a0a76616c75655f747970651a06726561736f6e2001200220032004200520062800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89bb8a89","value":"030ab2070a0a70726976696c656765731833200128013a00422d0a08757365726e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042290a047061746810021a0c0807100018003000501960002000300068007000780080010088010098010042400a0a70726976696c6567657310031a1d080f100018003000380750f1075a0c08071000180030005019600060002000300068007000780080010088010098010042430a0d6772616e745f6f7074696f6e7310041a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07757365725f696410051a0c080c100018003000501a60002000300068007000780080010088010098010048065293010a077072696d617279100118012208757365726e616d652204706174682a0a70726976696c656765732a0d6772616e745f6f7074696f6e732a07757365725f696430013002400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00103e001005a9d010a1b70726976696c656765735f706174685f757365725f69645f6b6579100218012204706174682207757365725f69642a0a70726976696c656765732a0d6772616e745f6f7074696f6e73300230053801400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e001005a9d010a1c70726976696c656765735f706174685f757365726e616d655f6b6579100318012204706174682208757365726e616d652a0a70726976696c656765732a0d6772616e745f6f7074696f6e7330023001400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00102e0010060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2014b0a077072696d61727910001a08757365726e616d651a04706174681a0a70726976696c656765731a0d6772616e745f6f7074696f6e731a07757365725f6964200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880304a80300b00300"} ,{"key":"8b89bc8a89","value":"030ac1060a1465787465726e616c5f636f6e6e656374696f6e731834200128013a0042340a0f636f6e6e656374696f6e5f6e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042400a076372656174656410021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042400a077570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042340a0f636f6e6e656374696f6e5f7479706510041a0c0807100018003000501960002000300068007000780080010088010098010042370a12636f6e6e656374696f6e5f64657461696c7310051a0c08081000180030005011600020003000680070007800800100880100980100422a0a056f776e657210061a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410071a0c080c100018003000501a600020003000680070007800800100880100980100480852ba010a077072696d61727910011801220f636f6e6e656374696f6e5f6e616d652a07637265617465642a07757064617465642a0f636f6e6e656374696f6e5f747970652a12636f6e6e656374696f6e5f64657461696c732a056f776e65722a086f776e65725f6964300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201740a077072696d61727910001a0f636f6e6e656374696f6e5f6e616d651a07637265617465641a07757064617465641a0f636f6e6e656374696f6e5f747970651a12636f6e6e656374696f6e5f64657461696c731a056f776e65721a086f776e65725f696420012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} -,{"key":"8b89bd8a89","value":"030a87040a086a6f625f696e666f1835200128013a00422b0a066a6f625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e666f5f6b657910021a0c0808100018003000501160002000300068007000780080010088010098010042420a077772697474656e10031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a0576616c756510041a0c0808100018003000501160002001300068007000780080010088010098010048055281010a077072696d6172791001180122066a6f625f69642208696e666f5f6b657922077772697474656e2a0576616c75653001300230034000400040014a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201370a077072696d61727910001a066a6f625f69641a08696e666f5f6b65791a077772697474656e1a0576616c756520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} +,{"key":"8b89bd8a89","value":"030a87040a086a6f625f696e666f1835200128013a00422b0a066a6f625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e666f5f6b657910021a0c0807100018003000501960002000300068007000780080010088010098010042420a077772697474656e10031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a0576616c756510041a0c0808100018003000501160002001300068007000780080010088010098010048055281010a077072696d6172791001180122066a6f625f69642208696e666f5f6b657922077772697474656e2a0576616c75653001300230034000400040014a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201370a077072696d61727910001a066a6f625f69641a08696e666f5f6b65791a077772697474656e1a0576616c756520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89be8a89","value":"030af9030a167370616e5f73746174735f756e697175655f6b6579731836200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422e0a096b65795f627974657310021a0c08081000180030005011600020013000680070007800800100880100980100480352660a077072696d61727910011801220269642a096b65795f6279746573300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e001005a740a19756e697175655f6b6579735f6b65795f62797465735f6964781002180122096b65795f62797465733002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201200a077072696d61727910001a0269641a096b65795f6279746573200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} ,{"key":"8b89bf8a89","value":"030adc050a127370616e5f73746174735f6275636b6574731837200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422f0a0973616d706c655f696410021a0d080e10001800300050861760002000300068007000780080010088010098010042320a0c73746172745f6b65795f696410031a0d080e10001800300050861760002000300068007000780080010088010098010042300a0a656e645f6b65795f696410041a0d080e100018003000508617600020003000680070007800800100880100980100422d0a08726571756573747310051a0c0801104018003000501460002000300068007000780080010088010098010048065290010a077072696d61727910011801220269642a0973616d706c655f69642a0c73746172745f6b65795f69642a0a656e645f6b65795f69642a087265717565737473300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e001005a700a156275636b6574735f73616d706c655f69645f69647810021800220973616d706c655f69643002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e0010060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2014a0a077072696d61727910001a0269641a0973616d706c655f69641a0c73746172745f6b65795f69641a0a656e645f6b65795f69641a087265717565737473200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89c08a89","value":"030a8f040a127370616e5f73746174735f73616d706c65731838200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f757569642829300068007000780080010088010098010042440a0b73616d706c655f74696d6510021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100480352680a077072696d61727910011801220269642a0b73616d706c655f74696d65300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e001005a740a1773616d706c65735f73616d706c655f74696d655f69647810021801220b73616d706c655f74696d653002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201220a077072696d61727910001a0269641a0b73616d706c655f74696d65200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} @@ -175,7 +175,7 @@ system hash=463a3ab3201f211297b8ff6b8782e2ebbe0cd3fbbc77e8159ae81a5f41a5a88f ,{"key":"c5"} ] -tenant hash=238142119bd7cc980ba280eba1ac2cefeab072aff41563a2c7d839066baa2e81 +tenant hash=502a0cb141b849cf8f4581d19cd94e7e458c486f399583d6c31a07bd5e35b253 ---- [{"key":""} ,{"key":"8b89898a89","value":"0312390a0673797374656d10011a250a0d0a0561646d696e1080101880100a0c0a04726f6f7410801018801012046e6f646518022200280140004a00"} @@ -217,7 +217,7 @@ tenant hash=238142119bd7cc980ba280eba1ac2cefeab072aff41563a2c7d839066baa2e81 ,{"key":"8b89ba8a89","value":"030aa5030a0a7370616e5f636f756e741832200128013a0042340a0973696e676c65746f6e10011a0c08001000180030005010600020002a04747275653000680070007800800100880100980100422f0a0a7370616e5f636f756e7410021a0c080110401800300050146000200030006800700078008001008801009801004803526e0a077072696d61727910011801220973696e676c65746f6e2a0a7370616e5f636f756e74300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100a201210a0973696e676c65746f6e120a73696e676c655f726f7718002801300038004002b201280a077072696d61727910001a0973696e676c65746f6e1a0a7370616e5f636f756e74200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} ,{"key":"8b89bb8a89","value":"030ab2070a0a70726976696c656765731833200128013a00422d0a08757365726e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042290a047061746810021a0c0807100018003000501960002000300068007000780080010088010098010042400a0a70726976696c6567657310031a1d080f100018003000380750f1075a0c08071000180030005019600060002000300068007000780080010088010098010042430a0d6772616e745f6f7074696f6e7310041a1d080f100018003000380750f1075a0c080710001800300050196000600020003000680070007800800100880100980100422c0a07757365725f696410051a0c080c100018003000501a60002000300068007000780080010088010098010048065293010a077072696d617279100118012208757365726e616d652204706174682a0a70726976696c656765732a0d6772616e745f6f7074696f6e732a07757365725f696430013002400040004a10080010001a00200028003000380040005a007003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00103e001005a9d010a1b70726976696c656765735f706174685f757365725f69645f6b6579100218012204706174682207757365725f69642a0a70726976696c656765732a0d6772616e745f6f7074696f6e73300230053801400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e001005a9d010a1c70726976696c656765735f706174685f757365726e616d655f6b6579100318012204706174682208757365726e616d652a0a70726976696c656765732a0d6772616e745f6f7074696f6e7330023001400040004a10080010001a00200028003000380040005a00700370047a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00102e0010060046a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2014b0a077072696d61727910001a08757365726e616d651a04706174681a0a70726976696c656765731a0d6772616e745f6f7074696f6e731a07757365725f6964200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880304a80300b00300"} ,{"key":"8b89bc8a89","value":"030ac1060a1465787465726e616c5f636f6e6e656374696f6e731834200128013a0042340a0f636f6e6e656374696f6e5f6e616d6510011a0c0807100018003000501960002000300068007000780080010088010098010042400a076372656174656410021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042400a077570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d50300068007000780080010088010098010042340a0f636f6e6e656374696f6e5f7479706510041a0c0807100018003000501960002000300068007000780080010088010098010042370a12636f6e6e656374696f6e5f64657461696c7310051a0c08081000180030005011600020003000680070007800800100880100980100422a0a056f776e657210061a0c08071000180030005019600020003000680070007800800100880100980100422d0a086f776e65725f696410071a0c080c100018003000501a600020003000680070007800800100880100980100480852ba010a077072696d61727910011801220f636f6e6e656374696f6e5f6e616d652a07637265617465642a07757064617465642a0f636f6e6e656374696f6e5f747970652a12636f6e6e656374696f6e5f64657461696c732a056f776e65722a086f776e65725f6964300140004a10080010001a00200028003000380040005a007002700370047005700670077a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201740a077072696d61727910001a0f636f6e6e656374696f6e5f6e616d651a07637265617465641a07757064617465641a0f636f6e6e656374696f6e5f747970651a12636f6e6e656374696f6e5f64657461696c731a056f776e65721a086f776e65725f696420012002200320042005200620072800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} -,{"key":"8b89bd8a89","value":"030a87040a086a6f625f696e666f1835200128013a00422b0a066a6f625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e666f5f6b657910021a0c0808100018003000501160002000300068007000780080010088010098010042420a077772697474656e10031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a0576616c756510041a0c0808100018003000501160002001300068007000780080010088010098010048055281010a077072696d6172791001180122066a6f625f69642208696e666f5f6b657922077772697474656e2a0576616c75653001300230034000400040014a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201370a077072696d61727910001a066a6f625f69641a08696e666f5f6b65791a077772697474656e1a0576616c756520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} +,{"key":"8b89bd8a89","value":"030a87040a086a6f625f696e666f1835200128013a00422b0a066a6f625f696410011a0c08011040180030005014600020003000680070007800800100880100980100422d0a08696e666f5f6b657910021a0c0807100018003000501960002000300068007000780080010088010098010042420a077772697474656e10031a0d080910001800300050a009600020002a136e6f7728293a3a3a54494d455354414d50545a3000680070007800800100880100980100422a0a0576616c756510041a0c0808100018003000501160002001300068007000780080010088010098010048055281010a077072696d6172791001180122066a6f625f69642208696e666f5f6b657922077772697474656e2a0576616c75653001300230034000400040014a10080010001a00200028003000380040005a0070047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201370a077072696d61727910001a066a6f625f69641a08696e666f5f6b65791a077772697474656e1a0576616c756520012002200320042804b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89be8a89","value":"030af9030a167370616e5f73746174735f756e697175655f6b6579731836200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422e0a096b65795f627974657310021a0c08081000180030005011600020013000680070007800800100880100980100480352660a077072696d61727910011801220269642a096b65795f6279746573300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e001005a740a19756e697175655f6b6579735f6b65795f62797465735f6964781002180122096b65795f62797465733002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201200a077072696d61727910001a0269641a096b65795f6279746573200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} ,{"key":"8b89bf8a89","value":"030adc050a127370616e5f73746174735f6275636b6574731837200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f7575696428293000680070007800800100880100980100422f0a0973616d706c655f696410021a0d080e10001800300050861760002000300068007000780080010088010098010042320a0c73746172745f6b65795f696410031a0d080e10001800300050861760002000300068007000780080010088010098010042300a0a656e645f6b65795f696410041a0d080e100018003000508617600020003000680070007800800100880100980100422d0a08726571756573747310051a0c0801104018003000501460002000300068007000780080010088010098010048065290010a077072696d61727910011801220269642a0973616d706c655f69642a0c73746172745f6b65795f69642a0a656e645f6b65795f69642a087265717565737473300140004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e001005a700a156275636b6574735f73616d706c655f69645f69647810021800220973616d706c655f69643002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e0010060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2014a0a077072696d61727910001a0269641a0973616d706c655f69641a0c73746172745f6b65795f69641a0a656e645f6b65795f69641a087265717565737473200120022003200420052800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89c08a89","value":"030a8f040a127370616e5f73746174735f73616d706c65731838200128013a00423b0a02696410011a0d080e100018003000508617600020002a1167656e5f72616e646f6d5f757569642829300068007000780080010088010098010042440a0b73616d706c655f74696d6510021a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100480352680a077072696d61727910011801220269642a0b73616d706c655f74696d65300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e001005a740a1773616d706c65735f73616d706c655f74696d655f69647810021801220b73616d706c655f74696d653002380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060036a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201220a077072696d61727910001a0269641a0b73616d706c655f74696d65200120022802b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 0888f740c41a..8cbc37eae203 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -913,7 +913,7 @@ CREATE TABLE system.task_payloads ( SystemJobInfoTableSchema = ` CREATE TABLE system.job_info ( job_id INT8 NOT NULL, - info_key BYTES NOT NULL, + info_key STRING NOT NULL, -- written is in the PK for this table to give it mvcc-over-sql semantics, so -- that revisions to the value for the logical job/key pair are separate SQL -- rows. This is done because we do not allow KV ranges to split between the @@ -3801,7 +3801,7 @@ var ( descpb.InvalidID, // dynamically assigned []descpb.ColumnDescriptor{ {Name: "job_id", ID: 1, Type: types.Int}, - {Name: "info_key", ID: 2, Type: types.Bytes}, + {Name: "info_key", ID: 2, Type: types.String}, {Name: "written", ID: 3, Type: types.TimestampTZ, DefaultExpr: &nowTZString}, {Name: "value", ID: 4, Type: types.Bytes, Nullable: true}, }, diff --git a/pkg/sql/catalog/systemschema_test/testdata/bootstrap b/pkg/sql/catalog/systemschema_test/testdata/bootstrap index 25b51807404c..0638ecb0653a 100644 --- a/pkg/sql/catalog/systemschema_test/testdata/bootstrap +++ b/pkg/sql/catalog/systemschema_test/testdata/bootstrap @@ -444,7 +444,7 @@ CREATE TABLE public.external_connections ( ); CREATE TABLE public.job_info ( job_id INT8 NOT NULL, - info_key BYTES NOT NULL, + info_key STRING NOT NULL, written TIMESTAMPTZ NOT NULL DEFAULT now():::TIMESTAMPTZ, value BYTES NULL, CONSTRAINT "primary" PRIMARY KEY (job_id ASC, info_key ASC, written DESC) @@ -561,7 +561,7 @@ schema_telemetry {"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}} {"table":{"name":"external_connections","id":52,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"connection_name","id":1,"type":{"family":"StringFamily","oid":25}},{"name":"created","id":2,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"updated","id":3,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"connection_type","id":4,"type":{"family":"StringFamily","oid":25}},{"name":"connection_details","id":5,"type":{"family":"BytesFamily","oid":17}},{"name":"owner","id":6,"type":{"family":"StringFamily","oid":25}},{"name":"owner_id","id":7,"type":{"family":"OidFamily","oid":26}}],"nextColumnId":8,"families":[{"name":"primary","columnNames":["connection_name","created","updated","connection_type","connection_details","owner","owner_id"],"columnIds":[1,2,3,4,5,6,7]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["connection_name"],"keyColumnDirections":["ASC"],"storeColumnNames":["created","updated","connection_type","connection_details","owner","owner_id"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7],"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":"job_info","id":53,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"job_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"info_key","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"written","id":3,"type":{"family":"TimestampTZFamily","oid":1184},"defaultExpr":"now():::TIMESTAMPTZ"},{"name":"value","id":4,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["job_id","info_key","written","value"],"columnIds":[1,2,3,4],"defaultColumnId":4}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["job_id","info_key","written"],"keyColumnDirections":["ASC","ASC","DESC"],"storeColumnNames":["value"],"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":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} +{"table":{"name":"job_info","id":53,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"job_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"info_key","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"written","id":3,"type":{"family":"TimestampTZFamily","oid":1184},"defaultExpr":"now():::TIMESTAMPTZ"},{"name":"value","id":4,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["job_id","info_key","written","value"],"columnIds":[1,2,3,4],"defaultColumnId":4}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["job_id","info_key","written"],"keyColumnDirections":["ASC","ASC","DESC"],"storeColumnNames":["value"],"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":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"jobs","id":15,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"status","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"created","id":3,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"payload","id":4,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"progress","id":5,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"created_by_type","id":6,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"created_by_id","id":7,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"claim_session_id","id":8,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"claim_instance_id","id":9,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"num_runs","id":10,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"last_run","id":11,"type":{"family":"TimestampFamily","oid":1114},"nullable":true},{"name":"job_type","id":12,"type":{"family":"StringFamily","oid":25},"nullable":true}],"nextColumnId":13,"families":[{"name":"fam_0_id_status_created_payload","columnNames":["id","status","created","payload","created_by_type","created_by_id","job_type"],"columnIds":[1,2,3,4,6,7,12]},{"name":"progress","id":1,"columnNames":["progress"],"columnIds":[5],"defaultColumnId":5},{"name":"claim","id":2,"columnNames":["claim_session_id","claim_instance_id","num_runs","last_run"],"columnIds":[8,9,10,11]}],"nextFamilyId":3,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["status","created","payload","progress","created_by_type","created_by_id","claim_session_id","claim_instance_id","num_runs","last_run","job_type"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6,7,8,9,10,11,12],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"jobs_status_created_idx","id":2,"version":3,"keyColumnNames":["status","created"],"keyColumnDirections":["ASC","ASC"],"keyColumnIds":[2,3],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"jobs_created_by_type_created_by_id_idx","id":3,"version":3,"keyColumnNames":["created_by_type","created_by_id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["status"],"keyColumnIds":[6,7],"keySuffixColumnIds":[1],"storeColumnIds":[2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"jobs_run_stats_idx","id":4,"version":3,"keyColumnNames":["claim_session_id","status","created"],"keyColumnDirections":["ASC","ASC","ASC"],"storeColumnNames":["last_run","num_runs","claim_instance_id"],"keyColumnIds":[8,2,3],"keySuffixColumnIds":[1],"storeColumnIds":[11,10,9],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"predicate":"status IN ('_':::STRING, '_':::STRING, '_':::STRING, '_':::STRING, '_':::STRING)"},{"name":"jobs_job_type_idx","id":5,"version":3,"keyColumnNames":["job_type"],"keyColumnDirections":["ASC"],"keyColumnIds":[12],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}}],"nextIndexId":6,"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":"join_tokens","id":41,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"UuidFamily","oid":2950}},{"name":"secret","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"expiration","id":3,"type":{"family":"TimestampTZFamily","oid":1184}}],"nextColumnId":4,"families":[{"name":"primary","columnNames":["id","secret","expiration"],"columnIds":[1,2,3]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["secret","expiration"],"keyColumnIds":[1],"storeColumnIds":[2,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":"lease","id":11,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"descID","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"version","id":2,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"nodeID","id":3,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"expiration","id":4,"type":{"family":"TimestampFamily","oid":1114}},{"name":"crdb_region","id":5,"type":{"family":"BytesFamily","oid":17}}],"nextColumnId":6,"families":[{"name":"primary","columnNames":["descID","version","nodeID","expiration","crdb_region"],"columnIds":[1,2,3,4,5]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":2,"unique":true,"version":4,"keyColumnNames":["crdb_region","descID","version","expiration","nodeID"],"keyColumnDirections":["ASC","ASC","ASC","ASC","ASC"],"keyColumnIds":[5,1,2,4,3],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"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":2}} @@ -612,7 +612,7 @@ schema_telemetry snapshot_id=7cd8a9ae-f35c-4cd2-970a-757174600874 max_records=10 ---- {"database":{"name":"defaultdb","id":100,"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":101}},"defaultPrivileges":{}}} {"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":"job_info","id":53,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"job_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"info_key","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"written","id":3,"type":{"family":"TimestampTZFamily","oid":1184},"defaultExpr":"now():::TIMESTAMPTZ"},{"name":"value","id":4,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["job_id","info_key","written","value"],"columnIds":[1,2,3,4],"defaultColumnId":4}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["job_id","info_key","written"],"keyColumnDirections":["ASC","ASC","DESC"],"storeColumnNames":["value"],"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":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} +{"table":{"name":"job_info","id":53,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"job_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"info_key","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"written","id":3,"type":{"family":"TimestampTZFamily","oid":1184},"defaultExpr":"now():::TIMESTAMPTZ"},{"name":"value","id":4,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["job_id","info_key","written","value"],"columnIds":[1,2,3,4],"defaultColumnId":4}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["job_id","info_key","written"],"keyColumnDirections":["ASC","ASC","DESC"],"storeColumnNames":["value"],"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":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"replication_stats","id":27,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"zone_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"subzone_id","id":2,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"report_id","id":3,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"total_ranges","id":4,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"unavailable_ranges","id":5,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"under_replicated_ranges","id":6,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"over_replicated_ranges","id":7,"type":{"family":"IntFamily","width":64,"oid":20}}],"nextColumnId":8,"families":[{"name":"primary","columnNames":["zone_id","subzone_id","report_id","total_ranges","unavailable_ranges","under_replicated_ranges","over_replicated_ranges"],"columnIds":[1,2,3,4,5,6,7]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["zone_id","subzone_id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["report_id","total_ranges","unavailable_ranges","under_replicated_ranges","over_replicated_ranges"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5,6,7],"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":"role_members","id":23,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"role","id":1,"type":{"family":"StringFamily","oid":25}},{"name":"member","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"isAdmin","id":3,"type":{"oid":16}},{"name":"role_id","id":4,"type":{"family":"OidFamily","oid":26}},{"name":"member_id","id":5,"type":{"family":"OidFamily","oid":26}}],"nextColumnId":6,"families":[{"name":"primary","columnNames":["role","member"],"columnIds":[1,2]},{"name":"fam_3_isAdmin","id":3,"columnNames":["isAdmin"],"columnIds":[3],"defaultColumnId":3},{"name":"fam_4_role_id","id":4,"columnNames":["role_id"],"columnIds":[4],"defaultColumnId":4},{"name":"fam_5_member_id","id":5,"columnNames":["member_id"],"columnIds":[5],"defaultColumnId":5}],"nextFamilyId":6,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["role","member"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["isAdmin","role_id","member_id"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":2},"indexes":[{"name":"role_members_role_idx","id":2,"version":3,"keyColumnNames":["role"],"keyColumnDirections":["ASC"],"keyColumnIds":[1],"keySuffixColumnIds":[2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"role_members_member_idx","id":3,"version":3,"keyColumnNames":["member"],"keyColumnDirections":["ASC"],"keyColumnIds":[2],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"role_members_role_id_idx","id":4,"version":3,"keyColumnNames":["role_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[4],"keySuffixColumnIds":[1,2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"role_members_member_id_idx","id":5,"version":3,"keyColumnNames":["member_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[5],"keySuffixColumnIds":[1,2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"role_members_role_id_member_id_key","id":6,"unique":true,"version":3,"keyColumnNames":["role_id","member_id"],"keyColumnDirections":["ASC","ASC"],"keyColumnIds":[4,5],"keySuffixColumnIds":[1,2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"constraintId":1}],"nextIndexId":7,"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":"span_stats_buckets","id":55,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"UuidFamily","oid":2950},"defaultExpr":"gen_random_uuid()"},{"name":"sample_id","id":2,"type":{"family":"UuidFamily","oid":2950}},{"name":"start_key_id","id":3,"type":{"family":"UuidFamily","oid":2950}},{"name":"end_key_id","id":4,"type":{"family":"UuidFamily","oid":2950}},{"name":"requests","id":5,"type":{"family":"IntFamily","width":64,"oid":20}}],"nextColumnId":6,"families":[{"name":"primary","columnNames":["id","sample_id","start_key_id","end_key_id","requests"],"columnIds":[1,2,3,4,5]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["sample_id","start_key_id","end_key_id","requests"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"buckets_sample_id_idx","id":2,"version":3,"keyColumnNames":["sample_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[2],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}}],"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":2}} @@ -624,7 +624,7 @@ schema_telemetry snapshot_id=7cd8a9ae-f35c-4cd2-970a-757174600874 max_records=10 ---- {"database":{"name":"defaultdb","id":100,"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":101}},"defaultPrivileges":{}}} {"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":"job_info","id":53,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"job_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"info_key","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"written","id":3,"type":{"family":"TimestampTZFamily","oid":1184},"defaultExpr":"now():::TIMESTAMPTZ"},{"name":"value","id":4,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["job_id","info_key","written","value"],"columnIds":[1,2,3,4],"defaultColumnId":4}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["job_id","info_key","written"],"keyColumnDirections":["ASC","ASC","DESC"],"storeColumnNames":["value"],"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":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} +{"table":{"name":"job_info","id":53,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"job_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"info_key","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"written","id":3,"type":{"family":"TimestampTZFamily","oid":1184},"defaultExpr":"now():::TIMESTAMPTZ"},{"name":"value","id":4,"type":{"family":"BytesFamily","oid":17},"nullable":true}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["job_id","info_key","written","value"],"columnIds":[1,2,3,4],"defaultColumnId":4}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["job_id","info_key","written"],"keyColumnDirections":["ASC","ASC","DESC"],"storeColumnNames":["value"],"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":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"replication_stats","id":27,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"zone_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"subzone_id","id":2,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"report_id","id":3,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"total_ranges","id":4,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"unavailable_ranges","id":5,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"under_replicated_ranges","id":6,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"over_replicated_ranges","id":7,"type":{"family":"IntFamily","width":64,"oid":20}}],"nextColumnId":8,"families":[{"name":"primary","columnNames":["zone_id","subzone_id","report_id","total_ranges","unavailable_ranges","under_replicated_ranges","over_replicated_ranges"],"columnIds":[1,2,3,4,5,6,7]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["zone_id","subzone_id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["report_id","total_ranges","unavailable_ranges","under_replicated_ranges","over_replicated_ranges"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5,6,7],"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":"role_members","id":23,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"role","id":1,"type":{"family":"StringFamily","oid":25}},{"name":"member","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"isAdmin","id":3,"type":{"oid":16}},{"name":"role_id","id":4,"type":{"family":"OidFamily","oid":26}},{"name":"member_id","id":5,"type":{"family":"OidFamily","oid":26}}],"nextColumnId":6,"families":[{"name":"primary","columnNames":["role","member"],"columnIds":[1,2]},{"name":"fam_3_isAdmin","id":3,"columnNames":["isAdmin"],"columnIds":[3],"defaultColumnId":3},{"name":"fam_4_role_id","id":4,"columnNames":["role_id"],"columnIds":[4],"defaultColumnId":4},{"name":"fam_5_member_id","id":5,"columnNames":["member_id"],"columnIds":[5],"defaultColumnId":5}],"nextFamilyId":6,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["role","member"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["isAdmin","role_id","member_id"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":2},"indexes":[{"name":"role_members_role_idx","id":2,"version":3,"keyColumnNames":["role"],"keyColumnDirections":["ASC"],"keyColumnIds":[1],"keySuffixColumnIds":[2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"role_members_member_idx","id":3,"version":3,"keyColumnNames":["member"],"keyColumnDirections":["ASC"],"keyColumnIds":[2],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"role_members_role_id_idx","id":4,"version":3,"keyColumnNames":["role_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[4],"keySuffixColumnIds":[1,2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"role_members_member_id_idx","id":5,"version":3,"keyColumnNames":["member_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[5],"keySuffixColumnIds":[1,2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}},{"name":"role_members_role_id_member_id_key","id":6,"unique":true,"version":3,"keyColumnNames":["role_id","member_id"],"keyColumnDirections":["ASC","ASC"],"keyColumnIds":[4,5],"keySuffixColumnIds":[1,2],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"constraintId":1}],"nextIndexId":7,"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":"span_stats_buckets","id":55,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"UuidFamily","oid":2950},"defaultExpr":"gen_random_uuid()"},{"name":"sample_id","id":2,"type":{"family":"UuidFamily","oid":2950}},{"name":"start_key_id","id":3,"type":{"family":"UuidFamily","oid":2950}},{"name":"end_key_id","id":4,"type":{"family":"UuidFamily","oid":2950}},{"name":"requests","id":5,"type":{"family":"IntFamily","width":64,"oid":20}}],"nextColumnId":6,"families":[{"name":"primary","columnNames":["id","sample_id","start_key_id","end_key_id","requests"],"columnIds":[1,2,3,4,5]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["sample_id","start_key_id","end_key_id","requests"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"indexes":[{"name":"buckets_sample_id_idx","id":2,"version":3,"keyColumnNames":["sample_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[2],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}}],"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":2}} diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 17fc3be4519a..1c482ebe572e 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -894,8 +894,8 @@ const ( // found. systemJobsAndJobInfoBaseQuery = ` WITH - latestpayload AS (SELECT job_id, value FROM system.job_info AS payload WHERE info_key = 'legacy_payload'::BYTES), - latestprogress AS (SELECT job_id, value FROM system.job_info AS progress WHERE info_key = 'legacy_progress'::BYTES) + latestpayload AS (SELECT job_id, value FROM system.job_info AS payload WHERE info_key = 'legacy_payload'), + latestprogress AS (SELECT job_id, value FROM system.job_info AS progress WHERE info_key = 'legacy_progress') SELECT id, status, created, payload.value AS payload, progress.value AS progress, created_by_type, created_by_id, claim_session_id, claim_instance_id, num_runs, last_run, job_type diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go index cb19c9d0f0b6..7c5e69519aa8 100644 --- a/pkg/sql/drop_database.go +++ b/pkg/sql/drop_database.go @@ -253,8 +253,9 @@ func (p *planner) accumulateAllObjectsToDelete( ctx context.Context, objects []toDelete, ) ([]*tabledesc.Mutable, map[descpb.ID]*tabledesc.Mutable, error) { implicitDeleteObjects := make(map[descpb.ID]*tabledesc.Mutable) + visited := make(map[descpb.ID]struct{}) for _, toDel := range objects { - err := p.accumulateCascadingViews(ctx, implicitDeleteObjects, toDel.desc) + err := p.accumulateCascadingViews(ctx, implicitDeleteObjects, visited, toDel.desc) if err != nil { return nil, nil, err } @@ -311,8 +312,12 @@ func (p *planner) accumulateOwnedSequences( // references, which means this list can't be constructed by simply scanning // the namespace table. func (p *planner) accumulateCascadingViews( - ctx context.Context, dependentObjects map[descpb.ID]*tabledesc.Mutable, desc *tabledesc.Mutable, + ctx context.Context, + dependentObjects map[descpb.ID]*tabledesc.Mutable, + visited map[descpb.ID]struct{}, + desc *tabledesc.Mutable, ) error { + visited[desc.ID] = struct{}{} for _, ref := range desc.DependedOnBy { desc, err := p.Descriptors().MutableByID(p.txn).Desc(ctx, ref.ID) if err != nil { @@ -327,8 +332,15 @@ func (p *planner) accumulateCascadingViews( if !dependentDesc.IsView() { continue } + + _, seen := visited[ref.ID] + if dependentObjects[ref.ID] == dependentDesc || seen { + // This view's dependencies are already added. + continue + } dependentObjects[ref.ID] = dependentDesc - if err := p.accumulateCascadingViews(ctx, dependentObjects, dependentDesc); err != nil { + + if err := p.accumulateCascadingViews(ctx, dependentObjects, visited, dependentDesc); err != nil { return err } } diff --git a/pkg/sql/drop_view.go b/pkg/sql/drop_view.go index 8146bd68aec6..1a3d2a897b94 100644 --- a/pkg/sql/drop_view.go +++ b/pkg/sql/drop_view.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scerrors" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -170,6 +171,18 @@ func (p *planner) canRemoveDependentFromTable( ref descpb.TableDescriptor_Reference, behavior tree.DropBehavior, ) error { + if p.trackDependency == nil { + p.trackDependency = make(map[catid.DescID]bool) + } + if p.trackDependency[ref.ID] { + // This table's dependencies are already tracked. + return nil + } + p.trackDependency[ref.ID] = true + defer func() { + p.trackDependency[ref.ID] = false + }() + return p.canRemoveDependent(ctx, string(from.DescriptorType()), from.Name, from.ParentID, ref, behavior) } diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index 7c118ee11401..2aa1bb5192d8 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -3184,3 +3184,13 @@ INSERT INTO t_98269 VALUES (0); statement error pgcode 0A000 .* cluster_logical_timestamp\(\): nil txn in cluster context ALTER TABLE t_98269 ADD COLUMN j DECIMAL NOT NULL DEFAULT cluster_logical_timestamp(); + +# In #99185, we saw test failures that result from adding a foreign key +# constraint to a non-existent table with IF EXISTS because we require all tables +# in the stmt should be marked as non-existent or fully resolved. We didn't do +# anything to the referenced table name, so the validation logic complained that +# referenced table name is not fully resolved nor marked as non-existent. +subtest alter_non_existent_table_with_if_exists + +statement ok +ALTER TABLE IF EXISTS t_non_existent_99185 ADD FOREIGN KEY (i) REFERENCES t_other_99185 (i); diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index 9531fcc613e5..6e7e88f163e0 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -146,7 +146,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 50 {"table": {"columns": [{"id": 1, "name": "tenant_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "value", "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "now():::TIMESTAMP", "id": 4, "name": "last_updated", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 5, "name": "value_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "reason", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 50, "name": "tenant_settings", "nextColumnId": 7, "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", "name"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 4, 5, 6], "storeColumnNames": ["value", "last_updated", "value_type", "reason"], "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"}} 51 {"table": {"columns": [{"id": 1, "name": "username", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "path", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privileges", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 4, "name": "grant_options", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "user_id", "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 51, "indexes": [{"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [2, 5], "keyColumnNames": ["path", "user_id"], "keySuffixColumnIds": [1], "name": "privileges_path_user_id_key", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 4], "storeColumnNames": ["privileges", "grant_options"], "unique": true, "version": 3}, {"constraintId": 2, "foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [2, 1], "keyColumnNames": ["path", "username"], "name": "privileges_path_username_key", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 4], "storeColumnNames": ["privileges", "grant_options"], "unique": true, "version": 3}], "name": "privileges", "nextColumnId": 6, "nextConstraintId": 4, "nextIndexId": 4, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 3, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 2], "keyColumnNames": ["username", "path"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [3, 4, 5], "storeColumnNames": ["privileges", "grant_options", "user_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"}} 52 {"table": {"columns": [{"id": 1, "name": "connection_name", "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "now():::TIMESTAMP", "id": 2, "name": "created", "type": {"family": "TimestampFamily", "oid": 1114}}, {"defaultExpr": "now():::TIMESTAMP", "id": 3, "name": "updated", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 4, "name": "connection_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "connection_details", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "owner", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "owner_id", "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 52, "name": "external_connections", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["connection_name"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7], "storeColumnNames": ["created", "updated", "connection_type", "connection_details", "owner", "owner_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"}} -53 {"table": {"columns": [{"id": 1, "name": "job_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "info_key", "type": {"family": "BytesFamily", "oid": 17}}, {"defaultExpr": "now():::TIMESTAMPTZ", "id": 3, "name": "written", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "value", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 53, "name": "job_info", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC", "DESC"], "keyColumnIds": [1, 2, 3], "keyColumnNames": ["job_id", "info_key", "written"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [4], "storeColumnNames": ["value"], "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"}} +53 {"table": {"columns": [{"id": 1, "name": "job_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "info_key", "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "now():::TIMESTAMPTZ", "id": 3, "name": "written", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "value", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 53, "name": "job_info", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC", "DESC"], "keyColumnIds": [1, 2, 3], "keyColumnNames": ["job_id", "info_key", "written"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [4], "storeColumnNames": ["value"], "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"}} 54 {"table": {"columns": [{"defaultExpr": "gen_random_uuid()", "id": 1, "name": "id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "key_bytes", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 54, "indexes": [{"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["key_bytes"], "keySuffixColumnIds": [1], "name": "unique_keys_key_bytes_idx", "partitioning": {}, "sharded": {}, "unique": true, "version": 3}], "name": "span_stats_unique_keys", "nextColumnId": 3, "nextConstraintId": 3, "nextIndexId": 3, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 2, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["key_bytes"], "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"}} 55 {"table": {"columns": [{"defaultExpr": "gen_random_uuid()", "id": 1, "name": "id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 2, "name": "sample_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 3, "name": "start_key_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 4, "name": "end_key_id", "type": {"family": "UuidFamily", "oid": 2950}}, {"id": 5, "name": "requests", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 55, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["sample_id"], "keySuffixColumnIds": [1], "name": "buckets_sample_id_idx", "partitioning": {}, "sharded": {}, "version": 3}], "name": "span_stats_buckets", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "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": ["sample_id", "start_key_id", "end_key_id", "requests"], "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"}} 56 {"table": {"columns": [{"defaultExpr": "gen_random_uuid()", "id": 1, "name": "id", "type": {"family": "UuidFamily", "oid": 2950}}, {"defaultExpr": "now():::TIMESTAMP", "id": 2, "name": "sample_time", "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 56, "indexes": [{"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["sample_time"], "keySuffixColumnIds": [1], "name": "samples_sample_time_idx", "partitioning": {}, "sharded": {}, "unique": true, "version": 3}], "name": "span_stats_samples", "nextColumnId": 3, "nextConstraintId": 3, "nextIndexId": 3, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 2, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["sample_time"], "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/create_index b/pkg/sql/logictest/testdata/logic_test/create_index index 5b03c4fb1528..51ca7755fe70 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_index +++ b/pkg/sql/logictest/testdata/logic_test/create_index @@ -338,7 +338,7 @@ UPDATE system.job_info '{"resumeSpans": $spans}'::jsonb ) ) -WHERE info_key = 'legacy_payload'::BYTES AND crdb_internal.pb_to_json('cockroach.sql.jobs.jobspb.Payload', value)->>'description' LIKE 'CREATE INDEX pauseidx%'; +WHERE info_key = 'legacy_payload' AND crdb_internal.pb_to_json('cockroach.sql.jobs.jobspb.Payload', value)->>'description' LIKE 'CREATE INDEX pauseidx%'; # confirm we see these bogus start and end keys in the job, both for the wrong # tenant and for no tenant. diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index b5f7840e7ea4..6b4a9de4d31e 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1165,7 +1165,7 @@ indexrelid indrelid indnatts indisunique indnullsnotdistinct indisprimary 1841972634 6 1 true false true false true false true false false true false 1 3403232968 0 2 NULL NULL 1 2008917577 37 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1 2008917578 37 1 false false false false false false true false false true false 5 0 0 2 NULL NULL 1 -2055313241 53 3 true false true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 1 NULL NULL 3 +2055313241 53 3 true false true false true false true false false true false 1 2 3 0 3403232968 0 0 0 0 2 2 1 NULL NULL 3 2101708905 5 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1 2148104569 21 2 true false true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL 2 2268653844 40 4 true false true false true false true false false true false 1 2 3 4 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 4 diff --git a/pkg/sql/logictest/testdata/logic_test/views b/pkg/sql/logictest/testdata/logic_test/views index 0aec342b2e4b..f9c28f4dd512 100644 --- a/pkg/sql/logictest/testdata/logic_test/views +++ b/pkg/sql/logictest/testdata/logic_test/views @@ -22,12 +22,12 @@ CREATE VIEW v1 AS SELECT a, b FROM t statement error pgcode 42P07 relation \"test.public.t\" already exists CREATE VIEW t AS SELECT a, b FROM t -# view statement ignored if other way around. statement ok -CREATE VIEW IF NOT EXISTS v1 AS SELECT b, a FROM v1 +CREATE VIEW IF NOT EXISTS v2 (x, y) AS SELECT a, b FROM t +# view statement ignored if other way around. statement ok -CREATE VIEW IF NOT EXISTS v2 (x, y) AS SELECT a, b FROM t +CREATE VIEW IF NOT EXISTS v2 AS SELECT b, a FROM v1 statement error pgcode 42601 CREATE VIEW specifies 1 column name, but data source has 2 columns CREATE VIEW v3 (x) AS SELECT a, b FROM t @@ -1835,3 +1835,37 @@ CREATE VIEW pg_comedies AS statement error pq: cannot drop columns from view CREATE OR REPLACE VIEW comedies AS SELECT ARRAY[films.*]::string FROM films; + +subtest circular_dependency + +statement ok +CREATE TABLE t (a INT PRIMARY KEY, b INT); + +statement ok +CREATE VIEW cd_v1 AS SELECT a, b FROM t; + +statement ok +CREATE VIEW cd_v2 AS SELECT a, b FROM cd_v1; + +# Note: Creating a circular dependency in views does not result in an error in +# postgres. In postgres, we only encounter errors during queries on the views. +statement error pgcode 42P17 pq: cyclic view dependency for relation test.public.cd_v1 +CREATE OR REPLACE VIEW cd_v1 AS SELECT a, b FROM cd_v2; + +statement ok +CREATE VIEW cd_v3 AS SELECT a, b FROM cd_v2; + +statement ok +SELECT * FROM cd_v3; + +statement error pgcode 42P17 pq: cyclic view dependency for relation test.public.cd_v1 +CREATE OR REPLACE VIEW cd_v1 AS SELECT a, b FROM cd_v3; + +statement ok +SELECT * FROM cd_v3; + +statement error pq: cannot drop relation "cd_v1" because view "cd_v2" depends on it +DROP VIEW cd_v1; + +statement ok +DROP VIEW cd_v1 CASCADE; diff --git a/pkg/sql/opt/optbuilder/builder.go b/pkg/sql/opt/optbuilder/builder.go index 207f60ab45af..483ffcf9c8b8 100644 --- a/pkg/sql/opt/optbuilder/builder.go +++ b/pkg/sql/opt/optbuilder/builder.go @@ -113,6 +113,10 @@ type Builder struct { // are referenced multiple times in the same query. views map[cat.View]*tree.Select + // sourceViews contains a map with all the views in the current data source + // chain. It is used to detect circular dependencies. + sourceViews map[string]struct{} + // subquery contains a pointer to the subquery which is currently being built // (if any). subquery *subquery diff --git a/pkg/sql/opt/optbuilder/create_view.go b/pkg/sql/opt/optbuilder/create_view.go index b5b508d4cbe9..7d217bc1cdae 100644 --- a/pkg/sql/opt/optbuilder/create_view.go +++ b/pkg/sql/opt/optbuilder/create_view.go @@ -36,12 +36,17 @@ func (b *Builder) buildCreateView(cv *tree.CreateView, inScope *scope) (outScope b.insideViewDef = true b.trackSchemaDeps = true b.qualifyDataSourceNamesInAST = true + if b.sourceViews == nil { + b.sourceViews = make(map[string]struct{}) + } + b.sourceViews[viewName.FQString()] = struct{}{} defer func() { b.insideViewDef = false b.trackSchemaDeps = false b.schemaDeps = nil b.schemaTypeDeps = intsets.Fast{} b.qualifyDataSourceNamesInAST = false + delete(b.sourceViews, viewName.FQString()) b.semaCtx.FunctionResolver = preFuncResolver switch recErr := recover().(type) { diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index 8af723363358..0325ce381ad0 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -280,6 +280,17 @@ func (b *Builder) buildDataSource( func (b *Builder) buildView( view cat.View, viewName *tree.TableName, locking lockingSpec, inScope *scope, ) (outScope *scope) { + if b.sourceViews == nil { + b.sourceViews = make(map[string]struct{}) + } + // Check whether there is a circular dependency between views. + if _, ok := b.sourceViews[viewName.FQString()]; ok { + panic(pgerror.Newf(pgcode.InvalidObjectDefinition, "cyclic view dependency for relation %s", viewName)) + } + b.sourceViews[viewName.FQString()] = struct{}{} + defer func() { + delete(b.sourceViews, viewName.FQString()) + }() // Cache the AST so that multiple references won't need to reparse. if b.views == nil { b.views = make(map[cat.View]*tree.Select) diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index a41eb6081a21..dbe81ea5cbf9 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/querycache" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/transform" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -262,6 +263,9 @@ type planner struct { // evalCatalogBuiltins is used as part of the eval.Context. evalCatalogBuiltins evalcatalog.Builtins + + // trackDependency is used to track circular dependencies when dropping views. + trackDependency map[catid.DescID]bool } // hasFlowForPausablePortal returns true if the planner is for re-executing a diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go index c7357fadee60..ecc6d0683f51 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go @@ -146,7 +146,12 @@ func AlterTable(b BuildCtx, n *tree.AlterTable) { }) _, target, tbl := scpb.FindTable(elts) if tbl == nil { - b.MarkNameAsNonExistent(&tn) + // Mark all table names (`tn` and others) in this ALTER TABLE stmt as non-existent. + tree.NewFmtCtx(tree.FmtSimple, tree.FmtReformatTableNames(func( + ctx *tree.FmtCtx, name *tree.TableName, + ) { + b.MarkNameAsNonExistent(name) + })).FormatNode(n) return } if target != scpb.ToPublic { diff --git a/pkg/sql/sem/tree/format.go b/pkg/sql/sem/tree/format.go index 3e079220e133..4cdca9090c65 100644 --- a/pkg/sql/sem/tree/format.go +++ b/pkg/sql/sem/tree/format.go @@ -306,7 +306,7 @@ func FmtPlaceholderFormat(placeholderFn func(_ *FmtCtx, _ *Placeholder)) FmtCtxO } } -// FmtReformatTableNames modifies FmtCtx to to substitute the printing of table +// FmtReformatTableNames modifies FmtCtx to substitute the printing of table // naFmtParsable using the provided function. func FmtReformatTableNames(tableNameFmt func(*FmtCtx, *TableName)) FmtCtxOption { return func(ctx *FmtCtx) { diff --git a/pkg/storage/.gitignore b/pkg/storage/.gitignore index 301b8d2260fa..bab33bc2452d 100644 --- a/pkg/storage/.gitignore +++ b/pkg/storage/.gitignore @@ -1,7 +1,6 @@ # Do not add environment-specific entries here (see the top-level .gitignore # for reasoning and alternatives). # -# Old benchmark data: +# Old benchmark data. mvcc_data_* -# New benchmark data: testdata/initial diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index 555785de5a3b..23e5bdf9ed77 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -166,6 +166,7 @@ go_test( "//pkg/testutils/datapathutils", "//pkg/testutils/echotest", "//pkg/testutils/skip", + "//pkg/testutils/testfixtures", "//pkg/testutils/zerofields", "//pkg/util", "//pkg/util/admission", diff --git a/pkg/storage/bench_data_test.go b/pkg/storage/bench_data_test.go index 5a4f8a7aa412..5f8f21949e90 100644 --- a/pkg/storage/bench_data_test.go +++ b/pkg/storage/bench_data_test.go @@ -16,16 +16,17 @@ import ( "math/rand" "os" "path/filepath" + "strings" "testing" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testfixtures" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" - "github.com/cockroachdb/errors/oserror" "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/vfs" "github.com/stretchr/testify/require" @@ -67,73 +68,45 @@ var latestReleaseFormatMajorVersionOpt ConfigOption = func(cfg *engineConfig) er return nil } -// getInitialStateEngine constructs an Engine with an initial database -// state necessary for a benchmark. The initial states are cached on the -// filesystem to avoid expensive reconstruction when possible. The return value -// of Key() must be unique for each unique initial database configuration, -// because the Key() value is used to key cached intial databases. +// getInitialStateEngine constructs an Engine with an initial database state +// necessary for a benchmark. The initial states are cached on the filesystem to +// avoid expensive reconstruction when possible (see +// testfixtures.ReuseOrGenerate). // -// TODO(jackson): Initial states are NOT cached across ./dev bench invocations, -// because the initial states are written to the temporary bazel sandbox and not -// copied out. See #83599. +// The return value of initial.Key() must be unique for each unique initial database +// configuration, because the Key() value is used to key cached initial databases. func getInitialStateEngine( ctx context.Context, b *testing.B, initial initialState, inMemory bool, ) engineWithLocation { - const initialStatesDir = `testdata/initial` - - require.NoError(b, os.MkdirAll(initialStatesDir, os.ModePerm)) - dir := filepath.Join(append([]string{initialStatesDir}, initial.Key()...)...) + name := strings.Join(initial.Key(), "-") + dir := testfixtures.ReuseOrGenerate(b, name, func(dir string) { + buildInitialState(ctx, b, initial, dir) + }) dataDir := filepath.Join(dir, "data") - completedFile := filepath.Join(dir, "completed") - - var buildFS vfs.FS - if _, err := os.Stat(completedFile); oserror.IsNotExist(err) { - // There's no completed existing engine state for these initial - // condtions. Produce it. - b.Logf("%q does not exist; building initial state first", completedFile) - buildFS = buildInitialState(ctx, b, initial, dir) - } else if err != nil { - b.Fatal(err) - } opts := append([]ConfigOption{ MustExist, latestReleaseFormatMajorVersionOpt, }, initial.ConfigOptions()...) - if !inMemory { - // The callers wants a durable engine. Copy the seed data to a temporary - // directory on the filesystem. - testRunDir := b.TempDir() - ok, err := vfs.Clone(vfs.Default, vfs.Default, dataDir, testRunDir, vfs.CloneSync) - require.NoError(b, err) - require.True(b, ok) - - // Load all the files into the OS buffer cache for better determinism. - testutils.ReadAllFiles(filepath.Join(testRunDir, "*")) - - loc := Filesystem(testRunDir) - e, err := Open(ctx, loc, cluster.MakeClusterSettings(), opts...) - require.NoError(b, err) - return engineWithLocation{Engine: e, Location: loc} + var loc Location + if inMemory { + loc = InMemory() + } else { + // The caller wants a durable engine; use a temp directory. + loc = Filesystem(b.TempDir()) } - var fs vfs.FS + // We now copy the initial state to the desired FS. + ok, err := vfs.Clone(vfs.Default, loc.fs, dataDir, loc.dir, vfs.CloneSync) + require.NoError(b, err) + require.True(b, ok) - // If the caller requests an in-memory engine and we just built the initial - // state using an in-memory filesystem, use the existing filesystem already - // ready. - if buildFS != nil { - fs = buildFS - } else { - // Load the initial state off the filesystem. - fs = vfs.NewMem() - ok, err := vfs.Clone(vfs.Default, fs, dataDir, "") - require.NoError(b, err) - require.True(b, ok) + if !inMemory { + // Load all the files into the OS buffer cache for better determinism. + testutils.ReadAllFiles(filepath.Join(loc.dir, "*")) } - loc := Location{fs: fs} e, err := Open(ctx, loc, cluster.MakeClusterSettings(), opts...) require.NoError(b, err) return engineWithLocation{Engine: e, Location: loc} @@ -179,11 +152,6 @@ func buildInitialState( require.NoError(b, err) require.True(b, ok) - // Create a marker file signalling that the persisted state is complete. - f, err := vfs.Default.Create(filepath.Join(dir, "completed")) - require.NoError(b, err) - require.NoError(b, f.Close()) - return buildFS } diff --git a/pkg/testutils/localtestcluster/BUILD.bazel b/pkg/testutils/localtestcluster/BUILD.bazel index d16746fc458d..037d6d1467d2 100644 --- a/pkg/testutils/localtestcluster/BUILD.bazel +++ b/pkg/testutils/localtestcluster/BUILD.bazel @@ -24,6 +24,7 @@ go_library( "//pkg/rpc", "//pkg/server/systemconfigwatcher", "//pkg/settings/cluster", + "//pkg/spanconfig/spanconfigbounds", "//pkg/spanconfig/spanconfigkvsubscriber", "//pkg/sql/catalog/bootstrap", "//pkg/storage", diff --git a/pkg/testutils/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index 7d424d1d2b26..10a30c06b17a 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigbounds" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvsubscriber" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" "github.com/cockroachdb/cockroach/pkg/storage" @@ -230,8 +231,9 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto 1<<20, /* 1 MB */ cfg.DefaultSpanConfig, cfg.Settings, - nil, - nil, + spanconfigbounds.NewEmptyReader(), + nil, /* knobs */ + nil, /* registry */ ) cfg.SystemConfigProvider = systemconfigwatcher.New( keys.SystemSQLCodec, diff --git a/pkg/testutils/testfixtures/BUILD.bazel b/pkg/testutils/testfixtures/BUILD.bazel new file mode 100644 index 000000000000..37d976236c5b --- /dev/null +++ b/pkg/testutils/testfixtures/BUILD.bazel @@ -0,0 +1,16 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "testfixtures", + srcs = ["test_fixtures.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/testutils/testfixtures", + visibility = ["//visibility:public"], + deps = [ + "//pkg/util/envutil", + "//pkg/util/syncutil", + "@com_github_cockroachdb_errors//oserror", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/testutils/testfixtures/test_fixtures.go b/pkg/testutils/testfixtures/test_fixtures.go new file mode 100644 index 000000000000..e206846a65ab --- /dev/null +++ b/pkg/testutils/testfixtures/test_fixtures.go @@ -0,0 +1,134 @@ +// 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 testfixtures + +import ( + "os" + "path/filepath" + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/envutil" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors/oserror" +) + +const baseDirEnv = "COCKROACH_TEST_FIXTURES_DIR" +const subdir = "crdb-test-fixtures" + +// ReuseOrGenerate is used for fixtures that can be reused across invocations of tests or +// benchmarks (or across different tests / benchmarks). +// +// A fixture corresponds to a directory. If a fixture with the given name was +// already generated, ReuseOrGenerate just returns the directory. +// +// Otherwise, generate() is called with the directory; upon completion +// ReuseOrGenerate returns the same directory. +// +// The directory is normally $HOME/.cache/crdb-text-fixtures/ +// +// The base directory can be customized via the `COCKROACH_TEST_FIXTURES_DIR` +// env var (which is set to $HOME/.cache/crdb-test-fixtures when using `dev`). +// If this variable is not specified and $HOME is not set, a temporary directory +// is used. +// +// ReuseOrGenerate gracefully handles interruptions / generation errors: we only +// declare a fixture as reusable once the generate() function returns. +// ReuseOrGenerate allows multiple calls in parallel with the same fixture name. +// +// ReuseOrGenerate does not support parallel invocations (with the same fixture +// name) in different processes. +// +// Note: currently this function is only used for benchmarks and `dev` only sets +// COCKROACH_TEST_FIXTURES_DIR for benchmarks. If we start using it for regular tests, +// `dev` should be updated to set it for tests as well. +func ReuseOrGenerate(tb testing.TB, name string, generate func(dir string)) (dir string) { + // If the same fixture name is in the in-progress map, wait. + for first := true; ; first = false { + mu.Lock() + ch, ok := mu.inProgress[name] + if !ok { + mu.inProgress[name] = make(chan struct{}) + mu.Unlock() + break + } + mu.Unlock() + if first { + tb.Logf("waiting for fixture %q", name) + } + // Wait for channel close. + <-ch + } + + defer func() { + mu.Lock() + defer mu.Unlock() + close(mu.inProgress[name]) + delete(mu.inProgress, name) + }() + + baseDir := envutil.EnvOrDefaultString(baseDirEnv, "") + if baseDir == "" { + if cacheDir, err := os.UserCacheDir(); err == nil { + baseDir = filepath.Join(cacheDir, subdir) + } else { + baseDir = filepath.Join(os.TempDir(), subdir) + } + } + dir = filepath.Join(baseDir, name) + const completedFile = ".crdb-test-fixture-completed" + completedPath := filepath.Join(dir, completedFile) + if exists(tb, dir, true /* expectDir */) { + if exists(tb, completedPath, false /* expectDir */) { + tb.Logf("using existing fixture %q in %q", name, dir) + return dir + } + // Directory exists but fixture was not completed; clean up. + if err := os.RemoveAll(dir); err != nil { + tb.Fatal(err) + } + } + if err := os.MkdirAll(dir, 0755); err != nil { + tb.Fatal(err) + } + tb.Logf("generating fixture %q in %q", name, dir) + generate(dir) + if err := os.WriteFile(completedPath, []byte("foo"), 0755); err != nil { + tb.Fatal(err) + } + tb.Logf("successfully generated fixture %q in %q", name, dir) + return dir +} + +type inProgressMap struct { + syncutil.Mutex + inProgress map[string]chan struct{} +} + +var mu = inProgressMap{ + inProgress: make(map[string]chan struct{}), +} + +// exists returns true if the file or directory exists. If expectDir is true, it +// also asserts that the path is a directory. +func exists(tb testing.TB, path string, expectDir bool) bool { + tb.Helper() + stat, err := os.Stat(path) + if err != nil { + if oserror.IsNotExist(err) { + return false + } + tb.Fatal(err) + } + if expectDir && !stat.IsDir() { + tb.Fatalf("%q exists but is not directory", path) + } + return true +}