From 84786e3c74771cbc4ff213c51970746eed8017e8 Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Thu, 13 Jan 2022 11:35:34 -0500 Subject: [PATCH] spanconfigsqltranslator: populate protected_timestamps in SpanConfig This change teaches the SQLTranslator to hydrate the SpanConfigs for a table with protected timestamps that apply to that table. Concretely, this change initializes a `spanconfig.ProtectedTimestampTableReader` in the txn in which the translation is taking place, thereby providing a transactional view of the `system.protected_ts_records` table. After generating the span configurations based on the zone configurations that apply to the table, we hydrate the newly introduced `protected_timestamps` field on each span configuration with all the protected timestamps that apply to this table. This includes protected timestamp records that directly target this table, as well as records targetting the table's parent database. This information is obtained from the `ProtectedTimestampTableReader` mentioned above. Additionally, this change modifies `StartTenant` to allow secondary tenants to interact with the protected timestamp subsystem using a "real" `protectedts.Provider` provided the migration `AlterSystemProtectedTimestampAddColumn` has run. This is sound because this migration will only run after span config reconciliation has started in tenants. For testing purposes, this change teaches the data driven framework of two additional commands `protect` and `release`. Informs: #73727 Release note: None --- .../spanconfigsqltranslatorccl/BUILD.bazel | 8 ++ .../datadriven_test.go | 53 +++++++++++- .../testdata/protectedts | 84 +++++++++++++++++++ .../testdata/tenant/protectedts | 83 ++++++++++++++++++ pkg/roachpb/span_config.proto | 6 ++ pkg/server/tenant.go | 2 +- pkg/server/testserver.go | 9 +- .../protectedts_table_reader.go | 3 +- .../spanconfigsqltranslator/BUILD.bazel | 1 + .../spanconfigsqltranslator/sqltranslator.go | 67 ++++++++++++++- pkg/spanconfig/spanconfigtestutils/utils.go | 15 ++++ 11 files changed, 323 insertions(+), 8 deletions(-) create mode 100644 pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts create mode 100644 pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/protectedts diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel index d8adfb8da891..40138fde8923 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel @@ -13,6 +13,10 @@ go_test( "//pkg/ccl/partitionccl", "//pkg/ccl/utilccl", "//pkg/config/zonepb", + "//pkg/jobs", + "//pkg/jobs/jobsprotectedts", + "//pkg/kv", + "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/roachpb:with-mocks", "//pkg/security", "//pkg/security/securitytest", @@ -20,15 +24,19 @@ go_test( "//pkg/spanconfig", "//pkg/spanconfig/spanconfigtestutils", "//pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster", + "//pkg/sql", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/tabledesc", + "//pkg/sql/distsql", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", + "//pkg/util/hlc", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/randutil", + "//pkg/util/uuid", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_stretchr_testify//require", ], diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go index f61d89ba3c4f..083aa014f938 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go @@ -19,17 +19,25 @@ import ( _ "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/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "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" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/distsql" "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/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/require" ) @@ -99,6 +107,29 @@ func TestDataDriven(t *testing.T) { tenant = spanConfigTestCluster.InitializeTenant(ctx, roachpb.SystemTenantID) } + execCfg := tenant.ExecutorConfig().(sql.ExecutorConfig) + ptp := tenant.DistSQLServer().(*distsql.ServerImpl).ServerConfig.ProtectedTimestampProvider + jr := tenant.JobRegistry().(*jobs.Registry) + + mkRecordAndProtect := func(recordID string, ts hlc.Timestamp, target *ptpb.Target) { + jobID := jr.MakeJobID() + require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + recID, err := uuid.FromBytes([]byte(strings.Repeat(recordID, 16))) + require.NoError(t, err) + rec := jobsprotectedts.MakeRecord(recID, int64(jobID), ts, + nil /* deprecatedSpans */, jobsprotectedts.Jobs, target) + return ptp.Protect(ctx, txn, rec) + })) + } + + releaseRecord := func(recordID string) { + require.NoError(t, execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + recID, err := uuid.FromBytes([]byte(strings.Repeat(recordID, 16))) + require.NoError(t, err) + return ptp.Release(ctx, txn, recID) + })) + } + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "exec-sql": @@ -183,7 +214,27 @@ func TestDataDriven(t *testing.T) { tenant.WithMutableTableDescriptor(ctx, dbName, tbName, func(mutable *tabledesc.Mutable) { mutable.SetPublic() }) - + case "protect": + var dbName, tbName string + var recordID string + var protectTS int + d.ScanArgs(t, "database", &dbName) + d.ScanArgs(t, "id", &recordID) + d.ScanArgs(t, "ts", &protectTS) + if d.HasArg("table") { + d.ScanArgs(t, "table", &tbName) + desc := tenant.LookupTableByName(ctx, dbName, tbName) + mkRecordAndProtect(recordID, hlc.Timestamp{WallTime: int64(protectTS)}, + ptpb.MakeSchemaObjectsTarget([]descpb.ID{desc.GetID()})) + } else { + desc := tenant.LookupDatabaseByName(ctx, dbName) + mkRecordAndProtect(recordID, hlc.Timestamp{WallTime: int64(protectTS)}, + ptpb.MakeSchemaObjectsTarget([]descpb.ID{desc.GetID()})) + } + case "release": + var recordID string + d.ScanArgs(t, "id", &recordID) + releaseRecord(recordID) default: t.Fatalf("unknown command: %s", d.Cmd) } diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts new file mode 100644 index 000000000000..96204dd43771 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts @@ -0,0 +1,84 @@ +# Create a database with some tables and write protected timestamps on the +# tables and database. Check that span configurations are as we expect. + +exec-sql +CREATE DATABASE db; +CREATE TABLE db.t1(id INT); +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' +---- +57 + +# We only expect there to be span config entries for tables t1 and t2. +translate database=db +---- +/Table/5{6-7} range default +/Table/5{7-8} 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; +---- + +# Write a protected timestamp on t1. +protect database=db table=t1 id=1 ts=1 +---- + +translate database=db +---- +/Table/5{6-7} num_replicas=7 num_voters=5 pts=[1] +/Table/5{7-8} num_replicas=7 + +# Write a protected timestamp on db, so we should see it on both t1 and t2. +protect database=db id=2 ts=2 +---- + +translate database=db +---- +/Table/5{6-7} num_replicas=7 num_voters=5 pts=[1 2] +/Table/5{7-8} num_replicas=7 pts=[2] + +# Release the protected timestamp on table t1 +release id=1 +---- + +translate database=db +---- +/Table/5{6-7} num_replicas=7 num_voters=5 pts=[2] +/Table/5{7-8} num_replicas=7 pts=[2] + +# Release the protected timestamp on database db +release id=2 +---- + +translate database=db +---- +/Table/5{6-7} num_replicas=7 num_voters=5 +/Table/5{7-8} num_replicas=7 + +# Create an index on t1 to ensure that subzones also see protected timestamps. +exec-sql +CREATE INDEX idx ON db.t1(id); +ALTER INDEX db.t1@idx CONFIGURE ZONE USING gc.ttlseconds = 1; +---- + +protect database=db table=t1 id=3 ts=3 +---- + +translate database=db +---- +/Table/56{-/2} num_replicas=7 num_voters=5 pts=[3] +/Table/56/{2-3} ttl_seconds=1 num_replicas=7 num_voters=5 pts=[3] +/Table/5{6/3-7} num_replicas=7 num_voters=5 pts=[3] +/Table/5{7-8} num_replicas=7 + diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/protectedts b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/protectedts new file mode 100644 index 000000000000..b42e8bb94911 --- /dev/null +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/tenant/protectedts @@ -0,0 +1,83 @@ +# Create a database with some tables and write protected timestamps on the +# tables and database. Check that span configurations are as we expect. + +exec-sql +CREATE DATABASE db; +CREATE TABLE db.t1(id INT); +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' +---- +57 + +# We only 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/5{7-8} 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; +---- + +# Write a protected timestamp on t1. +protect database=db table=t1 id=1 ts=1 +---- + +translate database=db +---- +/Tenant/10/Table/5{6-7} num_replicas=7 num_voters=5 pts=[1] +/Tenant/10/Table/5{7-8} num_replicas=7 + +# Write a protected timestamp on db, so we should see it on both t1 and t2. +protect database=db id=2 ts=2 +---- + +translate database=db +---- +/Tenant/10/Table/5{6-7} num_replicas=7 num_voters=5 pts=[1 2] +/Tenant/10/Table/5{7-8} num_replicas=7 pts=[2] + +# Release the protected timestamp on table t1 +release id=1 +---- + +translate database=db +---- +/Tenant/10/Table/5{6-7} num_replicas=7 num_voters=5 pts=[2] +/Tenant/10/Table/5{7-8} num_replicas=7 pts=[2] + +# Release the protected timestamp on database db +release id=2 +---- + +translate database=db +---- +/Tenant/10/Table/5{6-7} num_replicas=7 num_voters=5 +/Tenant/10/Table/5{7-8} num_replicas=7 + +# Create an index on t1 to ensure that subzones also see protected timestamps. +exec-sql +CREATE INDEX idx ON db.t1(id); +ALTER INDEX db.t1@idx CONFIGURE ZONE USING gc.ttlseconds = 1; +---- + +protect database=db table=t1 id=3 ts=3 +---- + +translate database=db +---- +/Tenant/10/Table/56{-/2} num_replicas=7 num_voters=5 pts=[3] +/Tenant/10/Table/56/{2-3} ttl_seconds=1 num_replicas=7 num_voters=5 pts=[3] +/Tenant/10/Table/5{6/3-7} num_replicas=7 num_voters=5 pts=[3] +/Tenant/10/Table/5{7-8} num_replicas=7 diff --git a/pkg/roachpb/span_config.proto b/pkg/roachpb/span_config.proto index 762a01887fae..61e220b2de45 100644 --- a/pkg/roachpb/span_config.proto +++ b/pkg/roachpb/span_config.proto @@ -14,6 +14,7 @@ option go_package = "roachpb"; import "roachpb/data.proto"; import "gogoproto/gogo.proto"; +import "util/hlc/timestamp.proto"; // TODO(irfansharif): We could have the proto definitions in pkg/config/zonepb // use these messages instead of duplicating everything. @@ -140,6 +141,11 @@ message SpanConfig { // preferred option to least. The first preference that an existing replica of // a range matches will take priority for the lease. repeated LeasePreference lease_preferences = 9 [(gogoproto.nullable) = false]; + + // ProtectedTimestamps captures all the protected timestamp records that apply + // to the range. The timestamp values represent the timestamps after which + // data will be protected from GC, if the protection succeeds. + repeated util.hlc.Timestamp protected_timestamps = 10 [(gogoproto.nullable) = false]; } // SpanConfigEntry ties a span to its corresponding config. diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 06cc4b4446b2..6ff6bdcea54b 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -458,7 +458,7 @@ func makeTenantSQLServerArgs( if err != nil { panic(err) } - protectedTSProvider = dummyProtectedTSProvider{pp} + protectedTSProvider = dummyProtectedTSProvider{Provider: pp, st: st} } recorder := status.NewMetricsRecorder(clock, nil, rpcContext, nil, st) diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 39eb3cfe3783..638790974db5 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -28,6 +28,7 @@ import ( "github.com/cenkalti/backoff" circuit "github.com/cockroachdb/circuitbreaker" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip" @@ -502,9 +503,15 @@ func (ts *TestServer) Start(ctx context.Context) error { type dummyProtectedTSProvider struct { protectedts.Provider + st *cluster.Settings } -func (d dummyProtectedTSProvider) Protect(context.Context, *kv.Txn, *ptpb.Record) error { +func (d dummyProtectedTSProvider) Protect( + ctx context.Context, txn *kv.Txn, rec *ptpb.Record, +) error { + if d.st.Version.IsActive(ctx, clusterversion.AlterSystemProtectedTimestampAddColumn) { + return d.Provider.Protect(ctx, txn, rec) + } return errors.New("fake protectedts.Provider") } diff --git a/pkg/spanconfig/spanconfigprotectedts/protectedts_table_reader.go b/pkg/spanconfig/spanconfigprotectedts/protectedts_table_reader.go index bc3d16e53a5f..ae5edf2f386b 100644 --- a/pkg/spanconfig/spanconfigprotectedts/protectedts_table_reader.go +++ b/pkg/spanconfig/spanconfigprotectedts/protectedts_table_reader.go @@ -83,11 +83,12 @@ var _ spanconfig.ProtectedTimestampTableReader = &ProtectedTimestampTableReader{ // `ptsRecordSystemTable`. func New( ctx context.Context, ptsRecordSystemTable string, ie sqlutil.InternalExecutor, txn *kv.Txn, -) (*ProtectedTimestampTableReader, error) { +) (spanconfig.ProtectedTimestampTableReader, error) { reader := &ProtectedTimestampTableReader{protections: make(map[key][]hlc.Timestamp)} if err := reader.loadProtectedTimestampRecords(ctx, ptsRecordSystemTable, ie, txn); err != nil { return nil, err } + fmt.Printf("this is the mapping: %+v\n\n", reader.protections) return reader, nil } diff --git a/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel b/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel index 5cb708f9738f..ab5a6dc7ba39 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel +++ b/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//pkg/kv", "//pkg/roachpb:with-mocks", "//pkg/spanconfig", + "//pkg/spanconfig/spanconfigprotectedts", "//pkg/sql", "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", diff --git a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go index f53b38ad7bac..e39d9cf4d385 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go +++ b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" + "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigprotectedts" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -69,6 +70,14 @@ func (s *SQLTranslator) Translate( // attempts. entries = entries[:0] + // Construct an in-memory view of the system.protected_ts_records table to + // populate the protected timestamp field on the emitted span configs. + ptsTableReader, err := spanconfigprotectedts.New(ctx, + spanconfigprotectedts.ProdSystemProtectedTimestampTable, s.execCfg.InternalExecutor, txn) + if err != nil { + return errors.Wrap(err, "failed to initialize protectedts table reader") + } + // 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 to not generate redundant entries. @@ -95,7 +104,7 @@ func (s *SQLTranslator) Translate( // For every unique leaf ID, generate span configurations. for _, leafID := range leafIDs { - translatedEntries, err := s.generateSpanConfigurations(ctx, leafID, txn, descsCol) + translatedEntries, err := s.generateSpanConfigurations(ctx, leafID, txn, descsCol, ptsTableReader) if err != nil { return err } @@ -126,7 +135,11 @@ var descLookupFlags = tree.CommonLookupFlags{ // ID. The ID must belong to an object that has a span configuration associated // with it, i.e, it should either belong to a table or a named zone. func (s *SQLTranslator) generateSpanConfigurations( - ctx context.Context, id descpb.ID, txn *kv.Txn, descsCol *descs.Collection, + ctx context.Context, + id descpb.ID, + txn *kv.Txn, + descsCol *descs.Collection, + ptsTableReader spanconfig.ProtectedTimestampTableReader, ) (entries []roachpb.SpanConfigEntry, err error) { if zonepb.IsNamedZoneID(id) { return s.generateSpanConfigurationsForNamedZone(ctx, txn, id) @@ -150,7 +163,7 @@ func (s *SQLTranslator) generateSpanConfigurations( ) } - return s.generateSpanConfigurationsForTable(ctx, txn, desc) + return s.generateSpanConfigurationsForTable(ctx, txn, desc, ptsTableReader) } // generateSpanConfigurationsForNamedZone expects an ID corresponding to a named @@ -214,11 +227,55 @@ func (s *SQLTranslator) generateSpanConfigurationsForNamedZone( return entries, nil } +// setProtectedTimestampsForTable aggregates the protected timestamps that apply +// to the table and sets the field in the table's generated span configs. +// This includes protected timestamps that apply to the table, and its parent +// database. +// +// This method mutates the passed in `entries`. +func setProtectedTimestampsForTable( + entries *[]roachpb.SpanConfigEntry, + desc catalog.Descriptor, + ptsTableReader spanconfig.ProtectedTimestampTableReader, +) { + protectedTimestamps := make([]hlc.Timestamp, 0) + // Get protections that apply directly to the table. + protectedTimestamps = append(protectedTimestamps, + ptsTableReader.GetProtectedTimestampsForSchemaObject(desc.GetID())...) + + // Get protections that apply to the database. + protectedTimestamps = append(protectedTimestamps, + ptsTableReader.GetProtectedTimestampsForSchemaObject(desc.GetParentID())...) + + if len(protectedTimestamps) == 0 { + return + } + + for i := range *entries { + (*entries)[i].Config.ProtectedTimestamps = protectedTimestamps + } +} + +// hydrateSpanConfigurationsForTable hydrates fields in a table's span +// configurations that are not derived from the table's zone configuration. +// +// This method mutates the passed in `entries`. +func hydrateSpanConfigurationsForTable( + entries *[]roachpb.SpanConfigEntry, + desc catalog.Descriptor, + ptsTableReader spanconfig.ProtectedTimestampTableReader, +) { + setProtectedTimestampsForTable(entries, desc, ptsTableReader) +} + // generateSpanConfigurationsForTable generates the span configurations // corresponding to the given tableID. It uses a transactional view of // system.zones and system.descriptors to do so. func (s *SQLTranslator) generateSpanConfigurationsForTable( - ctx context.Context, txn *kv.Txn, desc catalog.Descriptor, + ctx context.Context, + txn *kv.Txn, + desc catalog.Descriptor, + ptsTableReader spanconfig.ProtectedTimestampTableReader, ) ([]roachpb.SpanConfigEntry, error) { if desc.DescriptorType() != catalog.Table { return nil, errors.AssertionFailedf( @@ -271,6 +328,7 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( }) } + hydrateSpanConfigurationsForTable(&entries, desc, ptsTableReader) return entries, nil // TODO(irfansharif): There's an attack vector here that we haven't @@ -355,6 +413,7 @@ func (s *SQLTranslator) generateSpanConfigurationsForTable( }, ) } + hydrateSpanConfigurationsForTable(&entries, desc, ptsTableReader) return entries, nil } diff --git a/pkg/spanconfig/spanconfigtestutils/utils.go b/pkg/spanconfig/spanconfigtestutils/utils.go index 4afe6d7c5a2a..cd7f2bd35653 100644 --- a/pkg/spanconfig/spanconfigtestutils/utils.go +++ b/pkg/spanconfig/spanconfigtestutils/utils.go @@ -15,6 +15,8 @@ import ( "fmt" "reflect" "regexp" + "sort" + "strconv" "strings" "testing" @@ -236,6 +238,19 @@ func PrintSpanConfigDiffedAgainstDefaults(conf roachpb.SpanConfig) string { if !reflect.DeepEqual(conf.LeasePreferences, defaultConf.LeasePreferences) { diffs = append(diffs, fmt.Sprintf("lease_preferences=%v", conf.VoterConstraints)) } + if !reflect.DeepEqual(conf.ProtectedTimestamps, defaultConf.ProtectedTimestamps) { + sort.Slice(conf.ProtectedTimestamps, func(i, j int) bool { + return conf.ProtectedTimestamps[i].Less(conf.ProtectedTimestamps[j]) + }) + var s string + for i, pts := range conf.ProtectedTimestamps { + s += strconv.Itoa(int(pts.WallTime)) + if i != len(conf.ProtectedTimestamps)-1 { + s += " " + } + } + diffs = append(diffs, fmt.Sprintf("pts=[%s]", s)) + } return strings.Join(diffs, " ") }