From 665592590e67adba4c320523b0d126503066b04d Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Sun, 20 Feb 2022 15:50:40 -0500 Subject: [PATCH] spanconfigsqltranslator,jobsprotectedts: add `ignore_if_excluded_from_backup` to SpanConfig This change is a follow up to #75451 which taught ExportRequests to noop on ranges marked as `exclude_data_from_backup`. This change adds a field to the ProtectionPolicy in the SpanConfig. This field is set to true if the ProtectionPolicy can be ignored if the span it applies to has been marked to be excluded from backups i.e. the applied SpanConfig has `exclude_data_from_backup = true`. This field is currently only set to true when a protected timestamp record has been written by a backup job. This is to ensure that ProtectionPolicies written by non-backup users (CDC, streaming) on spans marked as `exclude_data_from_backup` are still respected when making GC decisions on the span. The work to teach the GC queue about this field will be done in a final follow up. Informs: #73536 Release note: None --- .../spanconfigsqltranslatorccl/BUILD.bazel | 2 + .../datadriven_test.go | 33 +++++++++++++++- .../testdata/exclude_data_from_backup | 24 +++++++++++- .../testdata/protectedts | 16 ++++---- pkg/jobs/jobsprotectedts/jobs_protected_ts.go | 32 +++++++++++++++- pkg/kv/kvserver/protectedts/protectedts.go | 4 +- pkg/roachpb/span_config.go | 12 ++++++ pkg/roachpb/span_config.proto | 11 ++++++ .../spanconfigsqltranslator/BUILD.bazel | 12 ++++++ .../spanconfigsqltranslator/main_test.go | 31 +++++++++++++++ .../protectedts_state_reader.go | 38 +++++++++++++------ .../protectedts_state_reader_test.go | 17 ++++++++- .../spanconfigsqltranslator/sqltranslator.go | 5 ++- pkg/spanconfig/spanconfigtestutils/utils.go | 8 ++-- 14 files changed, 214 insertions(+), 31 deletions(-) create mode 100644 pkg/spanconfig/spanconfigsqltranslator/main_test.go diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel index 56863d4679d9..977aea08e369 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/BUILD.bazel @@ -13,6 +13,8 @@ go_test( "//pkg/ccl/partitionccl", "//pkg/ccl/utilccl", "//pkg/config/zonepb", + "//pkg/jobs", + "//pkg/jobs/jobspb", "//pkg/jobs/jobsprotectedts", "//pkg/kv", "//pkg/kv/kvserver/protectedts", diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go index 84d106f5ca13..09daf1fa428a 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go @@ -19,10 +19,13 @@ 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/jobspb" "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils/spanconfigtestcluster" @@ -208,9 +211,37 @@ func TestDataDriven(t *testing.T) { var protectTS int d.ScanArgs(t, "record-id", &recordID) d.ScanArgs(t, "ts", &protectTS) - target := spanconfigtestutils.ParseProtectionTarget(t, d.Input) jobID := tenant.JobsRegistry().MakeJobID() + if d.HasArg("mk-job-with-type") { + var mkJobWithType string + d.ScanArgs(t, "mk-job-with-type", &mkJobWithType) + var rec jobs.Record + switch mkJobWithType { + case "backup": + rec = jobs.Record{ + Description: "backup", + Username: security.RootUserName(), + Details: jobspb.BackupDetails{}, + Progress: jobspb.BackupProgress{}, + } + default: + rec = jobs.Record{ + Description: "cdc", + Username: security.RootUserName(), + Details: jobspb.ChangefeedDetails{}, + Progress: jobspb.ChangefeedProgress{}, + } + } + require.NoError(t, tenant.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { + if _, err = tenant.JobsRegistry().CreateJobWithTxn(ctx, rec, jobID, txn); err != nil { + return err + } + return nil + })) + } + target := spanconfigtestutils.ParseProtectionTarget(t, d.Input) + require.NoError(t, tenant.ExecCfg().DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { require.Len(t, recordID, 1, diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup index 5e549f408456..9f8a6396e54f 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/exclude_data_from_backup @@ -41,12 +41,32 @@ translate database=db table=t2 ---- /Table/10{7-8} range default +# Write a protection record as a backup. +protect record-id=1 ts=1 mk-job-with-type=backup +descs 104 +---- + +# Write another protection record as a non-backup user. +protect record-id=2 ts=2 mk-job-with-type=cdc +descs 104 +---- + +# Translate to ensure that the ProtectionPolicy is set with +# `ignore_if_excluded_from_backup` for the record written by the backup only. +translate database=db +---- +/Table/10{6-7} protection_policies=[{ts: 1,ignore_if_excluded_from_backup: true} {ts: 2}] exclude_data_from_backup=true +/Table/10{7-8} protection_policies=[{ts: 1,ignore_if_excluded_from_backup: true} {ts: 2}] + # Alter table t1 to unmark its data ephemeral. exec-sql ALTER TABLE db.t1 SET (exclude_data_from_backup = false); ---- +release record-id=1 +---- + translate database=db ---- -/Table/10{6-7} range default -/Table/10{7-8} range default +/Table/10{6-7} protection_policies=[{ts: 2}] +/Table/10{7-8} protection_policies=[{ts: 2}] diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts index 5e6401befaf9..900dbe74c55f 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/testdata/protectedts @@ -28,7 +28,7 @@ descs 106 translate database=db ---- -/Table/10{6-7} num_replicas=7 num_voters=5 pts=[1] +/Table/10{6-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 1}] /Table/10{7-8} num_replicas=7 # Write a protected timestamp on db, so we should see it on both t1 and t2. @@ -38,8 +38,8 @@ descs 104 translate database=db ---- -/Table/10{6-7} num_replicas=7 num_voters=5 pts=[1 2] -/Table/10{7-8} num_replicas=7 pts=[2] +/Table/10{6-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 1} {ts: 2}] +/Table/10{7-8} num_replicas=7 protection_policies=[{ts: 2}] # Release the protected timestamp on table t1 release record-id=1 @@ -47,8 +47,8 @@ release record-id=1 translate database=db ---- -/Table/10{6-7} num_replicas=7 num_voters=5 pts=[2] -/Table/10{7-8} num_replicas=7 pts=[2] +/Table/10{6-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 2}] +/Table/10{7-8} num_replicas=7 protection_policies=[{ts: 2}] # Release the protected timestamp on database db release record-id=2 @@ -71,7 +71,7 @@ descs 106 translate database=db ---- -/Table/106{-/2} num_replicas=7 num_voters=5 pts=[3] -/Table/106/{2-3} ttl_seconds=1 num_replicas=7 num_voters=5 pts=[3] -/Table/10{6/3-7} num_replicas=7 num_voters=5 pts=[3] +/Table/106{-/2} num_replicas=7 num_voters=5 protection_policies=[{ts: 3}] +/Table/106/{2-3} ttl_seconds=1 num_replicas=7 num_voters=5 protection_policies=[{ts: 3}] +/Table/10{6/3-7} num_replicas=7 num_voters=5 protection_policies=[{ts: 3}] /Table/10{7-8} num_replicas=7 diff --git a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go index 5da0493f0651..b04fbf9adb6e 100644 --- a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go +++ b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go @@ -50,8 +50,38 @@ func GetMetaType(metaType MetaType) string { return metaTypes[metaType] } +// IsRecordWrittenByJobType returns true if the job implied with this value of +// meta is of type jobType. This method can be used to filter records written by +// a particular job type. +func IsRecordWrittenByJobType( + ctx context.Context, + jr *jobs.Registry, + txn *kv.Txn, + meta []byte, + metaType MetaType, + jobType jobspb.Type, +) (bool, error) { + switch metaType { + case Jobs: + jobID, err := decodeID(meta) + if err != nil { + return false, err + } + j, err := jr.LoadJobWithTxn(ctx, jobspb.JobID(jobID), txn) + if jobs.HasJobNotFoundError(err) { + return false, nil + } + if err != nil { + return false, err + } + payload := j.Payload() + return payload.Type() == jobType, nil + } + return false, nil +} + // MakeStatusFunc returns a function which determines whether the job or -// schedule implied with this value of meta should be removed by the reconciler. +// schedule implied with this value of meta should be removed by the ptreconciler. func MakeStatusFunc( jr *jobs.Registry, ie sqlutil.InternalExecutor, metaType MetaType, ) ptreconcile.StatusFunc { diff --git a/pkg/kv/kvserver/protectedts/protectedts.go b/pkg/kv/kvserver/protectedts/protectedts.go index 9124d1bd8968..10349f7e9aa9 100644 --- a/pkg/kv/kvserver/protectedts/protectedts.go +++ b/pkg/kv/kvserver/protectedts/protectedts.go @@ -92,10 +92,10 @@ type Storage interface { // passed txn remains safe for future use. Release(context.Context, *kv.Txn, uuid.UUID) error - // GetMetadata retreives the metadata with the provided Txn. + // GetMetadata retrieves the metadata with the provided Txn. GetMetadata(context.Context, *kv.Txn) (ptpb.Metadata, error) - // GetState retreives the entire state of protectedts.Storage with the + // GetState retrieves the entire state of protectedts.Storage with the // provided Txn. GetState(context.Context, *kv.Txn) (ptpb.State, error) diff --git a/pkg/roachpb/span_config.go b/pkg/roachpb/span_config.go index ad3ebc198a27..698234da8eab 100644 --- a/pkg/roachpb/span_config.go +++ b/pkg/roachpb/span_config.go @@ -96,6 +96,18 @@ func (c ConstraintsConjunction) String() string { return sb.String() } +// String implements the stringer interface. +func (p ProtectionPolicy) String() string { + var sb strings.Builder + sb.WriteString(fmt.Sprintf("{ts: %d", int(p.ProtectedTimestamp.WallTime))) + if p.IgnoreIfExcludedFromBackup { + sb.WriteString(fmt.Sprintf(",ignore_if_excluded_from_backup: %t", + p.IgnoreIfExcludedFromBackup)) + } + sb.WriteString("}") + return sb.String() +} + // TestingDefaultSpanConfig exports the default span config for testing purposes. func TestingDefaultSpanConfig() SpanConfig { return SpanConfig{ diff --git a/pkg/roachpb/span_config.proto b/pkg/roachpb/span_config.proto index b326da31cc2f..360d57a330e3 100644 --- a/pkg/roachpb/span_config.proto +++ b/pkg/roachpb/span_config.proto @@ -43,11 +43,22 @@ message GCPolicy { // applies over a given span. message ProtectionPolicy { option (gogoproto.equal) = true; + option (gogoproto.goproto_stringer) = false; option (gogoproto.populate) = true; // ProtectedTimestamp is a timestamp above which GC should not run, regardless // of the GC TTL. util.hlc.Timestamp protected_timestamp = 1 [(gogoproto.nullable) = false]; + + // IgnoreIfExcludedFromBackup is set to true if the ProtectionPolicy can be + // ignored if the span it applies to has been marked to be excluded from + // backups i.e. the applied SpanConfig has `exclude_data_from_backup = true`. + // This field is currently only set to true when a protected timestamp record + // has been written by a backup job. This is to ensure that ProtectionPolicies + // written by non-backup users (CDC, streaming) on spans marked as + // `exclude_data_from_backup` are still respected when making GC decisions on + // the span. + bool ignore_if_excluded_from_backup = 2; } // Constraint constrains the stores that a replica can be stored on. It diff --git a/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel b/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel index cf62bf4147d8..f8496c5e44f8 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel +++ b/pkg/spanconfig/spanconfigsqltranslator/BUILD.bazel @@ -10,6 +10,9 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/config/zonepb", + "//pkg/jobs", + "//pkg/jobs/jobspb", + "//pkg/jobs/jobsprotectedts", "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver/protectedts/ptpb", @@ -28,16 +31,25 @@ go_library( go_test( name = "spanconfigsqltranslator_test", srcs = [ + "main_test.go", "protectedts_state_reader_test.go", "sqltranslator_test.go", ], data = glob(["testdata/**"]), embed = [":spanconfigsqltranslator"], deps = [ + "//pkg/base", + "//pkg/jobs", "//pkg/jobs/jobsprotectedts", + "//pkg/kv", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/roachpb", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", "//pkg/sql/catalog/descpb", + "//pkg/testutils/serverutils", + "//pkg/testutils/testcluster", "//pkg/util/hlc", "//pkg/util/leaktest", "//pkg/util/log", diff --git a/pkg/spanconfig/spanconfigsqltranslator/main_test.go b/pkg/spanconfig/spanconfigsqltranslator/main_test.go new file mode 100644 index 000000000000..d6c5b5e90537 --- /dev/null +++ b/pkg/spanconfig/spanconfigsqltranslator/main_test.go @@ -0,0 +1,31 @@ +// Copyright 2022 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 spanconfigsqltranslator_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} + +//go:generate ../../util/leaktest/add-leaktest.sh *_test.go diff --git a/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader.go b/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader.go index 10b16c7b0015..bdcb566730ef 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader.go +++ b/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader.go @@ -13,9 +13,14 @@ package spanconfigsqltranslator import ( "context" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "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/sql/catalog/descpb" + "github.com/cockroachdb/errors" ) // protectedTimestampStateReader provides a target specific view of the @@ -31,15 +36,15 @@ type protectedTimestampStateReader struct { // protected timestamp records given the supplied ptpb.State. The ptpb.State is // the transactional state of the `system.protected_ts_records` table. func newProtectedTimestampStateReader( - _ context.Context, ptsState ptpb.State, -) *protectedTimestampStateReader { + ctx context.Context, jr *jobs.Registry, txn *kv.Txn, ptsState ptpb.State, +) (*protectedTimestampStateReader, error) { reader := &protectedTimestampStateReader{ schemaObjectProtections: make(map[descpb.ID][]roachpb.ProtectionPolicy), tenantProtections: make([]tenantProtectedTimestamps, 0), clusterProtections: make([]roachpb.ProtectionPolicy, 0), } - reader.loadProtectedTimestampRecords(ptsState) - return reader + err := reader.loadProtectedTimestampRecords(ctx, jr, txn, ptsState) + return reader, err } // getProtectionPoliciesForCluster returns all the protected timestamps that @@ -70,22 +75,32 @@ func (p *protectedTimestampStateReader) getProtectionPoliciesForSchemaObject( return p.schemaObjectProtections[descID] } -func (p *protectedTimestampStateReader) loadProtectedTimestampRecords(ptsState ptpb.State) { +func (p *protectedTimestampStateReader) loadProtectedTimestampRecords( + ctx context.Context, jr *jobs.Registry, txn *kv.Txn, ptsState ptpb.State, +) error { tenantProtections := make(map[roachpb.TenantID][]roachpb.ProtectionPolicy) for _, record := range ptsState.Records { + // TODO(adityamaru): When schedules start writing protected timestamp + // records extend this logic to also include those records. + isBackupProtectedTimestampRecord, err := jobsprotectedts.IsRecordWrittenByJobType(ctx, jr, txn, + record.Meta, jobsprotectedts.Jobs, jobspb.TypeBackup) + if err != nil { + return errors.Wrap(err, "failed to check if record was written by a BACKUP") + } + protectionPolicy := roachpb.ProtectionPolicy{ + ProtectedTimestamp: record.Timestamp, + IgnoreIfExcludedFromBackup: isBackupProtectedTimestampRecord, + } switch t := record.Target.GetUnion().(type) { case *ptpb.Target_Cluster: - p.clusterProtections = append(p.clusterProtections, - roachpb.ProtectionPolicy{ProtectedTimestamp: record.Timestamp}) + p.clusterProtections = append(p.clusterProtections, protectionPolicy) case *ptpb.Target_Tenants: for _, tenID := range t.Tenants.IDs { - tenantProtections[tenID] = append(tenantProtections[tenID], - roachpb.ProtectionPolicy{ProtectedTimestamp: record.Timestamp}) + tenantProtections[tenID] = append(tenantProtections[tenID], protectionPolicy) } case *ptpb.Target_SchemaObjects: for _, descID := range t.SchemaObjects.IDs { - p.schemaObjectProtections[descID] = append(p.schemaObjectProtections[descID], - roachpb.ProtectionPolicy{ProtectedTimestamp: record.Timestamp}) + p.schemaObjectProtections[descID] = append(p.schemaObjectProtections[descID], protectionPolicy) } } } @@ -94,4 +109,5 @@ func (p *protectedTimestampStateReader) loadProtectedTimestampRecords(ptsState p p.tenantProtections = append(p.tenantProtections, tenantProtectedTimestamps{tenantID: tenID, protections: tenantProtections}) } + return nil } diff --git a/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader_test.go b/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader_test.go index 0ff58332dc29..ef974ab1c41c 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader_test.go +++ b/pkg/spanconfig/spanconfigsqltranslator/protectedts_state_reader_test.go @@ -16,10 +16,14 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/base" + "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/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -31,6 +35,10 @@ func TestProtectedTimestampStateReader(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(context.Background()) + jr := s.JobRegistry().(*jobs.Registry) + mkRecordAndAddToState := func(state *ptpb.State, ts hlc.Timestamp, target *ptpb.Target) { recordID := uuid.MakeV4() rec := jobsprotectedts.MakeRecord(recordID, int64(1), ts, nil, /* deprecatedSpans */ @@ -64,7 +72,14 @@ func TestProtectedTimestampStateReader(t *testing.T) { protectTenants(state, ts(5), []roachpb.TenantID{roachpb.MakeTenantID(2)}) protectTenants(state, ts(6), []roachpb.TenantID{roachpb.MakeTenantID(2)}) - ptsStateReader := newProtectedTimestampStateReader(context.Background(), *state) + ctx := context.Background() + var ptsStateReader *protectedTimestampStateReader + require.NoError(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + var err error + ptsStateReader, err = newProtectedTimestampStateReader(context.Background(), jr, txn, *state) + require.NoError(t, err) + return nil + })) clusterTimestamps := ptsStateReader.getProtectionPoliciesForCluster() require.Len(t, clusterTimestamps, 1) require.Equal(t, []roachpb.ProtectionPolicy{{ProtectedTimestamp: ts(3)}}, clusterTimestamps) diff --git a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go index 2918547edb92..acdb3e6933d6 100644 --- a/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go +++ b/pkg/spanconfig/spanconfigsqltranslator/sqltranslator.go @@ -82,7 +82,10 @@ func (s *SQLTranslator) Translate( if err != nil { return errors.Wrap(err, "failed to get protected timestamp state") } - ptsStateReader := newProtectedTimestampStateReader(ctx, ptsState) + ptsStateReader, err := newProtectedTimestampStateReader(ctx, s.execCfg.JobRegistry, txn, ptsState) + if err != nil { + return errors.Wrap(err, "failed to create a ProtectedTimestampStateReader") + } // For every ID we want to translate, first expand it to descendant leaf // IDs that have span configurations associated for them. We also diff --git a/pkg/spanconfig/spanconfigtestutils/utils.go b/pkg/spanconfig/spanconfigtestutils/utils.go index 5ae42e7a398e..eab37c50713c 100644 --- a/pkg/spanconfig/spanconfigtestutils/utils.go +++ b/pkg/spanconfig/spanconfigtestutils/utils.go @@ -331,11 +331,11 @@ func PrintSpanConfigDiffedAgainstDefaults(conf roachpb.SpanConfig) string { rhs := conf.GCPolicy.ProtectionPolicies[j].ProtectedTimestamp return lhs.Less(rhs) }) - timestamps := make([]string, 0, len(conf.GCPolicy.ProtectionPolicies)) - for _, pts := range conf.GCPolicy.ProtectionPolicies { - timestamps = append(timestamps, strconv.Itoa(int(pts.ProtectedTimestamp.WallTime))) + protectionPolicies := make([]string, 0, len(conf.GCPolicy.ProtectionPolicies)) + for _, pp := range conf.GCPolicy.ProtectionPolicies { + protectionPolicies = append(protectionPolicies, pp.String()) } - diffs = append(diffs, fmt.Sprintf("pts=[%s]", strings.Join(timestamps, " "))) + diffs = append(diffs, fmt.Sprintf("protection_policies=[%s]", strings.Join(protectionPolicies, " "))) } if conf.ExcludeDataFromBackup != defaultConf.ExcludeDataFromBackup { diffs = append(diffs, fmt.Sprintf("exclude_data_from_backup=%v", conf.ExcludeDataFromBackup))