diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 3bcbfb96a06e..fb184e62743f 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -32,7 +32,7 @@ ALL_TESTS = [ "//pkg/ccl/serverccl/diagnosticsccl:diagnosticsccl_test", "//pkg/ccl/serverccl/statusccl:statusccl_test", "//pkg/ccl/serverccl:serverccl_test", - "//pkg/ccl/spanconfigccl:spanconfigccl_test", + "//pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl:spanconfigsqltranslatorccl_test", "//pkg/ccl/sqlproxyccl/denylist:denylist_test", "//pkg/ccl/sqlproxyccl/idle:idle_test", "//pkg/ccl/sqlproxyccl/tenant:tenant_test", diff --git a/pkg/ccl/spanconfigccl/datadriven_test.go b/pkg/ccl/spanconfigccl/datadriven_test.go deleted file mode 100644 index 1fe55568f0f7..000000000000 --- a/pkg/ccl/spanconfigccl/datadriven_test.go +++ /dev/null @@ -1,269 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Licensed as a CockroachDB Enterprise file under the Cockroach Community -// License (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt - -package spanconfigccl_test - -import ( - "context" - "fmt" - "reflect" - "strings" - "testing" - "time" - - "github.com/cockroachdb/cockroach/pkg/base" - _ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl" - "github.com/cockroachdb/cockroach/pkg/config/zonepb" - "github.com/cockroachdb/cockroach/pkg/kv" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/spanconfig" - "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/datadriven" - "github.com/stretchr/testify/require" -) - -// TestSQLTranslatorDataDriven is a data-driven test for the -// spanconfigsqltranslator.SQLTranslator. It allows users to set up zone config -// hierarchies and validate their translation to SpanConfigs is as expected. -// Only fields that are different from the (default) RANGE DEFAULT are printed -// in the test output for readability. -// -// It offers the following commands: -// -// "exec-sql": executes the input SQL query. -// -// "query-sql": executes the input SQL query and prints the results. -// -// "translate [database=] [table=] [named-zone=] -// [id=]: -// translates the SQL zone config state to the span config state starting from -// the referenced object (named zone, database, database + table, or descriptor -// id) as the root. -// -// "full-translate": performs a full translation of the SQL zone config state -// to the implied span config state. -// -// "sleep" [duration=]: sleep for the provided duration. -// -// "mark-table-offline" [database=] [table=]: marks the given -// table as offline for testing purposes. -// -// "mark-table-public" [database=] [table=]: marks the given -// table as public. -// -// TODO(arul): Add a secondary tenant configuration for this test as well. -func TestSQLTranslatorDataDriven(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - datadriven.Walk(t, "testdata/", func(t *testing.T, path string) { - tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ - ServerArgs: base.TestServerArgs{ - EnableSpanConfigs: true, - Knobs: base.TestingKnobs{ - SpanConfig: &spanconfig.TestingKnobs{ - ManagerDisableJobCreation: true, - }, - }, - }, - }) - defer tc.Stopper().Stop(ctx) - sqlDB := tc.ServerConn(0 /* idx */) - - sqlTranslator := tc.Server(0).SpanConfigSQLTranslator().(spanconfig.SQLTranslator) - datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { - switch d.Cmd { - case "exec-sql": - _, err := sqlDB.Exec(d.Input) - if err != nil { - return err.Error() - } - case "query-sql": - rows, err := sqlDB.Query(d.Input) - if err != nil { - return err.Error() - } - output, err := sqlutils.RowsToDataDrivenOutput(rows) - require.NoError(t, err) - return output - case "translate": - // Parse the args to get the object ID we're interested in translating. - objID := descpb.InvalidID - if d.HasArg(namedZone) { - var zone string - d.ScanArgs(t, namedZone, &zone) - namedZoneID, found := zonepb.NamedZones[zonepb.NamedZone(zone)] - require.Truef(t, found, "unknown named zone: %s", zone) - objID = descpb.ID(namedZoneID) - } else if d.HasArg(database) { - var dbName string - d.ScanArgs(t, database, &dbName) - if d.HasArg(table) { - var tbName string - d.ScanArgs(t, table, &tbName) - tableDesc := catalogkv.TestingGetTableDescriptor( - tc.Server(0).DB(), - tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).Codec, - dbName, - tbName, - ) - objID = tableDesc.GetID() - } else { - dbDesc := catalogkv.TestingGetDatabaseDescriptor( - tc.Server(0).DB(), - tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).Codec, - dbName, - ) - objID = dbDesc.GetID() - } - } else if d.HasArg(id) { - var scanID int - d.ScanArgs(t, id, &scanID) - objID = descpb.ID(scanID) - } else { - t.Fatal("insufficient args provided to translate") - } - entries, _, err := sqlTranslator.Translate(ctx, descpb.IDs{objID}) - require.NoError(t, err) - return datadrivenTranslationResult(entries) - case "full-translate": - entries, _, err := spanconfig.FullTranslate(ctx, sqlTranslator) - require.NoError(t, err) - return datadrivenTranslationResult(entries) - case "sleep": - var sleepDuration int - d.ScanArgs(t, duration, &sleepDuration) - time.Sleep(time.Second * time.Duration(sleepDuration)) - case "mark-table-offline": - var dbName string - d.ScanArgs(t, database, &dbName) - var tbName string - d.ScanArgs(t, table, &tbName) - err := modifyTableDescriptor(ctx, tc, dbName, tbName, func(mutable *tabledesc.Mutable) { - mutable.SetOffline("for testing") - }) - require.NoError(t, err) - case "mark-table-public": - var dbName string - d.ScanArgs(t, database, &dbName) - var tbName string - d.ScanArgs(t, table, &tbName) - err := modifyTableDescriptor(ctx, tc, dbName, tbName, func(mutable *tabledesc.Mutable) { - mutable.SetPublic() - }) - require.NoError(t, err) - default: - t.Fatalf("unknown command: %s", d.Cmd) - } - return "" - }) - }) -} - -// Constants for data-driven args. -const ( - id = "id" - namedZone = "named-zone" - table = "table" - database = "database" - duration = "duration" -) - -func modifyTableDescriptor( - ctx context.Context, - tc *testcluster.TestCluster, - dbName string, - tbName string, - f func(*tabledesc.Mutable), -) error { - cfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) - return sql.DescsTxn(ctx, &cfg, func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, - ) error { - _, tableDesc, err := descsCol.GetMutableTableByName( - ctx, - txn, - tree.NewTableNameWithSchema(tree.Name(dbName), "public", tree.Name(tbName)), - tree.ObjectLookupFlags{ - CommonLookupFlags: tree.CommonLookupFlags{ - Required: true, - IncludeOffline: true, - }, - }, - ) - if err != nil { - return err - } - f(tableDesc) - return descsCol.WriteDesc(ctx, false, tableDesc, txn) - }) -} - -// datadrivenTranslationResult constructs the datadriven output for a given -// slice of (translated) span config entries. -func datadrivenTranslationResult(entries []roachpb.SpanConfigEntry) string { - var output strings.Builder - for _, entry := range entries { - res := diffEntryAgainstRangeDefault(entry) - output.WriteString(res) - output.WriteByte('\n') - } - return output.String() -} - -// diffEntryAgainstRangeDefault computes the difference between the given config -// and RANGE DEFAULT. It then constructs a (span<->mismatching field(s)) string -// and returns it. If there config is same as RANGE DEFAULT, a (span, DEFAULT) -// string is returned instead. -func diffEntryAgainstRangeDefault(entry roachpb.SpanConfigEntry) string { - defaultSpanConfig := roachpb.TestingDefaultSpanConfig() - var diffs []string - - if entry.Config.RangeMaxBytes != defaultSpanConfig.RangeMaxBytes { - diffs = append(diffs, fmt.Sprintf("range_max_bytes=%d", entry.Config.RangeMaxBytes)) - } - if entry.Config.RangeMinBytes != defaultSpanConfig.RangeMinBytes { - diffs = append(diffs, fmt.Sprintf("range_min_bytes=%d", entry.Config.RangeMinBytes)) - } - if entry.Config.GCPolicy.TTLSeconds != defaultSpanConfig.GCPolicy.TTLSeconds { - diffs = append(diffs, fmt.Sprintf("ttl_seconds=%d", entry.Config.GCPolicy.TTLSeconds)) - } - if entry.Config.GlobalReads != defaultSpanConfig.GlobalReads { - diffs = append(diffs, fmt.Sprintf("global_reads=%v", entry.Config.GlobalReads)) - } - if entry.Config.NumReplicas != defaultSpanConfig.NumReplicas { - diffs = append(diffs, fmt.Sprintf("num_replicas=%d", entry.Config.NumReplicas)) - } - if entry.Config.NumVoters != defaultSpanConfig.NumVoters { - diffs = append(diffs, fmt.Sprintf("num_voters=%d", entry.Config.NumVoters)) - } - if !reflect.DeepEqual(entry.Config.Constraints, defaultSpanConfig.Constraints) { - diffs = append(diffs, fmt.Sprintf("constraints=%v", entry.Config.Constraints)) - } - if !reflect.DeepEqual(entry.Config.VoterConstraints, defaultSpanConfig.VoterConstraints) { - diffs = append(diffs, fmt.Sprintf("voter_constraints=%v", entry.Config.VoterConstraints)) - } - if !reflect.DeepEqual(entry.Config.LeasePreferences, defaultSpanConfig.LeasePreferences) { - diffs = append(diffs, fmt.Sprintf("lease_preferences=%v", entry.Config.VoterConstraints)) - } - - if len(diffs) == 0 { - diffs = []string{"DEFAULT"} - } - return fmt.Sprintf("%-30s %s", entry.Span.String(), strings.Join(diffs, " ")) -} diff --git a/pkg/ccl/spanconfigccl/BUILD.bazel b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel similarity index 78% rename from pkg/ccl/spanconfigccl/BUILD.bazel rename to pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel index 60b773d08cbf..d8adfb8da891 100644 --- a/pkg/ccl/spanconfigccl/BUILD.bazel +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel @@ -1,7 +1,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test") go_test( - name = "spanconfigccl_test", + name = "spanconfigsqltranslatorccl_test", srcs = [ "datadriven_test.go", "main_test.go", @@ -9,21 +9,20 @@ go_test( data = glob(["testdata/**"]), deps = [ "//pkg/base", + "//pkg/ccl/kvccl/kvtenantccl", "//pkg/ccl/partitionccl", "//pkg/ccl/utilccl", "//pkg/config/zonepb", - "//pkg/kv", "//pkg/roachpb:with-mocks", "//pkg/security", "//pkg/security/securitytest", "//pkg/server", "//pkg/spanconfig", - "//pkg/sql", - "//pkg/sql/catalog/catalogkv", + "//pkg/spanconfig/spanconfigtestutils", + "//pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster", "//pkg/sql/catalog/descpb", - "//pkg/sql/catalog/descs", "//pkg/sql/catalog/tabledesc", - "//pkg/sql/sem/tree", + "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go new file mode 100644 index 000000000000..f61d89ba3c4f --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go @@ -0,0 +1,194 @@ +// Copyright 2021 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package spanconfigsqltranslatorccl + +import ( + "context" + "fmt" + "sort" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" + _ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl" + "github.com/cockroachdb/cockroach/pkg/config/zonepb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/datadriven" + "github.com/stretchr/testify/require" +) + +// TestDataDriven is a data-driven test for the spanconfig.SQLTranslator. It +// allows users to set up zone config hierarchies and validate their translation +// to SpanConfigs is as expected. Only fields that are different from the +// (default) RANGE DEFAULT are printed in the test output for readability. It +// offers the following commands: +// +// - "exec-sql" +// Executes the input SQL query. +// +// - "query-sql" +// Executes the input SQL query and prints the results. +// +// - "translate" [database=] [table=] [named-zone=] [id=] +// Translates the SQL zone config state to the span config state starting +// from the referenced object (named zone, database, database + table, or +// descriptor id) as the root. +// +// - "full-translate" +// Performs a full translation of the SQL zone config state to the implied +// span config state. +// +// - "mark-table-offline" [database=] [table=] +// Marks the given table as offline for testing purposes. +// +// - "mark-table-public" [database=] [table=] +// Marks the given table as public. +// +func TestDataDriven(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + scKnobs := &spanconfig.TestingKnobs{ + // Instead of relying on the GC job to wait out TTLs and clear out + // descriptors, let's simply exclude dropped tables to simulate + // descriptors no longer existing. See comment on + // ExcludeDroppedDescriptorsFromLookup for more details. + ExcludeDroppedDescriptorsFromLookup: true, + // We run the reconciler manually in this test (through the span config + // test cluster). + ManagerDisableJobCreation: true, + } + datadriven.Walk(t, testutils.TestDataPath(t), func(t *testing.T, path string) { + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + EnableSpanConfigs: true, + Knobs: base.TestingKnobs{ + SpanConfig: scKnobs, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + spanConfigTestCluster := spanconfigtestcluster.NewHandle(t, tc) + defer spanConfigTestCluster.Cleanup() + + var tenant *spanconfigtestcluster.Tenant + if strings.Contains(path, "tenant") { + tenant = spanConfigTestCluster.InitializeTenant(ctx, roachpb.MakeTenantID(10)) + tenant.Exec(`SET CLUSTER SETTING sql.zone_configs.experimental_allow_for_secondary_tenant.enabled = true`) + } else { + tenant = spanConfigTestCluster.InitializeTenant(ctx, roachpb.SystemTenantID) + } + + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "exec-sql": + tenant.Exec(d.Input) + + case "query-sql": + rows := tenant.Query(d.Input) + output, err := sqlutils.RowsToDataDrivenOutput(rows) + require.NoError(t, err) + return output + + case "translate": + // Parse the args to get the object ID we're looking to + // translate. + var objID descpb.ID + switch { + case d.HasArg("named-zone"): + var zone string + d.ScanArgs(t, "named-zone", &zone) + namedZoneID, found := zonepb.NamedZones[zonepb.NamedZone(zone)] + require.Truef(t, found, "unknown named zone: %s", zone) + objID = descpb.ID(namedZoneID) + case d.HasArg("id"): + var scanID int + d.ScanArgs(t, "id", &scanID) + objID = descpb.ID(scanID) + case d.HasArg("database"): + var dbName string + d.ScanArgs(t, "database", &dbName) + if d.HasArg("table") { + var tbName string + d.ScanArgs(t, "table", &tbName) + objID = tenant.LookupTableByName(ctx, dbName, tbName).GetID() + } else { + objID = tenant.LookupDatabaseByName(ctx, dbName).GetID() + } + default: + d.Fatalf(t, "insufficient/improper args (%v) provided to translate", d.CmdArgs) + } + + sqlTranslator := tenant.SpanConfigSQLTranslator().(spanconfig.SQLTranslator) + entries, _, err := sqlTranslator.Translate(ctx, descpb.IDs{objID}) + require.NoError(t, err) + sort.Slice(entries, func(i, j int) bool { + return entries[i].Span.Key.Compare(entries[j].Span.Key) < 0 + }) + + var output strings.Builder + for _, entry := range entries { + output.WriteString(fmt.Sprintf("%-42s %s\n", entry.Span, + spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(entry.Config))) + } + return output.String() + + case "full-translate": + sqlTranslator := tenant.SpanConfigSQLTranslator().(spanconfig.SQLTranslator) + entries, _, err := spanconfig.FullTranslate(ctx, sqlTranslator) + require.NoError(t, err) + + sort.Slice(entries, func(i, j int) bool { + return entries[i].Span.Key.Compare(entries[j].Span.Key) < 0 + }) + var output strings.Builder + for _, entry := range entries { + output.WriteString(fmt.Sprintf("%-42s %s\n", entry.Span, + spanconfigtestutils.PrintSpanConfigDiffedAgainstDefaults(entry.Config))) + } + return output.String() + + case "mark-table-offline": + var dbName, tbName string + d.ScanArgs(t, "database", &dbName) + d.ScanArgs(t, "table", &tbName) + tenant.WithMutableTableDescriptor(ctx, dbName, tbName, func(mutable *tabledesc.Mutable) { + mutable.SetOffline("for testing") + }) + + case "mark-table-public": + var dbName, tbName string + d.ScanArgs(t, "database", &dbName) + d.ScanArgs(t, "table", &tbName) + tenant.WithMutableTableDescriptor(ctx, dbName, tbName, func(mutable *tabledesc.Mutable) { + mutable.SetPublic() + }) + + default: + t.Fatalf("unknown command: %s", d.Cmd) + } + + return "" + }) + }) +} diff --git a/pkg/ccl/spanconfigccl/main_test.go b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/main_test.go similarity index 91% rename from pkg/ccl/spanconfigccl/main_test.go rename to pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/main_test.go index ac929a3c3a94..2edadf57efbc 100644 --- a/pkg/ccl/spanconfigccl/main_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/main_test.go @@ -6,7 +6,7 @@ // // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package spanconfigccl_test +package spanconfigsqltranslatorccl import ( "os" @@ -21,7 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/randutil" ) -//go:generate ../../util/leaktest/add-leaktest.sh *_test.go +//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go func TestMain(m *testing.M) { defer utilccl.TestingEnableEnterprise()() diff --git a/pkg/ccl/spanconfigccl/testdata/databases b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/databases similarity index 70% rename from pkg/ccl/spanconfigccl/testdata/databases rename to pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/databases index 88f2a8fda776..bde89cea80fd 100644 --- a/pkg/ccl/spanconfigccl/testdata/databases +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/databases @@ -22,8 +22,8 @@ SELECT id FROM system.namespace WHERE name='t2' # We only expect there to be span config entries for tables t1 and t2. translate database=db ---- -/Table/5{6-7} DEFAULT -/Table/6{0-1} DEFAULT +/Table/5{6-7} range default +/Table/6{0-1} range default # Alter zone config fields on the database and one of the tables to ensure # things are cascading. @@ -34,16 +34,16 @@ ALTER TABLE db.t1 CONFIGURE ZONE USING num_voters=5; translate database=db ---- -/Table/5{6-7} num_replicas=7 num_voters=5 -/Table/6{0-1} num_replicas=7 +/Table/5{6-7} num_replicas=7 num_voters=5 +/Table/6{0-1} num_replicas=7 # Translating the tables in the database individually should result in the same # config as above. translate database=db table=t1 ---- -/Table/5{6-7} num_replicas=7 num_voters=5 +/Table/5{6-7} num_replicas=7 num_voters=5 translate database=db table=t2 ---- -/Table/6{0-1} num_replicas=7 +/Table/6{0-1} num_replicas=7 diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate new file mode 100644 index 000000000000..58840042cf66 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate @@ -0,0 +1,64 @@ +# Test a full reconciliation pass with default named zone configs, a single +# database with a single table within it. + +exec-sql +CREATE DATABASE db; +CREATE SCHEMA sc; +CREATE TYPE typ AS ENUM(); +CREATE TABLE db.t(); +---- + +# We expect only the following spans: +# - Meta ranges: min -> Liveness range start +# - Liveness range +# - System ranges: +# - Liveness range end -> Timeseries range start +# - Timeseries range end -> system range end +# - Timeseries range +# - All system tables (there should be no entry for pseudo IDs or IDs for which +# no table exist) +# - The user created table +full-translate +---- +/{Min-System/NodeLiveness} ttl_seconds=3600 num_replicas=5 +/System/NodeLiveness{-Max} ttl_seconds=600 num_replicas=5 +/System/{NodeLivenessMax-tsd} range system +/System{/tsd-tse} range default +/System{tse-/Max} range system +/Table/{3-4} range system +/Table/{4-5} range system +/Table/{5-6} range system +/Table/{6-7} range system +/Table/{8-9} range system +/Table/1{1-2} range system +/Table/1{2-3} range system +/Table/1{3-4} range system +/Table/1{4-5} range system +/Table/1{5-6} range system +/Table/{19-20} range system +/Table/2{0-1} range system +/Table/2{1-2} range system +/Table/2{3-4} range system +/Table/2{4-5} range system +/Table/2{5-6} ttl_seconds=600 num_replicas=5 +/Table/2{6-7} range system +/Table/2{7-8} ttl_seconds=600 num_replicas=5 +/Table/2{8-9} range system +/NamespaceTable/{30-Max} range system +/{NamespaceTable/Max-Table/32} range system +/Table/3{2-3} range system +/Table/3{3-4} range system +/Table/3{4-5} range system +/Table/3{5-6} range system +/Table/3{6-7} range system +/Table/3{7-8} range system +/Table/{39-40} range system +/Table/4{0-1} range system +/Table/4{1-2} range system +/Table/4{2-3} range system +/Table/4{3-4} range system +/Table/4{4-5} range system +/Table/4{5-6} ttl_seconds=7200 num_replicas=5 +/Table/4{6-7} range system +/Table/4{7-8} range system +/Table/{59-60} range default diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate_named_zones_deleted b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate_named_zones_deleted new file mode 100644 index 000000000000..109b54f323fa --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/full_translate_named_zones_deleted @@ -0,0 +1,79 @@ +# This file tests a full translation when all named zone entries have been +# removed from system.zones. The expectation is for us to still generate span +# config updates for these named zones. These should be the same as RANGE +# DEFAULT because that's what they inherit from in the absence of an explicit +# zone config. + +query-sql +SELECT id FROM system.zones +---- +0 +1 +16 +17 +22 +25 +27 +45 + +# Note that discarding RANGE_DEFAULT isn't allowed. +exec-sql +ALTER RANGE liveness CONFIGURE ZONE DISCARD; +ALTER RANGE meta CONFIGURE ZONE DISCARD; +ALTER RANGE system CONFIGURE ZONE DISCARD; +ALTER RANGE timeseries CONFIGURE ZONE DISCARD; +---- + +query-sql +SELECT id FROM system.zones +---- +0 +1 +25 +27 +45 + + +full-translate +---- +/{Min-System/NodeLiveness} range default +/System/NodeLiveness{-Max} range default +/System/{NodeLivenessMax-tsd} range default +/System{/tsd-tse} range default +/System{tse-/Max} range default +/Table/{3-4} range system +/Table/{4-5} range system +/Table/{5-6} range system +/Table/{6-7} range system +/Table/{8-9} range system +/Table/1{1-2} range system +/Table/1{2-3} range system +/Table/1{3-4} range system +/Table/1{4-5} range system +/Table/1{5-6} range system +/Table/{19-20} range system +/Table/2{0-1} range system +/Table/2{1-2} range system +/Table/2{3-4} range system +/Table/2{4-5} range system +/Table/2{5-6} ttl_seconds=600 num_replicas=5 +/Table/2{6-7} range system +/Table/2{7-8} ttl_seconds=600 num_replicas=5 +/Table/2{8-9} range system +/NamespaceTable/{30-Max} range system +/{NamespaceTable/Max-Table/32} range system +/Table/3{2-3} range system +/Table/3{3-4} range system +/Table/3{4-5} range system +/Table/3{5-6} range system +/Table/3{6-7} range system +/Table/3{7-8} range system +/Table/{39-40} range system +/Table/4{0-1} range system +/Table/4{1-2} range system +/Table/4{2-3} range system +/Table/4{3-4} range system +/Table/4{4-5} range system +/Table/4{5-6} ttl_seconds=7200 num_replicas=5 +/Table/4{6-7} range system +/Table/4{7-8} range system diff --git a/pkg/ccl/spanconfigccl/testdata/indexes b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/indexes similarity index 75% rename from pkg/ccl/spanconfigccl/testdata/indexes rename to pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/indexes index d484f91e1c7e..d2acf24243d5 100644 --- a/pkg/ccl/spanconfigccl/testdata/indexes +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/indexes @@ -51,9 +51,9 @@ INDEX db.public.t@idx ALTER INDEX db.public.t@idx CONFIGURE ZONE USING # the same as the table's config. translate database=db table=t ---- -/Table/56{-/2} num_replicas=7 -/Table/56/{2-3} num_replicas=7 num_voters=5 -/Table/5{6/3-7} num_replicas=7 +/Table/56{-/2} num_replicas=7 +/Table/56/{2-3} num_replicas=7 num_voters=5 +/Table/5{6/3-7} num_replicas=7 # Configure GC ttl on the database and override it for the index. The table # continues to hold a placeholder zone config. @@ -64,9 +64,9 @@ ALTER INDEX db.t@idx CONFIGURE ZONE USING gc.ttlseconds = 25 translate database=db table=t ---- -/Table/56{-/2} ttl_seconds=3600 num_replicas=7 -/Table/56/{2-3} ttl_seconds=25 num_replicas=7 num_voters=5 -/Table/5{6/3-7} ttl_seconds=3600 num_replicas=7 +/Table/56{-/2} ttl_seconds=3600 num_replicas=7 +/Table/56/{2-3} ttl_seconds=25 num_replicas=7 num_voters=5 +/Table/5{6/3-7} ttl_seconds=3600 num_replicas=7 # Configure a zone config field on the table, so that it is no longer a # placeholder zone config. @@ -89,6 +89,6 @@ INDEX db.public.t@idx ALTER INDEX db.public.t@idx CONFIGURE ZONE USING translate database=db table=t ---- -/Table/56{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7 -/Table/56/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5 -/Table/5{6/3-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7 +/Table/56{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7 +/Table/56/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5 +/Table/5{6/3-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7 diff --git a/pkg/ccl/spanconfigccl/testdata/misc b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/misc similarity index 68% rename from pkg/ccl/spanconfigccl/testdata/misc rename to pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/misc index 56cbda6f7f24..b2f5a4cab94e 100644 --- a/pkg/ccl/spanconfigccl/testdata/misc +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/misc @@ -11,34 +11,27 @@ ALTER TABLE db.t1 CONFIGURE ZONE USING gc.ttlseconds=1; translate database=db ---- -/Table/5{6-7} ttl_seconds=1 -/Table/5{7-8} DEFAULT +/Table/5{6-7} ttl_seconds=1 +/Table/5{7-8} range default # Drop the table. exec-sql DROP TABLE db.t1; ---- -# We still should be able to generate the span configuration for it when -# starting our translation from the database. +# We should no longer see the dropped table's spans. translate database=db ---- -/Table/5{6-7} ttl_seconds=1 -/Table/5{7-8} DEFAULT +/Table/5{7-8} range default # Same as above, except this time the translation starts from the table's ID. translate id=53 ---- -# Sleep for 5 seconds, which is more than the TTL on db.t1, so that the gc job -# can delete the descriptor. -sleep duration=5 ----- - # By now t1's descriptor should have been deleted. translate database=db ---- -/Table/5{7-8} DEFAULT +/Table/5{7-8} range default # This no longer exists, so no span configuration should be generated. translate id=53 @@ -53,11 +46,11 @@ mark-table-offline database=db table=t2 # table. translate database=db table=t2 ---- -/Table/5{7-8} DEFAULT +/Table/5{7-8} range default translate database=db ---- -/Table/5{7-8} DEFAULT +/Table/5{7-8} range default # Mark the table as public again. @@ -66,7 +59,7 @@ mark-table-public database=db table=t2 translate database=db table=t2 ---- -/Table/5{7-8} DEFAULT +/Table/5{7-8} range default # Test schemas/types don't generate a span configuration. exec-sql @@ -85,7 +78,7 @@ translate id=56 # Array type alias. translate id=57 ---- -/Table/5{7-8} DEFAULT +/Table/5{7-8} range default # Test that non-existent IDs do not generate span configurations either. translate id=500 diff --git a/pkg/ccl/spanconfigccl/testdata/named_zones b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/named_zones similarity index 73% rename from pkg/ccl/spanconfigccl/testdata/named_zones rename to pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/named_zones index 9ef8220e53bc..6bd36c51c3a3 100644 --- a/pkg/ccl/spanconfigccl/testdata/named_zones +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/named_zones @@ -13,7 +13,7 @@ SELECT count(*) FROM system.zones WHERE id=18 translate named-zone=timeseries ---- -/System{/tsd-tse} DEFAULT +/System{/tsd-tse} range default # Adding an explicit zone configuration for the timeseries range should work # as expected. @@ -23,7 +23,7 @@ ALTER RANGE timeseries CONFIGURE ZONE USING gc.ttlseconds=1000 translate named-zone=timeseries ---- -/System{/tsd-tse} ttl_seconds=1000 +/System{/tsd-tse} ttl_seconds=1000 # Change a field on the liveness range and ensure it behaves as expected. exec-sql @@ -32,7 +32,7 @@ ALTER RANGE liveness CONFIGURE ZONE USING num_replicas=7; translate named-zone=liveness ---- -/System/NodeLiveness{-Max} ttl_seconds=600 num_replicas=7 +/System/NodeLiveness{-Max} ttl_seconds=600 num_replicas=7 # We are allowed to discard the liveness range's zone configuration. The # generated span should have the RANGE DEFAULT config. @@ -42,4 +42,9 @@ ALTER RANGE liveness CONFIGURE ZONE DISCARD translate named-zone=liveness ---- -/System/NodeLiveness{-Max} DEFAULT +/System/NodeLiveness{-Max} range default + +translate named-zone=system +---- +/System/{NodeLivenessMax-tsd} range system +/System{tse-/Max} range system diff --git a/pkg/ccl/spanconfigccl/testdata/partitions_primary_index b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/partitions_primary_index similarity index 64% rename from pkg/ccl/spanconfigccl/testdata/partitions_primary_index rename to pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/partitions_primary_index index f69597def721..89c818740b5c 100644 --- a/pkg/ccl/spanconfigccl/testdata/partitions_primary_index +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/partitions_primary_index @@ -55,7 +55,7 @@ TABLE db.public.t ALTER TABLE db.public.t CONFIGURE ZONE USING # there is just one entry here which covers the entire table's span. translate database=db table=t ---- -/Table/5{6-7} num_replicas=7 num_voters=5 +/Table/5{6-7} num_replicas=7 num_voters=5 exec-sql @@ -68,10 +68,10 @@ ALTER PARTITION one_two OF TABLE db.t CONFIGURE ZONE USING global_reads=true # have the table's zone configuration. translate database=db table=t ---- -/Table/56{-/1/1} num_replicas=7 num_voters=5 -/Table/56/1/{1-2} global_reads=true num_replicas=7 num_voters=5 -/Table/56/1/{2-3} global_reads=true num_replicas=7 num_voters=5 -/Table/5{6/1/3-7} num_replicas=7 num_voters=5 +/Table/56{-/1/1} num_replicas=7 num_voters=5 +/Table/56/1/{1-2} global_reads=true num_replicas=7 num_voters=5 +/Table/56/1/{2-3} global_reads=true num_replicas=7 num_voters=5 +/Table/5{6/1/3-7} num_replicas=7 num_voters=5 # Change two fields on the second partition. One of them (num_voters) is an # override on the value set on the database's zone config. The other, gc.ttlseconds, @@ -85,12 +85,12 @@ ALTER PARTITION three_four OF TABLE db.t CONFIGURE ZONE USING num_voters=3 # have the correct values of gc.ttlseconds (5) and num_voters (3). translate database=db table=t ---- -/Table/56{-/1/1} num_replicas=7 num_voters=5 -/Table/56/1/{1-2} global_reads=true num_replicas=7 num_voters=5 -/Table/56/1/{2-3} global_reads=true num_replicas=7 num_voters=5 -/Table/56/1/{3-4} ttl_seconds=5 num_replicas=7 num_voters=3 -/Table/56/1/{4-5} ttl_seconds=5 num_replicas=7 num_voters=3 -/Table/5{6/1/5-7} num_replicas=7 num_voters=5 +/Table/56{-/1/1} num_replicas=7 num_voters=5 +/Table/56/1/{1-2} global_reads=true num_replicas=7 num_voters=5 +/Table/56/1/{2-3} global_reads=true num_replicas=7 num_voters=5 +/Table/56/1/{3-4} ttl_seconds=5 num_replicas=7 num_voters=3 +/Table/56/1/{4-5} ttl_seconds=5 num_replicas=7 num_voters=3 +/Table/5{6/1/5-7} num_replicas=7 num_voters=5 exec-sql ALTER PARTITION default OF TABLE db.t CONFIGURE ZONE USING num_voters=6 @@ -107,14 +107,14 @@ ALTER PARTITION default OF TABLE db.t CONFIGURE ZONE USING num_voters=6 # set to 6, as that's what we did above. translate database=db table=t ---- -/Table/56{-/1} num_replicas=7 num_voters=5 -/Table/56/1{-/1} num_replicas=7 num_voters=6 -/Table/56/1/{1-2} global_reads=true num_replicas=7 num_voters=5 -/Table/56/1/{2-3} global_reads=true num_replicas=7 num_voters=5 -/Table/56/1/{3-4} ttl_seconds=5 num_replicas=7 num_voters=3 -/Table/56/1/{4-5} ttl_seconds=5 num_replicas=7 num_voters=3 -/Table/56/{1/5-2} num_replicas=7 num_voters=6 -/Table/5{6/2-7} num_replicas=7 num_voters=5 +/Table/56{-/1} num_replicas=7 num_voters=5 +/Table/56/1{-/1} num_replicas=7 num_voters=6 +/Table/56/1/{1-2} global_reads=true num_replicas=7 num_voters=5 +/Table/56/1/{2-3} global_reads=true num_replicas=7 num_voters=5 +/Table/56/1/{3-4} ttl_seconds=5 num_replicas=7 num_voters=3 +/Table/56/1/{4-5} ttl_seconds=5 num_replicas=7 num_voters=3 +/Table/56/{1/5-2} num_replicas=7 num_voters=6 +/Table/5{6/2-7} num_replicas=7 num_voters=5 # Discard the table's zone configuration. This essentially means that the table # has a "placeholder" zone config (to capture partition subzone configs). @@ -128,11 +128,11 @@ ALTER TABLE db.t CONFIGURE ZONE DISCARD # table above (which is where num_voters=5 was coming from). translate database=db table=t ---- -/Table/56{-/1} num_replicas=7 -/Table/56/1{-/1} num_replicas=7 num_voters=6 -/Table/56/1/{1-2} global_reads=true num_replicas=7 -/Table/56/1/{2-3} global_reads=true num_replicas=7 -/Table/56/1/{3-4} ttl_seconds=5 num_replicas=7 num_voters=3 -/Table/56/1/{4-5} ttl_seconds=5 num_replicas=7 num_voters=3 -/Table/56/{1/5-2} num_replicas=7 num_voters=6 -/Table/5{6/2-7} num_replicas=7 +/Table/56{-/1} num_replicas=7 +/Table/56/1{-/1} num_replicas=7 num_voters=6 +/Table/56/1/{1-2} global_reads=true num_replicas=7 +/Table/56/1/{2-3} global_reads=true num_replicas=7 +/Table/56/1/{3-4} ttl_seconds=5 num_replicas=7 num_voters=3 +/Table/56/1/{4-5} ttl_seconds=5 num_replicas=7 num_voters=3 +/Table/56/{1/5-2} num_replicas=7 num_voters=6 +/Table/5{6/2-7} num_replicas=7 diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/databases b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/databases new file mode 100644 index 000000000000..d8f4c9ca30b8 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/databases @@ -0,0 +1,48 @@ +# Create a database with some tables, types, and schemas for a secondary tenant. +# Check that span configurations are as we expect. + +exec-sql +CREATE DATABASE db; +CREATE TABLE db.t1(); +CREATE TYPE db.typ AS ENUM(); +CREATE SCHEMA db.sc; +CREATE TABLE db.t2(); +---- + +query-sql +SELECT id FROM system.namespace WHERE name='t1' +---- +56 + +query-sql +SELECT id FROM system.namespace WHERE name='t2' +---- +60 + +# We expect there to be span config entries for tables t1 and t2. +translate database=db +---- +/Tenant/10/Table/5{6-7} range default +/Tenant/10/Table/6{0-1} range default + +# Alter zone config fields on the database and one of the tables to ensure +# things are cascading. +exec-sql +ALTER DATABASE db CONFIGURE ZONE USING num_replicas=7; +ALTER TABLE db.t1 CONFIGURE ZONE USING num_voters=5; +---- + +translate database=db +---- +/Tenant/10/Table/5{6-7} num_replicas=7 num_voters=5 +/Tenant/10/Table/6{0-1} num_replicas=7 + +# Translating the tables in the database individually should result in the same +# config as above. +translate database=db table=t1 +---- +/Tenant/10/Table/5{6-7} num_replicas=7 num_voters=5 + +translate database=db table=t2 +---- +/Tenant/10/Table/6{0-1} num_replicas=7 diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate new file mode 100644 index 000000000000..c19ec8465faa --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/full_translate @@ -0,0 +1,90 @@ +# Test a secondary tenant full reconciliation pass with a single created +# database containing a single table. + +exec-sql +CREATE DATABASE db; +CREATE SCHEMA sc; +CREATE TYPE typ AS ENUM(); +CREATE TABLE db.t(); +---- + +# We expect only the following spans: +# - All system tables (there should be no entry for pseudo IDs or IDs for which +# no table exist) +# - The user created table +full-translate +---- +/Tenant/10/Table/{3-4} range default +/Tenant/10/Table/{4-5} range default +/Tenant/10/Table/{5-6} range default +/Tenant/10/Table/{6-7} range default +/Tenant/10/Table/{7-8} range default +/Tenant/10/Table/1{1-2} range default +/Tenant/10/Table/1{2-3} range default +/Tenant/10/Table/1{3-4} range default +/Tenant/10/Table/1{4-5} range default +/Tenant/10/Table/1{5-6} range default +/Tenant/10/Table/{19-20} range default +/Tenant/10/Table/2{0-1} range default +/Tenant/10/Table/2{1-2} range default +/Tenant/10/Table/2{3-4} range default +/Tenant/10/Table/2{4-5} range default +/Tenant/10/Table/2{5-6} range default +/Tenant/10/Table/2{6-7} range default +/Tenant/10/Table/2{7-8} range default +/Tenant/10/Table/2{8-9} range default +/Tenant/10/NamespaceTable/{30-Max} range default +/Tenant/10/{NamespaceTable/Max-Table/32} range default +/Tenant/10/Table/3{2-3} range default +/Tenant/10/Table/3{3-4} range default +/Tenant/10/Table/3{4-5} range default +/Tenant/10/Table/3{5-6} range default +/Tenant/10/Table/3{6-7} range default +/Tenant/10/Table/3{7-8} range default +/Tenant/10/Table/{39-40} range default +/Tenant/10/Table/4{0-1} range default +/Tenant/10/Table/4{1-2} range default +/Tenant/10/Table/4{2-3} range default +/Tenant/10/Table/4{3-4} range default +/Tenant/10/Table/4{4-5} range default +/Tenant/10/Table/4{6-7} range default +/Tenant/10/Table/{59-60} range default + +# We should expect the same for RANGE DEFAULT. +translate named-zone=default +---- +/Tenant/10/Table/{3-4} range default +/Tenant/10/Table/{4-5} range default +/Tenant/10/Table/{5-6} range default +/Tenant/10/Table/{6-7} range default +/Tenant/10/Table/{7-8} range default +/Tenant/10/Table/1{1-2} range default +/Tenant/10/Table/1{2-3} range default +/Tenant/10/Table/1{3-4} range default +/Tenant/10/Table/1{4-5} range default +/Tenant/10/Table/1{5-6} range default +/Tenant/10/Table/{19-20} range default +/Tenant/10/Table/2{0-1} range default +/Tenant/10/Table/2{1-2} range default +/Tenant/10/Table/2{3-4} range default +/Tenant/10/Table/2{4-5} range default +/Tenant/10/Table/2{5-6} range default +/Tenant/10/Table/2{6-7} range default +/Tenant/10/Table/2{7-8} range default +/Tenant/10/Table/2{8-9} range default +/Tenant/10/NamespaceTable/{30-Max} range default +/Tenant/10/{NamespaceTable/Max-Table/32} range default +/Tenant/10/Table/3{2-3} range default +/Tenant/10/Table/3{3-4} range default +/Tenant/10/Table/3{4-5} range default +/Tenant/10/Table/3{5-6} range default +/Tenant/10/Table/3{6-7} range default +/Tenant/10/Table/3{7-8} range default +/Tenant/10/Table/{39-40} range default +/Tenant/10/Table/4{0-1} range default +/Tenant/10/Table/4{1-2} range default +/Tenant/10/Table/4{2-3} range default +/Tenant/10/Table/4{3-4} range default +/Tenant/10/Table/4{4-5} range default +/Tenant/10/Table/4{6-7} range default +/Tenant/10/Table/{59-60} range default diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/indexes b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/indexes new file mode 100644 index 000000000000..b184fd6f0643 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/indexes @@ -0,0 +1,94 @@ +# We start off by creating a simple database -> table -> index hierarchy for a +# secondary tenant. We set a zone configuration on the index but not on the +# table. This means the table has a "placeholder zone config". + +exec-sql +CREATE DATABASE db; +CREATE TABLE db.t(i INT PRIMARY KEY, j INT); +CREATE INDEX idx ON db.t (j); +ALTER DATABASE db CONFIGURE ZONE USING num_replicas=7; +ALTER INDEX db.t@idx CONFIGURE ZONE USING num_voters = 5; +---- + +query-sql +SHOW ZONE CONFIGURATION FOR DATABASE db +---- +DATABASE db ALTER DATABASE db CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 7, + constraints = '[]', + lease_preferences = '[]' + +query-sql +SHOW ZONE CONFIGURATION FOR TABLE db.t +---- +DATABASE db ALTER DATABASE db CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 7, + constraints = '[]', + lease_preferences = '[]' + +query-sql +SHOW ZONE CONFIGURATION FOR INDEX db.t@idx +---- +INDEX db.public.t@idx ALTER INDEX db.public.t@idx CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 7, + num_voters = 5, + constraints = '[]', + voter_constraints = '[]', + lease_preferences = '[]' + +# First entry = primary index (table's config above) +# Second entry = index idx, so numvoters should be overridden. +# Third entry = Any future indexes that may be added to this table. Should be +# the same as the table's config. +translate database=db table=t +---- +/Tenant/10/Table/56{-/2} num_replicas=7 +/Tenant/10/Table/56/{2-3} num_replicas=7 num_voters=5 +/Tenant/10/Table/5{6/3-7} num_replicas=7 + +# Configure GC ttl on the database and override it for the index. The table +# continues to hold a placeholder zone config. +exec-sql +ALTER DATABASE db CONFIGURE ZONE USING gc.ttlseconds = 3600; +ALTER INDEX db.t@idx CONFIGURE ZONE USING gc.ttlseconds = 25 +---- + +translate database=db table=t +---- +/Tenant/10/Table/56{-/2} ttl_seconds=3600 num_replicas=7 +/Tenant/10/Table/56/{2-3} ttl_seconds=25 num_replicas=7 num_voters=5 +/Tenant/10/Table/5{6/3-7} ttl_seconds=3600 num_replicas=7 + +# Configure a zone config field on the table, so that it is no longer a +# placeholder zone config. +exec-sql +ALTER TABLE db.t CONFIGURE ZONE USING range_min_bytes = 1000, range_max_bytes=100000; +---- + +query-sql +SHOW ZONE CONFIGURATION FOR INDEX db.t@idx +---- +INDEX db.public.t@idx ALTER INDEX db.public.t@idx CONFIGURE ZONE USING + range_min_bytes = 1000, + range_max_bytes = 100000, + gc.ttlseconds = 25, + num_replicas = 7, + num_voters = 5, + constraints = '[]', + voter_constraints = '[]', + lease_preferences = '[]' + +translate database=db table=t +---- +/Tenant/10/Table/56{-/2} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7 +/Tenant/10/Table/56/{2-3} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=25 num_replicas=7 num_voters=5 +/Tenant/10/Table/5{6/3-7} range_max_bytes=100000 range_min_bytes=1000 ttl_seconds=3600 num_replicas=7 diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/misc b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/misc new file mode 100644 index 000000000000..e8e8b4bc577a --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/misc @@ -0,0 +1,88 @@ +# Miscellaneous edge case tests for the SQLTranslator, on a secondary tenant. + +# Test dropped tables/databases work correctly. +exec-sql +CREATE DATABASE db; +CREATE TABLE db.t1(); +CREATE TABLE db.t2(); +ALTER TABLE db.t1 CONFIGURE ZONE USING gc.ttlseconds=1; +---- + +translate database=db +---- +/Tenant/10/Table/5{6-7} ttl_seconds=1 +/Tenant/10/Table/5{7-8} range default + +# Drop the table. +exec-sql +DROP TABLE db.t1; +---- + +# We should no longer see the dropped table's spans. +translate database=db +---- +/Tenant/10/Table/5{6-7} ttl_seconds=1 +/Tenant/10/Table/5{7-8} range default + +# Same as above, except this time the translation starts from the table's ID. +translate id=53 +---- + +# By now t1's descriptor should have been deleted. +translate database=db +---- +/Tenant/10/Table/5{6-7} ttl_seconds=1 +/Tenant/10/Table/5{7-8} range default + +# This no longer exists, so no span configuration should be generated. +translate id=53 +---- + +# Mark table t2 as offline, we should still be able to generate a span +# configuration for it. +mark-table-offline database=db table=t2 +---- + +# Should work for both when we start from the table and when we start from the +# table. +translate database=db table=t2 +---- +/Tenant/10/Table/5{7-8} range default + +translate database=db +---- +/Tenant/10/Table/5{6-7} ttl_seconds=1 +/Tenant/10/Table/5{7-8} range default + + +# Mark the table as public again. +mark-table-public database=db table=t2 +---- + +translate database=db table=t2 +---- +/Tenant/10/Table/5{7-8} range default + +# Test schemas/types don't generate a span configuration. +exec-sql +CREATE SCHEMA db.sc; +CREATE TYPE db.typ AS ENUM(); +---- + +# Schema. +translate id=55 +---- + +# Enum. +translate id=56 +---- +/Tenant/10/Table/5{6-7} ttl_seconds=1 + +# Array type alias. +translate id=57 +---- +/Tenant/10/Table/5{7-8} range default + +# Test that non-existent IDs do not generate span configurations either. +translate id=500 +---- diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/partitions_primary_index b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/partitions_primary_index new file mode 100644 index 000000000000..e1de57fdbdb0 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/partitions_primary_index @@ -0,0 +1,138 @@ +# This test creates partitions for a secondary tenant, on a table's primary +# index. It moves the zone configurations on these partitions through the steps +# described inline, making assertions along the way. + +exec-sql +CREATE DATABASE db; +CREATE TABLE db.t(i INT PRIMARY KEY, j INT) PARTITION BY LIST (i) ( + PARTITION one_two VALUES IN (1, 2), + PARTITION three_four VALUES IN (3, 4), + PARTITION default VALUES IN (default) +); +ALTER DATABASE db CONFIGURE ZONE USING num_replicas=7; +ALTER TABLE db.t CONFIGURE ZONE USING num_voters=5; +---- + +query-sql +SHOW ZONE CONFIGURATION FOR DATABASE db +---- +DATABASE db ALTER DATABASE db CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 7, + constraints = '[]', + lease_preferences = '[]' + +query-sql +SHOW ZONE CONFIGURATION FOR TABLE db.t +---- +TABLE db.public.t ALTER TABLE db.public.t CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 7, + num_voters = 5, + constraints = '[]', + voter_constraints = '[]', + lease_preferences = '[]' + +query-sql +SHOW ZONE CONFIGURATION FOR PARTITION one_two OF TABLE db.t +---- +TABLE db.public.t ALTER TABLE db.public.t CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 7, + num_voters = 5, + constraints = '[]', + voter_constraints = '[]', + lease_preferences = '[]' + + +# There is no zone configuration set on any of the partitions initially. So +# there is just one entry here which covers the entire table's span. +translate database=db table=t +---- +/Tenant/10/Table/5{6-7} num_replicas=7 num_voters=5 + + +exec-sql +ALTER PARTITION one_two OF TABLE db.t CONFIGURE ZONE USING global_reads=true +---- + +# Now that we have a zone configuration on one of the partitions, `one_two`, +# which implies two (adjacent) spans. Both these configs have global reads set +# to true. The table's spans before and after the partitions span continue to +# have the table's zone configuration. +translate database=db table=t +---- +/Tenant/10/Table/56{-/1/1} num_replicas=7 num_voters=5 +/Tenant/10/Table/56/1/{1-2} global_reads=true num_replicas=7 num_voters=5 +/Tenant/10/Table/56/1/{2-3} global_reads=true num_replicas=7 num_voters=5 +/Tenant/10/Table/5{6/1/3-7} num_replicas=7 num_voters=5 + +# Change two fields on the second partition. One of them (num_voters) is an +# override on the value set on the database's zone config. The other, gc.ttlseconds, +# is not set on either the table or the database. +exec-sql +ALTER PARTITION three_four OF TABLE db.t CONFIGURE ZONE USING gc.ttlseconds=5; +ALTER PARTITION three_four OF TABLE db.t CONFIGURE ZONE USING num_voters=3 +---- + +# We expect 2 more (adjacent) spans for the second partition now. These should +# have the correct values of gc.ttlseconds (5) and num_voters (3). +translate database=db table=t +---- +/Tenant/10/Table/56{-/1/1} num_replicas=7 num_voters=5 +/Tenant/10/Table/56/1/{1-2} global_reads=true num_replicas=7 num_voters=5 +/Tenant/10/Table/56/1/{2-3} global_reads=true num_replicas=7 num_voters=5 +/Tenant/10/Table/56/1/{3-4} ttl_seconds=5 num_replicas=7 num_voters=3 +/Tenant/10/Table/56/1/{4-5} ttl_seconds=5 num_replicas=7 num_voters=3 +/Tenant/10/Table/5{6/1/5-7} num_replicas=7 num_voters=5 + +exec-sql +ALTER PARTITION default OF TABLE db.t CONFIGURE ZONE USING num_voters=6 +---- + +# Now that we've set a zone configuration on the default partition we get a few +# more entries. Of note is that this default partition is on the primary index. +# This results in a span before the primary index starts +# (which will always be empty) but has the span config of the table. We also have +# a span for all other secondary indexes with the same config at the end. +# We get two more spans that subdivide the primary index as well. One before the +# start of the first partition and one after the end of the second partition. This +# second span covers the remaining primary index. Both these spans have num_voters +# set to 6, as that's what we did above. +translate database=db table=t +---- +/Tenant/10/Table/56{-/1} num_replicas=7 num_voters=5 +/Tenant/10/Table/56/1{-/1} num_replicas=7 num_voters=6 +/Tenant/10/Table/56/1/{1-2} global_reads=true num_replicas=7 num_voters=5 +/Tenant/10/Table/56/1/{2-3} global_reads=true num_replicas=7 num_voters=5 +/Tenant/10/Table/56/1/{3-4} ttl_seconds=5 num_replicas=7 num_voters=3 +/Tenant/10/Table/56/1/{4-5} ttl_seconds=5 num_replicas=7 num_voters=3 +/Tenant/10/Table/56/{1/5-2} num_replicas=7 num_voters=6 +/Tenant/10/Table/5{6/2-7} num_replicas=7 num_voters=5 + +# Discard the table's zone configuration. This essentially means that the table +# has a "placeholder" zone config (to capture partition subzone configs). +exec-sql +ALTER TABLE db.t CONFIGURE ZONE DISCARD +---- + +# The expectation here is very similar to above, except the spans before the +# primary index and for secondary indexes now inherit num_voters from the +# database. This is because we removed the explicit zone configuration on the +# table above (which is where num_voters=5 was coming from). +translate database=db table=t +---- +/Tenant/10/Table/56{-/1} num_replicas=7 +/Tenant/10/Table/56/1{-/1} num_replicas=7 num_voters=6 +/Tenant/10/Table/56/1/{1-2} global_reads=true num_replicas=7 +/Tenant/10/Table/56/1/{2-3} global_reads=true num_replicas=7 +/Tenant/10/Table/56/1/{3-4} ttl_seconds=5 num_replicas=7 num_voters=3 +/Tenant/10/Table/56/1/{4-5} ttl_seconds=5 num_replicas=7 num_voters=3 +/Tenant/10/Table/56/{1/5-2} num_replicas=7 num_voters=6 +/Tenant/10/Table/5{6/2-7} num_replicas=7 diff --git a/pkg/ccl/spanconfigccl/testdata/full_translate b/pkg/ccl/spanconfigccl/testdata/full_translate deleted file mode 100644 index 0e23efa2b225..000000000000 --- a/pkg/ccl/spanconfigccl/testdata/full_translate +++ /dev/null @@ -1,66 +0,0 @@ -# This file tests a full reconciliation scenario when none of the named zone -# entries have been messed with and the user has created a single database -# and a single table inside that database. - -exec-sql -CREATE DATABASE db; -CREATE SCHEMA sc; -CREATE TYPE typ AS ENUM(); -CREATE TABLE db.t(); ----- - -# - User created table above, with ID 56. -# There should be no entry for IDs 52, 53, 54, and 55 as these belong to the -# database, schema, type, and type alias respectively. -# - All system tables. Note that there should be no entry for pseudo IDs or IDs -# for which no table exists. -# - NodeLivenessSpan -# - Meta ranges: min -> NodeLiveness start -# - System ranges: -# - NodeLiveness end -> TimeSeries Start -# - TimeSeries end -> System Ranges end -# - Time Series Span -full-translate ----- -/Table/{59-60} DEFAULT -/Table/{3-4} num_replicas=5 -/Table/{4-5} num_replicas=5 -/Table/{5-6} num_replicas=5 -/Table/{6-7} num_replicas=5 -/Table/{8-9} num_replicas=5 -/Table/1{1-2} num_replicas=5 -/Table/1{2-3} num_replicas=5 -/Table/1{3-4} num_replicas=5 -/Table/1{4-5} num_replicas=5 -/Table/1{5-6} num_replicas=5 -/Table/{19-20} num_replicas=5 -/Table/2{0-1} num_replicas=5 -/Table/2{1-2} num_replicas=5 -/Table/2{3-4} num_replicas=5 -/Table/2{4-5} num_replicas=5 -/Table/2{5-6} ttl_seconds=600 num_replicas=5 -/Table/2{6-7} num_replicas=5 -/Table/2{7-8} ttl_seconds=600 num_replicas=5 -/Table/2{8-9} num_replicas=5 -/NamespaceTable/{30-Max} num_replicas=5 -/{NamespaceTable/Max-Table/32} num_replicas=5 -/Table/3{2-3} num_replicas=5 -/Table/3{3-4} num_replicas=5 -/Table/3{4-5} num_replicas=5 -/Table/3{5-6} num_replicas=5 -/Table/3{6-7} num_replicas=5 -/Table/3{7-8} num_replicas=5 -/Table/{39-40} num_replicas=5 -/Table/4{0-1} num_replicas=5 -/Table/4{1-2} num_replicas=5 -/Table/4{2-3} num_replicas=5 -/Table/4{3-4} num_replicas=5 -/Table/4{4-5} num_replicas=5 -/Table/4{5-6} ttl_seconds=7200 num_replicas=5 -/Table/4{6-7} num_replicas=5 -/Table/4{7-8} num_replicas=5 -/System/NodeLiveness{-Max} ttl_seconds=600 num_replicas=5 -/{Min-System/NodeLiveness} ttl_seconds=3600 num_replicas=5 -/System/{NodeLivenessMax-tsd} num_replicas=5 -/System{tse-/Max} num_replicas=5 -/System{/tsd-tse} DEFAULT diff --git a/pkg/ccl/spanconfigccl/testdata/full_translate_named_zones_deleted b/pkg/ccl/spanconfigccl/testdata/full_translate_named_zones_deleted deleted file mode 100644 index 5095938c5860..000000000000 --- a/pkg/ccl/spanconfigccl/testdata/full_translate_named_zones_deleted +++ /dev/null @@ -1,79 +0,0 @@ -# This file tests a full translation when all named zone entries have been -# removed from system.zones. The expectation is for us to still generate span -# config updates for these named zones. These should be the same as RANGE -# DEFAULT because that's what they inherit from in the absence of an explicit -# zone config. - -query-sql -SELECT id FROM system.zones ----- -0 -1 -16 -17 -22 -25 -27 -45 - -# Note that discarding RANGE_DEFAULT isn't allowed. -exec-sql -ALTER RANGE liveness CONFIGURE ZONE DISCARD; -ALTER RANGE meta CONFIGURE ZONE DISCARD; -ALTER RANGE system CONFIGURE ZONE DISCARD; -ALTER RANGE timeseries CONFIGURE ZONE DISCARD; ----- - -query-sql -SELECT id FROM system.zones ----- -0 -1 -25 -27 -45 - - -full-translate ----- -/Table/{3-4} num_replicas=5 -/Table/{4-5} num_replicas=5 -/Table/{5-6} num_replicas=5 -/Table/{6-7} num_replicas=5 -/Table/{8-9} num_replicas=5 -/Table/1{1-2} num_replicas=5 -/Table/1{2-3} num_replicas=5 -/Table/1{3-4} num_replicas=5 -/Table/1{4-5} num_replicas=5 -/Table/1{5-6} num_replicas=5 -/Table/{19-20} num_replicas=5 -/Table/2{0-1} num_replicas=5 -/Table/2{1-2} num_replicas=5 -/Table/2{3-4} num_replicas=5 -/Table/2{4-5} num_replicas=5 -/Table/2{5-6} ttl_seconds=600 num_replicas=5 -/Table/2{6-7} num_replicas=5 -/Table/2{7-8} ttl_seconds=600 num_replicas=5 -/Table/2{8-9} num_replicas=5 -/NamespaceTable/{30-Max} num_replicas=5 -/{NamespaceTable/Max-Table/32} num_replicas=5 -/Table/3{2-3} num_replicas=5 -/Table/3{3-4} num_replicas=5 -/Table/3{4-5} num_replicas=5 -/Table/3{5-6} num_replicas=5 -/Table/3{6-7} num_replicas=5 -/Table/3{7-8} num_replicas=5 -/Table/{39-40} num_replicas=5 -/Table/4{0-1} num_replicas=5 -/Table/4{1-2} num_replicas=5 -/Table/4{2-3} num_replicas=5 -/Table/4{3-4} num_replicas=5 -/Table/4{4-5} num_replicas=5 -/Table/4{5-6} ttl_seconds=7200 num_replicas=5 -/Table/4{6-7} num_replicas=5 -/Table/4{7-8} num_replicas=5 -/System/NodeLiveness{-Max} DEFAULT -/{Min-System/NodeLiveness} DEFAULT -/System/{NodeLivenessMax-tsd} DEFAULT -/System{tse-/Max} DEFAULT -/System{/tsd-tse} DEFAULT diff --git a/pkg/config/zonepb/zone_test.go b/pkg/config/zonepb/zone_test.go index fbc95152980a..3e59516d8654 100644 --- a/pkg/config/zonepb/zone_test.go +++ b/pkg/config/zonepb/zone_test.go @@ -1426,7 +1426,13 @@ func TestZoneConfigToSpanConfigConversion(t *testing.T) { } } -func TestDefaultZoneAndSpanConfigs(t *testing.T) { - converted := DefaultZoneConfigRef().AsSpanConfig() - require.True(t, converted.Equal(roachpb.TestingDefaultSpanConfig())) +func TestHardCodedSpanConfigs(t *testing.T) { + { + converted := DefaultZoneConfigRef().AsSpanConfig() + require.True(t, converted.Equal(roachpb.TestingDefaultSpanConfig())) + } + { + converted := DefaultSystemZoneConfigRef().AsSpanConfig() + require.True(t, converted.Equal(roachpb.TestingSystemSpanConfig())) + } } diff --git a/pkg/roachpb/span_config.go b/pkg/roachpb/span_config.go index f65d664e7516..38cc852c919d 100644 --- a/pkg/roachpb/span_config.go +++ b/pkg/roachpb/span_config.go @@ -125,3 +125,23 @@ func TestingDefaultSpanConfig() SpanConfig { NumReplicas: 3, } } + +// TestingSystemSpanConfig exports the system span config for testing purposes. +func TestingSystemSpanConfig() SpanConfig { + return SpanConfig{ + RangeMinBytes: 128 << 20, // 128 MB + RangeMaxBytes: 512 << 20, // 512 MB + // Use 25 hours instead of the previous 24 to make users successful by + // default. Users desiring to take incremental backups every 24h may + // incorrectly assume that the previous default 24h was sufficient to do + // that. But the equation for incremental backups is: + // GC TTLSeconds >= (desired backup interval) + (time to perform incremental backup) + // We think most new users' incremental backups will complete within an + // hour, and larger clusters will have more experienced operators and will + // understand how to change these settings if needed. + GCPolicy: GCPolicy{ + TTLSeconds: 25 * 60 * 60, + }, + NumReplicas: 5, + } +} diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index acd468b306dd..11bd7fe13a00 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -856,7 +856,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { // Instantiate a span config manager. If we're the host tenant we'll // only do it if COCKROACH_EXPERIMENTAL_SPAN_CONFIGS is set. spanConfigKnobs, _ := cfg.TestingKnobs.SpanConfig.(*spanconfig.TestingKnobs) - sqlTranslator := spanconfigsqltranslator.New(execCfg, codec) + sqlTranslator := spanconfigsqltranslator.New(execCfg, codec, spanConfigKnobs) sqlWatcher := spanconfigsqlwatcher.New( codec, cfg.Settings, diff --git a/pkg/spanconfig/spanconfig.go b/pkg/spanconfig/spanconfig.go index 33720f98938d..85163591ec4f 100644 --- a/pkg/spanconfig/spanconfig.go +++ b/pkg/spanconfig/spanconfig.go @@ -106,15 +106,15 @@ type SQLTranslator interface { Translate(ctx context.Context, ids descpb.IDs) ([]roachpb.SpanConfigEntry, hlc.Timestamp, error) } -// FullTranslate translates the entire SQL zone configuration state to the -// span configuration state. The timestamp at which such a translation is valid -// is also returned. +// FullTranslate translates the entire SQL zone configuration state to the span +// configuration state. The timestamp at which such a translation is valid is +// also returned. func FullTranslate( ctx context.Context, s SQLTranslator, ) ([]roachpb.SpanConfigEntry, hlc.Timestamp, error) { - // As RANGE DEFAULT is the root of all zone configurations (including - // other named zones for the system tenant), we can construct the entire - // span configuration state by starting from RANGE DEFAULT. + // As RANGE DEFAULT is the root of all zone configurations (including other + // named zones for the system tenant), we can construct the entire span + // configuration state by starting from RANGE DEFAULT. return s.Translate(ctx, descpb.IDs{keys.RootNamespaceID}) } @@ -143,17 +143,6 @@ type SQLWatcher interface { ) error } -// DescriptorUpdate captures the ID and type of a descriptor or zone that the -// SQLWatcher has observed updated. -type DescriptorUpdate struct { - // ID of the descriptor/zone that has been updated. - ID descpb.ID - - // DescriptorType of the descriptor/zone that has been updated. Could be either - // the specific type or catalog.Any if no information is available. - DescriptorType catalog.DescriptorType -} - // ReconciliationDependencies captures what's needed by the span config // reconciliation job to perform its task. The job is responsible for // reconciling a tenant's zone configurations with the clusters span @@ -259,6 +248,17 @@ type StoreReader interface { GetSpanConfigForKey(ctx context.Context, key roachpb.RKey) (roachpb.SpanConfig, error) } +// DescriptorUpdate captures the ID and type of a descriptor or zone that the +// SQLWatcher has observed updated. +type DescriptorUpdate struct { + // ID of the descriptor/zone that has been updated. + ID descpb.ID + + // DescriptorType of the descriptor/zone that has been updated. Could be either + // the specific type or catalog.Any if no information is available. + DescriptorType catalog.DescriptorType +} + // Update captures a span and the corresponding config change. It's the unit of // what can be applied to a StoreWriter. type Update struct { diff --git a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go index 72314c67332f..4a6557a7c67a 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go +++ b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go @@ -37,13 +37,20 @@ var _ spanconfig.SQLTranslator = &SQLTranslator{} type SQLTranslator struct { execCfg *sql.ExecutorConfig codec keys.SQLCodec + knobs *spanconfig.TestingKnobs } // New constructs and returns a SQLTranslator. -func New(execCfg *sql.ExecutorConfig, codec keys.SQLCodec) *SQLTranslator { +func New( + execCfg *sql.ExecutorConfig, codec keys.SQLCodec, knobs *spanconfig.TestingKnobs, +) *SQLTranslator { + if knobs == nil { + knobs = &spanconfig.TestingKnobs{} + } return &SQLTranslator{ execCfg: execCfg, codec: codec, + knobs: knobs, } } @@ -64,7 +71,7 @@ func (s *SQLTranslator) Translate( // For every ID we want to translate, first expand it to descendant leaf // IDs that have span configurations associated for them. We also - // de-duplicate leaf IDs so as to not generate redundant entries. + // de-duplicate leaf IDs to not generate redundant entries. seen := make(map[descpb.ID]struct{}) var leafIDs descpb.IDs for _, id := range ids { @@ -80,8 +87,7 @@ func (s *SQLTranslator) Translate( } } - // For every leaf ID, which has been de-duplicated, generate span - // configurations. + // For every unique leaf ID, generate span configurations. for _, leafID := range leafIDs { translatedEntries, err := s.generateSpanConfigurations(ctx, leafID, txn, descsCol) if err != nil { @@ -116,11 +122,13 @@ func (s *SQLTranslator) generateSpanConfigurations( }) if err != nil { if errors.Is(err, catalog.ErrDescriptorNotFound) { - // The descriptor has been deleted. Nothing to do here. - return nil, nil + return nil, nil // the descriptor has been deleted; nothing to do here } return nil, err } + if s.knobs.ExcludeDroppedDescriptorsFromLookup && desc.Dropped() { + return nil, nil // we're excluding this descriptor; nothing to do here + } if desc.DescriptorType() != catalog.Table { return nil, errors.AssertionFailedf( @@ -297,11 +305,13 @@ func (s *SQLTranslator) findDescendantLeafIDsForDescriptor( }) if err != nil { if errors.Is(err, catalog.ErrDescriptorNotFound) { - // The descriptor has been deleted. Nothing to do here. - return nil, nil + return nil, nil // the descriptor has been deleted; nothing to do here } return nil, err } + if s.knobs.ExcludeDroppedDescriptorsFromLookup && desc.Dropped() { + return nil, nil // we're excluding this descriptor; nothing to do here + } switch desc.DescriptorType() { case catalog.Type, catalog.Schema: diff --git a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator_test.go b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator_test.go index e37fb9098593..5f82021f0395 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator_test.go +++ b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator_test.go @@ -10,4 +10,5 @@ package spanconfigsqltranslator_test -// Tests for this package can be found under pkg/ccl/spanconfigccl. +// Tests for this package can be found under +// pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl. diff --git a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go index a9f473f04bdf..35ff14b9129a 100644 --- a/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go +++ b/pkg/spanconfig/spanconfigsqlwatcher/sqlwatcher.go @@ -61,6 +61,11 @@ func New( if knobs == nil { knobs = &spanconfig.TestingKnobs{} } + + if override := knobs.SQLWatcherCheckpointNoopsEveryDurationOverride; override.Nanoseconds() != 0 { + checkpointNoopsEvery = override + } + return &SQLWatcher{ codec: codec, settings: settings, @@ -90,7 +95,6 @@ func (s *SQLWatcher) watch( startTS hlc.Timestamp, handler func(context.Context, []spanconfig.DescriptorUpdate, hlc.Timestamp) error, ) error { - // The callbacks below are invoked by both the rangefeeds we establish, both // of which run on separate goroutines. We serialize calls to the handler // function by invoking in this single watch thread (instead of pushing it diff --git a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/BUILD.bazel b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/BUILD.bazel new file mode 100644 index 000000000000..f0ee625179be --- /dev/null +++ b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/BUILD.bazel @@ -0,0 +1,27 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "spanconfigtestcluster", + srcs = [ + "cluster.go", + "tenant_state.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster", + visibility = ["//visibility:public"], + deps = [ + "//pkg/base", + "//pkg/kv", + "//pkg/roachpb:with-mocks", + "//pkg/security", + "//pkg/sql", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descs", + "//pkg/sql/catalog/tabledesc", + "//pkg/sql/sem/tree", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/log", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/cluster.go b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/cluster.go new file mode 100644 index 000000000000..6e06f97b885b --- /dev/null +++ b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/cluster.go @@ -0,0 +1,101 @@ +// Copyright 2021 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 spanconfigtestcluster + +import ( + "context" + gosql "database/sql" + "net/url" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +// Handle is a testing helper that lets users operate a multi-tenant test +// cluster while providing convenient, scoped access to each tenant's specific +// span config primitives. It's not safe for concurrent use. +type Handle struct { + t *testing.T + tc *testcluster.TestCluster + ts map[roachpb.TenantID]*Tenant +} + +// NewHandle returns a new Handle. +func NewHandle(t *testing.T, tc *testcluster.TestCluster) *Handle { + return &Handle{ + t: t, + tc: tc, + ts: make(map[roachpb.TenantID]*Tenant), + } +} + +// InitializeTenant initializes a tenant with the given ID, returning the +// relevant tenant state. +func (h *Handle) InitializeTenant(ctx context.Context, tenID roachpb.TenantID) *Tenant { + // Prevent a logging assertion that the server ID is initialized multiple + // times. + log.TestingClearServerIdentifiers() + + testServer := h.tc.Server(0) + tenantState := &Tenant{t: h.t} + if tenID == roachpb.SystemTenantID { + tenantState.TestTenantInterface = testServer + tenantState.db = sqlutils.MakeSQLRunner(h.tc.ServerConn(0)) + tenantState.cleanup = func() {} // noop + } else { + tenantArgs := base.TestTenantArgs{TenantID: tenID} + var err error + tenantState.TestTenantInterface, err = testServer.StartTenant(ctx, tenantArgs) + require.NoError(h.t, err) + + pgURL, cleanupPGUrl := sqlutils.PGUrl(h.t, tenantState.SQLAddr(), "Tenant", url.User(security.RootUser)) + tenantSQLDB, err := gosql.Open("postgres", pgURL.String()) + require.NoError(h.t, err) + + tenantState.db = sqlutils.MakeSQLRunner(tenantSQLDB) + tenantState.cleanup = func() { + require.NoError(h.t, tenantSQLDB.Close()) + cleanupPGUrl() + } + } + + h.ts[tenID] = tenantState + return tenantState +} + +// LookupTenant returns the relevant tenant state, if any. +func (h *Handle) LookupTenant(tenantID roachpb.TenantID) (_ *Tenant, found bool) { + s, ok := h.ts[tenantID] + return s, ok +} + +// Tenants returns all available tenant states. +func (h *Handle) Tenants() []*Tenant { + ts := make([]*Tenant, 0, len(h.ts)) + for _, tenantState := range h.ts { + ts = append(ts, tenantState) + } + return ts +} + +// Cleanup frees up internal resources. +func (h *Handle) Cleanup() { + for _, tenantState := range h.ts { + tenantState.cleanup() + } + h.ts = nil +} diff --git a/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go new file mode 100644 index 000000000000..4f9d8f2121cb --- /dev/null +++ b/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster/tenant_state.go @@ -0,0 +1,124 @@ +// Copyright 2021 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 spanconfigtestcluster + +import ( + "context" + gosql "database/sql" + "testing" + + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/stretchr/testify/require" +) + +// Tenant captures per-tenant span config state and encapsulates convenient +// span config testing primitives. It's safe for concurrent use. +type Tenant struct { + serverutils.TestTenantInterface + + t *testing.T + db *sqlutils.SQLRunner + cleanup func() +} + +// Exec is a wrapper around gosql.Exec that kills the test on error. +func (s *Tenant) Exec(query string, args ...interface{}) { + s.db.Exec(s.t, query, args...) +} + +// Query is a wrapper around gosql.Query that kills the test on error. +func (s *Tenant) Query(query string, args ...interface{}) *gosql.Rows { + return s.db.Query(s.t, query, args...) +} + +// WithMutableTableDescriptor invokes the provided callback with a mutable table +// descriptor, changes to which are then committed back to the system. The +// callback needs to be idempotent. +func (s *Tenant) WithMutableTableDescriptor( + ctx context.Context, dbName string, tbName string, f func(*tabledesc.Mutable), +) { + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + require.NoError(s.t, sql.DescsTxn(ctx, &execCfg, func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + ) error { + _, desc, err := descsCol.GetMutableTableByName( + ctx, + txn, + tree.NewTableNameWithSchema(tree.Name(dbName), "public", tree.Name(tbName)), + tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, + IncludeOffline: true, + }, + }, + ) + if err != nil { + return err + } + f(desc) + return descsCol.WriteDesc(ctx, false, desc, txn) + })) +} + +// LookupTableByName returns the table descriptor identified by the given name. +func (s *Tenant) LookupTableByName( + ctx context.Context, dbName string, tbName string, +) (desc catalog.TableDescriptor) { + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + require.NoError(s.t, sql.DescsTxn(ctx, &execCfg, + func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection) error { + var err error + _, desc, err = descsCol.GetMutableTableByName(ctx, txn, + tree.NewTableNameWithSchema(tree.Name(dbName), "public", tree.Name(tbName)), + tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, + IncludeOffline: true, + }, + }, + ) + if err != nil { + return err + } + return nil + })) + return desc +} + +// LookupDatabaseByName returns the database descriptor identified by the given +// name. +func (s *Tenant) LookupDatabaseByName( + ctx context.Context, dbName string, +) (desc catalog.DatabaseDescriptor) { + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + require.NoError(s.t, sql.DescsTxn(ctx, &execCfg, + func(ctx context.Context, txn *kv.Txn, descsCol *descs.Collection) error { + var err error + desc, err = descsCol.GetMutableDatabaseByName(ctx, txn, dbName, + tree.DatabaseLookupFlags{ + Required: true, + IncludeOffline: true, + }, + ) + if err != nil { + return err + } + return nil + })) + return desc +} diff --git a/pkg/spanconfig/spanconfigtestutils/utils.go b/pkg/spanconfig/spanconfigtestutils/utils.go index aa0e072171bb..7b057b6a77cb 100644 --- a/pkg/spanconfig/spanconfigtestutils/utils.go +++ b/pkg/spanconfig/spanconfigtestutils/utils.go @@ -12,6 +12,7 @@ package spanconfigtestutils import ( "fmt" + "reflect" "regexp" "strings" "testing" @@ -191,3 +192,47 @@ func PrintSpanConfig(conf roachpb.SpanConfig) string { func PrintSpanConfigEntry(entry roachpb.SpanConfigEntry) string { return fmt.Sprintf("%s:%s", PrintSpan(entry.Span), PrintSpanConfig(entry.Config)) } + +// PrintSpanConfigDiffedAgainstDefaults is a helper function that diffs the given +// config against RANGE {DEFAULT, SYSTEM} and returns a string for the +// mismatched fields. If there are none, "range {default,system}" is returned. +func PrintSpanConfigDiffedAgainstDefaults(conf roachpb.SpanConfig) string { + if conf.Equal(roachpb.TestingDefaultSpanConfig()) { + return "range default" + } + if conf.Equal(roachpb.TestingSystemSpanConfig()) { + return "range system" + } + + defaultConf := roachpb.TestingDefaultSpanConfig() + var diffs []string + if conf.RangeMaxBytes != defaultConf.RangeMaxBytes { + diffs = append(diffs, fmt.Sprintf("range_max_bytes=%d", conf.RangeMaxBytes)) + } + if conf.RangeMinBytes != defaultConf.RangeMinBytes { + diffs = append(diffs, fmt.Sprintf("range_min_bytes=%d", conf.RangeMinBytes)) + } + if conf.GCPolicy.TTLSeconds != defaultConf.GCPolicy.TTLSeconds { + diffs = append(diffs, fmt.Sprintf("ttl_seconds=%d", conf.GCPolicy.TTLSeconds)) + } + if conf.GlobalReads != defaultConf.GlobalReads { + diffs = append(diffs, fmt.Sprintf("global_reads=%v", conf.GlobalReads)) + } + if conf.NumReplicas != defaultConf.NumReplicas { + diffs = append(diffs, fmt.Sprintf("num_replicas=%d", conf.NumReplicas)) + } + if conf.NumVoters != defaultConf.NumVoters { + diffs = append(diffs, fmt.Sprintf("num_voters=%d", conf.NumVoters)) + } + if !reflect.DeepEqual(conf.Constraints, defaultConf.Constraints) { + diffs = append(diffs, fmt.Sprintf("constraints=%v", conf.Constraints)) + } + if !reflect.DeepEqual(conf.VoterConstraints, defaultConf.VoterConstraints) { + diffs = append(diffs, fmt.Sprintf("voter_constraints=%v", conf.VoterConstraints)) + } + if !reflect.DeepEqual(conf.LeasePreferences, defaultConf.LeasePreferences) { + diffs = append(diffs, fmt.Sprintf("lease_preferences=%v", conf.VoterConstraints)) + } + + return strings.Join(diffs, " ") +} diff --git a/pkg/spanconfig/testing_knobs.go b/pkg/spanconfig/testing_knobs.go index 38b9489c02e1..132f3ca777d4 100644 --- a/pkg/spanconfig/testing_knobs.go +++ b/pkg/spanconfig/testing_knobs.go @@ -11,6 +11,8 @@ package spanconfig import ( + "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) @@ -57,6 +59,16 @@ type TestingKnobs struct { // SQLWatcherOnEventInterceptor, if set, is invoked when the SQLWatcher // receives an event on one of its rangefeeds. SQLWatcherOnEventInterceptor func() error + + // SQLWatcherCheckpointNoopsEveryDurationOverride, if set, overrides how + // often the SQLWatcher checkpoints noops. + SQLWatcherCheckpointNoopsEveryDurationOverride time.Duration + + // ExcludeDroppedDescriptorsFromLookup is used to control if the + // SQLTranslator ignores dropped descriptors. If enabled, dropped + // descriptors appear as missing -- a convenient+faster alternative to + // waiting for the descriptor to actually get GC-ed in tests. + ExcludeDroppedDescriptorsFromLookup bool } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface.