From 2528c3867c5562d680891f8c394d5be1b3754d3f Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Thu, 23 Dec 2021 13:47:23 -0500 Subject: [PATCH] {backup,changefeed,streaming}ccl: start populating pts target This change touches all jobs that create a protected timestamp record before calling `Protect`. Previously, the created record would contain the spans that the record was going to protect. With this change, the record will also populate the `Target` field on `ptpb.Record` with the object it is going to protect. The `Target` field is a proto message defined in `ptpb.Target` and can be one of: - Cluster - Tenants - Schema object (database or table) For backups, this target field is determined based on the targets passed in by the user via the `BACKUP ` query. For changefeeds, this target is the group of tables on which the changefeed is being started + `system.descriptors` table. For the streaming job, this target is the tenant that is being streamed. This change does not touch any test files that create a raw `ptpb.Record` for testing purposes. That will come in a follow up PR where we actually teach `Protect` to validate and make use of this `Target` field. A test for how backup chooses its target will also come in a follow up PR once Protect actually writes the encoded protobuf target field to the underlying system table. Informs: #73727 Release note: None --- pkg/ccl/backupccl/BUILD.bazel | 1 + pkg/ccl/backupccl/backup_planning.go | 55 ++++++++++++++--- pkg/ccl/backupccl/schedule_pts_chaining.go | 36 ++++++----- pkg/ccl/changefeedccl/changefeed.go | 20 +++++- pkg/ccl/changefeedccl/changefeed_stmt.go | 7 +-- .../streamingccl/streamproducer/BUILD.bazel | 1 + .../streamproducer/producer_job_test.go | 4 +- .../streamproducer/stream_lifetime.go | 7 ++- pkg/jobs/jobsprotectedts/jobs_protected_ts.go | 9 ++- .../jobsprotectedts/jobs_protected_ts_test.go | 8 ++- pkg/kv/kvserver/protectedts/ptpb/BUILD.bazel | 5 ++ .../kvserver/protectedts/ptpb/protectedts.go | 34 +++++++++++ .../protectedts/ptpb/protectedts.proto | 61 +++++++++++-------- 13 files changed, 181 insertions(+), 67 deletions(-) create mode 100644 pkg/kv/kvserver/protectedts/ptpb/protectedts.go diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index c0231ff59796..eeb21b94581e 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -56,6 +56,7 @@ go_library( "//pkg/kv/kvserver/batcheval", "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/protectedts", + "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/roachpb:with-mocks", "//pkg/scheduledjobs", "//pkg/security", diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 51f7104ab7ff..194bc8c8a039 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" "github.com/cockroachdb/cockroach/pkg/keys" "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/scheduledjobs" "github.com/cockroachdb/cockroach/pkg/security" @@ -858,8 +859,7 @@ func backupPlanHook( jobID := p.ExecCfg().JobRegistry.MakeJobID() if err := protectTimestampForBackup( - ctx, p, plannerTxn, jobID, backupManifest.Spans, - backupManifest.StartTime, endTime, backupDetails, + ctx, p, plannerTxn, jobID, backupManifest, backupDetails, ); err != nil { return err } @@ -1204,25 +1204,62 @@ func makeNewEncryptionOptions( return encryptionOptions, encryptionInfo, nil } +func getProtectedTimestampTargetForBackup(backupManifest BackupManifest) *ptpb.Target { + if backupManifest.DescriptorCoverage == tree.AllDescriptors { + return ptpb.MakeClusterTarget() + } + + if len(backupManifest.Tenants) > 0 { + tenantID := make([]roachpb.TenantID, 0) + for _, tenant := range backupManifest.Tenants { + tenantID = append(tenantID, roachpb.MakeTenantID(tenant.ID)) + } + return ptpb.MakeTenantsTarget(tenantID) + } + + // ResolvedCompleteDBs contains all the "complete" databases being backed up. + // + // This includes explicit `BACKUP DATABASE` targets as well as expansions as a + // result of `BACKUP TABLE db.*`. In both cases we want to write a protected + // timestamp record that covers the entire database. + if len(backupManifest.CompleteDbs) > 0 { + return ptpb.MakeSchemaObjectsTarget(backupManifest.CompleteDbs) + } + + // At this point we are dealing with a `BACKUP TABLE`, so we write a protected + // timestamp record on each table being backed up. + tableIDs := make(descpb.IDs, 0) + for _, desc := range backupManifest.Descriptors { + t, _, _, _ := descpb.FromDescriptorWithMVCCTimestamp(&desc, hlc.Timestamp{}) + if t != nil { + tableIDs = append(tableIDs, t.GetID()) + } + } + return ptpb.MakeSchemaObjectsTarget(tableIDs) +} + func protectTimestampForBackup( ctx context.Context, p sql.PlanHookState, txn *kv.Txn, jobID jobspb.JobID, - spans []roachpb.Span, - startTime, endTime hlc.Timestamp, + backupManifest BackupManifest, backupDetails jobspb.BackupDetails, ) error { if backupDetails.ProtectedTimestampRecord == nil { return nil } - if len(spans) > 0 { - tsToProtect := endTime - if !startTime.IsEmpty() { - tsToProtect = startTime + if len(backupManifest.Spans) > 0 { + tsToProtect := backupManifest.EndTime + if !backupManifest.StartTime.IsEmpty() { + tsToProtect = backupManifest.StartTime } + + // Resolve the target that the PTS record will protect as part of this + // backup. + target := getProtectedTimestampTargetForBackup(backupManifest) rec := jobsprotectedts.MakeRecord(*backupDetails.ProtectedTimestampRecord, int64(jobID), - tsToProtect, spans, jobsprotectedts.Jobs) + tsToProtect, backupManifest.Spans, jobsprotectedts.Jobs, target) err := p.ExecCfg().ProtectedTimestampProvider.Protect(ctx, txn, rec) if err != nil { return err diff --git a/pkg/ccl/backupccl/schedule_pts_chaining.go b/pkg/ccl/backupccl/schedule_pts_chaining.go index c4632ca3dbf6..2edb3fc1dc37 100644 --- a/pkg/ccl/backupccl/schedule_pts_chaining.go +++ b/pkg/ccl/backupccl/schedule_pts_chaining.go @@ -17,7 +17,8 @@ import ( "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/kv/kvserver/protectedts/ptpb" + roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/scheduledjobs" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" @@ -145,22 +146,22 @@ func manageFullBackupPTSChaining( return err } - // Resolve the spans that need to be protected on this execution of the + // Resolve the target that needs to be protected on this execution of the // scheduled backup. - spansToProtect, err := getSpansProtectedByBackup(ctx, backupDetails, txn, exec) + targetToProtect, deprecatedSpansToProtect, err := getTargetProtectedByBackup(ctx, backupDetails, txn, exec) if err != nil { return errors.Wrap(err, "getting spans to protect") } - // Protect the spans after the EndTime of the current backup. We do not need + // Protect the target after the EndTime of the current backup. We do not need // to verify this new record as we have a record written by the backup during - // planning, already protecting these spans after EndTime. + // planning, already protecting this target after EndTime. // // Since this record will be stored on the incremental schedule, we use the // inc schedule ID as the records' Meta. This ensures that even if the full // schedule is dropped, the reconciliation job will not release the pts // record stored on the inc schedule, and the chaining will continue. - ptsRecord, err := protectTimestampRecordForSchedule(ctx, spansToProtect, + ptsRecord, err := protectTimestampRecordForSchedule(ctx, targetToProtect, deprecatedSpansToProtect, backupDetails.EndTime, incSj.ScheduleID(), exec, txn) if err != nil { return errors.Wrap(err, "protect and verify pts record for schedule") @@ -213,36 +214,33 @@ func manageIncrementalBackupPTSChaining( return err } -func getSpansProtectedByBackup( +func getTargetProtectedByBackup( ctx context.Context, backupDetails jobspb.BackupDetails, txn *kv.Txn, exec *sql.ExecutorConfig, -) ([]roachpb.Span, error) { +) (target *ptpb.Target, deprecatedSpans []roachpb.Span, err error) { if backupDetails.ProtectedTimestampRecord == nil { - return nil, nil + return nil, nil, nil } ptsRecord, err := exec.ProtectedTimestampProvider.GetRecord(ctx, txn, *backupDetails.ProtectedTimestampRecord) if err != nil { - return nil, err + return nil, nil, err } - return ptsRecord.DeprecatedSpans, nil + return ptsRecord.Target, ptsRecord.DeprecatedSpans, nil } func protectTimestampRecordForSchedule( ctx context.Context, - spansToProtect []roachpb.Span, + targetToProtect *ptpb.Target, + deprecatedSpansToProtect roachpb.Spans, tsToProtect hlc.Timestamp, scheduleID int64, exec *sql.ExecutorConfig, txn *kv.Txn, ) (uuid.UUID, error) { - var protectedtsID uuid.UUID - if len(spansToProtect) == 0 { - return protectedtsID, nil - } - protectedtsID = uuid.MakeV4() - rec := jobsprotectedts.MakeRecord(protectedtsID, scheduleID, tsToProtect, spansToProtect, - jobsprotectedts.Schedules) + protectedtsID := uuid.MakeV4() + rec := jobsprotectedts.MakeRecord(protectedtsID, scheduleID, tsToProtect, deprecatedSpansToProtect, + jobsprotectedts.Schedules, targetToProtect) return protectedtsID, exec.ProtectedTimestampProvider.Protect(ctx, txn, rec) } diff --git a/pkg/ccl/changefeedccl/changefeed.go b/pkg/ccl/changefeedccl/changefeed.go index dcf80c62a0ea..544900e7f5fb 100644 --- a/pkg/ccl/changefeedccl/changefeed.go +++ b/pkg/ccl/changefeedccl/changefeed.go @@ -17,7 +17,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" + "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/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -64,12 +66,26 @@ func createProtectedTimestampRecord( progress.ProtectedTimestampRecord = uuid.MakeV4() log.VEventf(ctx, 2, "creating protected timestamp %v at %v", progress.ProtectedTimestampRecord, resolved) - spansToProtect := makeSpansToProtect(codec, targets) + deprecatedSpansToProtect := makeSpansToProtect(codec, targets) + targetToProtect := makeTargetToProtect(targets) rec := jobsprotectedts.MakeRecord( - progress.ProtectedTimestampRecord, int64(jobID), resolved, spansToProtect, jobsprotectedts.Jobs) + progress.ProtectedTimestampRecord, int64(jobID), resolved, deprecatedSpansToProtect, + jobsprotectedts.Jobs, targetToProtect) return pts.Protect(ctx, txn, rec) } +func makeTargetToProtect(targets jobspb.ChangefeedTargets) *ptpb.Target { + // NB: We add 1 because we're also going to protect system.descriptors. + // We protect system.descriptors because a changefeed needs all of the history + // of table descriptors to version data. + tablesToProtect := make(descpb.IDs, 0, len(targets)+1) + for t := range targets { + tablesToProtect = append(tablesToProtect, t) + } + tablesToProtect = append(tablesToProtect, keys.DescriptorTableID) + return ptpb.MakeSchemaObjectsTarget(tablesToProtect) +} + func makeSpansToProtect(codec keys.SQLCodec, targets jobspb.ChangefeedTargets) []roachpb.Span { // NB: We add 1 because we're also going to protect system.descriptors. // We protect system.descriptors because a changefeed needs all of the history diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index 3cef5eb5b3fb..44fe5e669db0 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -31,7 +31,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -415,16 +414,16 @@ func changefeedPlanHook( { var protectedTimestampID uuid.UUID - var spansToProtect []roachpb.Span var ptr *ptpb.Record shouldProtectTimestamp := initialScanFromOptions(details.Opts) && p.ExecCfg().Codec.ForSystemTenant() if shouldProtectTimestamp { protectedTimestampID = uuid.MakeV4() - spansToProtect = makeSpansToProtect(p.ExecCfg().Codec, details.Targets) + deprecatedSpansToProtect := makeSpansToProtect(p.ExecCfg().Codec, details.Targets) + targetToProtect := makeTargetToProtect(details.Targets) progress.GetChangefeed().ProtectedTimestampRecord = protectedTimestampID ptr = jobsprotectedts.MakeRecord(protectedTimestampID, int64(jobID), statementTime, - spansToProtect, jobsprotectedts.Jobs) + deprecatedSpansToProtect, jobsprotectedts.Jobs, targetToProtect) } jr := jobs.Record{ diff --git a/pkg/ccl/streamingccl/streamproducer/BUILD.bazel b/pkg/ccl/streamingccl/streamproducer/BUILD.bazel index de7efa3e7656..fadd05000078 100644 --- a/pkg/ccl/streamingccl/streamproducer/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamproducer/BUILD.bazel @@ -24,6 +24,7 @@ go_library( "//pkg/kv", "//pkg/kv/kvclient/rangefeed:with-mocks", "//pkg/kv/kvserver/protectedts", + "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/roachpb:with-mocks", "//pkg/security", "//pkg/server/telemetry", diff --git a/pkg/ccl/streamingccl/streamproducer/producer_job_test.go b/pkg/ccl/streamingccl/streamproducer/producer_job_test.go index ac64113425fa..bea7111c3a8c 100644 --- a/pkg/ccl/streamingccl/streamproducer/producer_job_test.go +++ b/pkg/ccl/streamingccl/streamproducer/producer_job_test.go @@ -151,9 +151,11 @@ func TestStreamReplicationProducerJob(t *testing.T) { } runJobWithProtectedTimestamp := func(ptsID uuid.UUID, ts hlc.Timestamp, jr jobs.Record) error { return source.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + deprecatedTenantSpan := roachpb.Spans{*makeTenantSpan(30)} + tenantTarget := ptpb.MakeTenantsTarget([]roachpb.TenantID{roachpb.MakeTenantID(30)}) if err := ptp.Protect(ctx, txn, jobsprotectedts.MakeRecord(ptsID, int64(jr.JobID), ts, - []roachpb.Span{*makeTenantSpan(30)}, jobsprotectedts.Jobs)); err != nil { + deprecatedTenantSpan, jobsprotectedts.Jobs, tenantTarget)); err != nil { return err } _, err := registry.CreateAdoptableJobWithTxn(ctx, jr, jr.JobID, txn) diff --git a/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go b/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go index 824d243d2245..3143570bbf0d 100644 --- a/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go +++ b/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go @@ -18,6 +18,7 @@ import ( "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/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -129,8 +130,12 @@ func startReplicationStreamJob( statementTime := hlc.Timestamp{ WallTime: evalCtx.GetStmtTimestamp().UnixNano(), } + + deprecatedSpansToProtect := roachpb.Spans{*makeTenantSpan(tenantID)} + targetToProtect := ptpb.MakeTenantsTarget([]roachpb.TenantID{roachpb.MakeTenantID(tenantID)}) + pts := jobsprotectedts.MakeRecord(ptsID, int64(jr.JobID), statementTime, - []roachpb.Span{*makeTenantSpan(tenantID)}, jobsprotectedts.Jobs) + deprecatedSpansToProtect, jobsprotectedts.Jobs, targetToProtect) if err := ptp.Protect(evalCtx.Ctx(), txn, pts); err != nil { return streaming.InvalidStreamID, err diff --git a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go index 4219a1d9f270..5da0493f0651 100644 --- a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go +++ b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go @@ -90,12 +90,16 @@ func MakeStatusFunc( // MakeRecord makes a protected timestamp record to protect a timestamp on // behalf of this job. +// +// TODO(adityamaru): In 22.2 stop passing `deprecatedSpans` since PTS records +// will stop protecting key spans. func MakeRecord( recordID uuid.UUID, metaID int64, tsToProtect hlc.Timestamp, - spans []roachpb.Span, + deprecatedSpans []roachpb.Span, metaType MetaType, + target *ptpb.Target, ) *ptpb.Record { return &ptpb.Record{ ID: recordID.GetBytesMut(), @@ -103,7 +107,8 @@ func MakeRecord( Mode: ptpb.PROTECT_AFTER, MetaType: metaTypes[metaType], Meta: encodeID(metaID), - DeprecatedSpans: spans, + DeprecatedSpans: deprecatedSpans, + Target: target, } } diff --git a/pkg/jobs/jobsprotectedts/jobs_protected_ts_test.go b/pkg/jobs/jobsprotectedts/jobs_protected_ts_test.go index 09fd1ac58c87..2ab4f5bb5775 100644 --- a/pkg/jobs/jobsprotectedts/jobs_protected_ts_test.go +++ b/pkg/jobs/jobsprotectedts/jobs_protected_ts_test.go @@ -79,8 +79,10 @@ func TestJobsProtectedTimestamp(t *testing.T) { if j, err = jr.CreateJobWithTxn(ctx, mkJobRec(), jobID, txn); err != nil { return err } + deprecatedSpansToProtect := roachpb.Spans{{Key: keys.MinKey, EndKey: keys.MaxKey}} + targetToProtect := ptpb.MakeClusterTarget() rec = jobsprotectedts.MakeRecord(uuid.MakeV4(), int64(jobID), ts, - []roachpb.Span{{Key: keys.MinKey, EndKey: keys.MaxKey}}, jobsprotectedts.Jobs) + deprecatedSpansToProtect, jobsprotectedts.Jobs, targetToProtect) return ptp.Protect(ctx, txn, rec) })) return j, rec @@ -163,8 +165,10 @@ func TestSchedulesProtectedTimestamp(t *testing.T) { require.NoError(t, s0.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) (err error) { sj = mkScheduledJobRec(scheduleLabel) require.NoError(t, sj.Create(ctx, s0.InternalExecutor().(sqlutil.InternalExecutor), txn)) + deprecatedSpansToProtect := roachpb.Spans{{Key: keys.MinKey, EndKey: keys.MaxKey}} + targetToProtect := ptpb.MakeClusterTarget() rec = jobsprotectedts.MakeRecord(uuid.MakeV4(), sj.ScheduleID(), ts, - []roachpb.Span{{Key: keys.MinKey, EndKey: keys.MaxKey}}, jobsprotectedts.Schedules) + deprecatedSpansToProtect, jobsprotectedts.Schedules, targetToProtect) return ptp.Protect(ctx, txn, rec) })) return sj, rec diff --git a/pkg/kv/kvserver/protectedts/ptpb/BUILD.bazel b/pkg/kv/kvserver/protectedts/ptpb/BUILD.bazel index c8d7d38f24c1..d69978211086 100644 --- a/pkg/kv/kvserver/protectedts/ptpb/BUILD.bazel +++ b/pkg/kv/kvserver/protectedts/ptpb/BUILD.bazel @@ -34,7 +34,12 @@ go_proto_library( go_library( name = "ptpb", + srcs = ["protectedts.go"], embed = [":ptpb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb", visibility = ["//visibility:public"], + deps = [ + "//pkg/roachpb:with-mocks", + "//pkg/sql/catalog/descpb", + ], ) diff --git a/pkg/kv/kvserver/protectedts/ptpb/protectedts.go b/pkg/kv/kvserver/protectedts/ptpb/protectedts.go new file mode 100644 index 000000000000..a3ea01697d91 --- /dev/null +++ b/pkg/kv/kvserver/protectedts/ptpb/protectedts.go @@ -0,0 +1,34 @@ +// 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 ptpb + +import ( + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" +) + +// MakeClusterTarget returns a target, which when used in a Record, will +// protect the entire keyspace of the cluster. +func MakeClusterTarget() *Target { + return &Target{&Target_Cluster{Cluster: &Target_ClusterTarget{}}} +} + +// MakeTenantsTarget returns a target, which when used in a Record, will +// protect the keyspace of all tenants in ids. +func MakeTenantsTarget(ids []roachpb.TenantID) *Target { + return &Target{&Target_Tenants{Tenants: &Target_TenantsTarget{IDs: ids}}} +} + +// MakeSchemaObjectsTarget returns a target, which when used in a Record, +// will protect the keyspace of all schema objects (database/table). +func MakeSchemaObjectsTarget(ids descpb.IDs) *Target { + return &Target{&Target_SchemaObjects{SchemaObjects: &Target_SchemaObjectsTarget{IDs: ids}}} +} diff --git a/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto b/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto index a8dd52c96b52..e4fb5fabe799 100644 --- a/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto +++ b/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto @@ -89,28 +89,6 @@ message Metadata { // Record corresponds to a protected timestamp. message Record { - message SchemaObjectsTarget { - // IDs are the descriptor IDs of the schema objects being protected by this - // Record. This field will only contain database and table IDs. - repeated uint32 ids = 1 [(gogoproto.customname) = "IDs", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; - } - - message TenantsTarget { - // IDs correspond to the tenant keyspacs being protected by this Record. - repeated roachpb.TenantID ids = 1 [(gogoproto.customname) = "IDs"]; - } - - message ClusterTarget { - // ClusterTarget indicates that all SQL state in the cluster is being - // protected by this Record. This includes all user defined schema objects, - // as well as system tables used to configure the cluster. In a system - // tenant this target will also protect all secondary tenant keyspaces that - // exist in it. - // - // Today, this target is only used by cluster backups. - } - // ID uniquely identifies this row. bytes id = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.Bytes", (gogoproto.customname) = "ID"]; @@ -147,11 +125,7 @@ message Record { // Target holds information about what this Record protects. The Record can // either protect the entire cluster, a subset of tenants, or individual // schema objects (database and table). - oneof target { - SchemaObjectsTarget schema_objects = 8; - TenantsTarget tenants = 9; - ClusterTarget cluster = 10; - } + Target target = 8; // next ID: 11 } @@ -161,3 +135,36 @@ message State { Metadata metadata = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; repeated Record records = 2 [(gogoproto.nullable) = false]; } + +// Target is the format of the message encoded in the target column of the +// system.protectedts_records table. +message Target { + message SchemaObjectsTarget { + // IDs are the descriptor IDs of the schema objects being protected by this + // Record. This field will only contain database and table IDs. + repeated uint32 ids = 1 [(gogoproto.customname) = "IDs", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; + } + + message TenantsTarget { + // IDs correspond to the tenant keyspaces being protected by this Record. + repeated roachpb.TenantID ids = 1 [(gogoproto.customname) = "IDs", + (gogoproto.nullable) = false]; + } + + message ClusterTarget { + // ClusterTarget indicates that all SQL state in the cluster is being + // protected by this Record. This includes all user defined schema objects, + // as well as system tables used to configure the cluster. In a system + // tenant this target will also protect all secondary tenant keyspaces that + // exist in it. + // + // Today, this target is only used by cluster backups. + } + + oneof union { + SchemaObjectsTarget schema_objects = 1; + TenantsTarget tenants = 2; + ClusterTarget cluster = 3; + } +}