diff --git a/pkg/ccl/backupccl/schedule_pts_chaining.go b/pkg/ccl/backupccl/schedule_pts_chaining.go index 9008c99d8095..c4632ca3dbf6 100644 --- a/pkg/ccl/backupccl/schedule_pts_chaining.go +++ b/pkg/ccl/backupccl/schedule_pts_chaining.go @@ -226,7 +226,7 @@ func getSpansProtectedByBackup( return nil, err } - return ptsRecord.Spans, nil + return ptsRecord.DeprecatedSpans, nil } func protectTimestampRecordForSchedule( diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 947e6a094b4d..1d7aacd554d8 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -3553,8 +3553,8 @@ func TestChangefeedProtectedTimestamps(t *testing.T) { if ptr == nil { return errors.Errorf("expected protected timestamp") } - require.Equal(t, len(ptr.Spans), len(expectedKeys), ptr.Spans, expectedKeys) - for _, s := range ptr.Spans { + require.Equal(t, len(ptr.DeprecatedSpans), len(expectedKeys), ptr.DeprecatedSpans, expectedKeys) + for _, s := range ptr.DeprecatedSpans { require.Contains(t, expectedKeys, string(s.Key)) } return nil diff --git a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go index bddb8f46fd74..4219a1d9f270 100644 --- a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go +++ b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go @@ -98,12 +98,12 @@ func MakeRecord( metaType MetaType, ) *ptpb.Record { return &ptpb.Record{ - ID: recordID.GetBytesMut(), - Timestamp: tsToProtect, - Mode: ptpb.PROTECT_AFTER, - MetaType: metaTypes[metaType], - Meta: encodeID(metaID), - Spans: spans, + ID: recordID.GetBytesMut(), + Timestamp: tsToProtect, + Mode: ptpb.PROTECT_AFTER, + MetaType: metaTypes[metaType], + Meta: encodeID(metaID), + DeprecatedSpans: spans, } } diff --git a/pkg/kv/kvserver/client_protectedts_test.go b/pkg/kv/kvserver/client_protectedts_test.go index cf91982d881d..7b9fd21bf99b 100644 --- a/pkg/kv/kvserver/client_protectedts_test.go +++ b/pkg/kv/kvserver/client_protectedts_test.go @@ -144,7 +144,7 @@ func TestProtectedTimestamps(t *testing.T) { ID: uuid.MakeV4().GetBytes(), Timestamp: s0.Clock().Now(), Mode: ptpb.PROTECT_AFTER, - Spans: []roachpb.Span{ + DeprecatedSpans: []roachpb.Span{ { Key: startKey, EndKey: startKey.PrefixEnd(), diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index d91fb3c5d0ad..f83d0201d0da 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -3373,9 +3373,9 @@ func TestStrictGCEnforcement(t *testing.T) { tableSpan = roachpb.Span{Key: tableKey, EndKey: tableKey.PrefixEnd()} mkRecord = func() ptpb.Record { return ptpb.Record{ - ID: uuid.MakeV4().GetBytes(), - Timestamp: tenSecondsAgo.Add(-10*time.Second.Nanoseconds(), 0), - Spans: []roachpb.Span{tableSpan}, + ID: uuid.MakeV4().GetBytes(), + Timestamp: tenSecondsAgo.Add(-10*time.Second.Nanoseconds(), 0), + DeprecatedSpans: []roachpb.Span{tableSpan}, } } mkStaleTxn = func() *kv.Txn { diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache.go b/pkg/kv/kvserver/protectedts/ptcache/cache.go index 1a481cc35a71..b379cdf49871 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache.go @@ -259,8 +259,8 @@ func (c *Cache) upToDate(asOf hlc.Timestamp) bool { } func overlaps(r *ptpb.Record, sp roachpb.Span) bool { - for i := range r.Spans { - if r.Spans[i].Overlaps(sp) { + for i := range r.DeprecatedSpans { + if r.DeprecatedSpans[i].Overlaps(sp) { return true } } diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go index f2df3904fa6d..3e173e3e4826 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache_test.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache_test.go @@ -431,10 +431,10 @@ func protect( ) (r *ptpb.Record, createdAt hlc.Timestamp) { protectTS := s.Clock().Now() r = &ptpb.Record{ - ID: uuid.MakeV4().GetBytes(), - Timestamp: protectTS, - Mode: ptpb.PROTECT_AFTER, - Spans: spans, + ID: uuid.MakeV4().GetBytes(), + Timestamp: protectTS, + Mode: ptpb.PROTECT_AFTER, + DeprecatedSpans: spans, } ctx := context.Background() txn := s.DB().NewTxn(ctx, "test") diff --git a/pkg/kv/kvserver/protectedts/ptpb/BUILD.bazel b/pkg/kv/kvserver/protectedts/ptpb/BUILD.bazel index ec13933d796d..d5cb05be846e 100644 --- a/pkg/kv/kvserver/protectedts/ptpb/BUILD.bazel +++ b/pkg/kv/kvserver/protectedts/ptpb/BUILD.bazel @@ -8,6 +8,7 @@ proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/roachpb:roachpb_proto", + "//pkg/sql/catalog/descpb:descpb_proto", "//pkg/util/hlc:hlc_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", "@go_googleapis//google/api:annotations_proto", @@ -22,6 +23,7 @@ go_proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/roachpb:with-mocks", + "//pkg/sql/catalog/descpb", "//pkg/util/hlc", "//pkg/util/uuid", # keep "@com_github_gogo_protobuf//gogoproto", diff --git a/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto b/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto index 97afac134851..a8dd52c96b52 100644 --- a/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto +++ b/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto @@ -17,6 +17,7 @@ import "gogoproto/gogo.proto"; import "google/api/annotations.proto"; import "roachpb/data.proto"; import "util/hlc/timestamp.proto"; +import "sql/catalog/descpb/structured.proto"; // TODO(ajwerner): Consider splitting up Record into two pieces. It would @@ -88,6 +89,27 @@ 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"]; @@ -119,8 +141,19 @@ message Record { // change the Version of the subsystem. bool verified = 6; - // Spans are the spans which this Record protects. - repeated roachpb.Span spans = 7 [(gogoproto.nullable) = false]; + // DeprecatedSpans are the spans which this Record protects. + repeated roachpb.Span deprecated_spans = 7 [(gogoproto.nullable) = false]; + + // 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; + } + + // next ID: 11 } // State is the complete system state. diff --git a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go index ac0c40187531..1b54da94447c 100644 --- a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go +++ b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go @@ -79,7 +79,7 @@ func TestReconciler(t *testing.T) { Mode: ptpb.PROTECT_AFTER, MetaType: testTaskType, Meta: []byte(recMeta), - Spans: []roachpb.Span{ + DeprecatedSpans: []roachpb.Span{ {Key: keys.MinKey, EndKey: keys.MaxKey}, }, } diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage.go b/pkg/kv/kvserver/protectedts/ptstorage/storage.go index f75e9f80122b..1115801531a2 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/storage.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/storage.go @@ -78,7 +78,7 @@ func (p *storage) Protect(ctx context.Context, txn *kv.Txn, r *ptpb.Record) erro if txn == nil { return errNoTxn } - encodedSpans, err := protoutil.Marshal(&Spans{Spans: r.Spans}) + encodedSpans, err := protoutil.Marshal(&Spans{Spans: r.DeprecatedSpans}) if err != nil { // how can this possibly fail? return errors.Wrap(err, "failed to marshal spans") } @@ -95,10 +95,10 @@ func (p *storage) Protect(ctx context.Context, txn *kv.Txn, r *ptpb.Record) erro it, err := p.ex.QueryIteratorEx(ctx, "protectedts-protect", txn, sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, protectQuery, - s.maxSpans, s.maxBytes, len(r.Spans), + s.maxSpans, s.maxBytes, len(r.DeprecatedSpans), r.ID, r.Timestamp.AsOfSystemTime(), r.MetaType, meta, - len(r.Spans), encodedSpans) + len(r.DeprecatedSpans), encodedSpans) if err != nil { return errors.Wrapf(err, "failed to write record %v", r.ID) } @@ -115,10 +115,10 @@ func (p *storage) Protect(ctx context.Context, txn *kv.Txn, r *ptpb.Record) erro } if failed := *row[0].(*tree.DBool); failed { curNumSpans := int64(*row[1].(*tree.DInt)) - if s.maxSpans > 0 && curNumSpans+int64(len(r.Spans)) > s.maxSpans { + if s.maxSpans > 0 && curNumSpans+int64(len(r.DeprecatedSpans)) > s.maxSpans { return errors.WithHint( errors.Errorf("protectedts: limit exceeded: %d+%d > %d spans", curNumSpans, - len(r.Spans), s.maxSpans), + len(r.DeprecatedSpans), s.maxSpans), "SET CLUSTER SETTING kv.protectedts.max_spans to a higher value") } curBytes := int64(*row[2].(*tree.DInt)) @@ -271,7 +271,7 @@ func rowToRecord(ctx context.Context, row tree.Datums, r *ptpb.Record) error { if err := protoutil.Unmarshal([]byte(*row[4].(*tree.DBytes)), &spans); err != nil { return errors.Wrapf(err, "failed to unmarshal spans for %v", r.ID) } - r.Spans = spans.Spans + r.DeprecatedSpans = spans.Spans r.Verified = bool(*row[5].(*tree.DBool)) return nil } @@ -303,7 +303,7 @@ func validateRecordForProtect(r *ptpb.Record) error { if r.ID.GetUUID() == uuid.Nil { return errZeroID } - if len(r.Spans) == 0 { + if len(r.DeprecatedSpans) == 0 { return errEmptySpans } if len(r.Meta) > 0 && len(r.MetaType) == 0 { diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go index 6bd7aba2840f..e7e9a576359e 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go @@ -336,8 +336,8 @@ func (r releaseOp) run(ctx context.Context, t *testing.T, tCtx *testContext) { } tCtx.state.Version++ tCtx.state.NumRecords-- - tCtx.state.NumSpans -= uint64(len(rec.Spans)) - encoded, err := protoutil.Marshal(&ptstorage.Spans{Spans: rec.Spans}) + tCtx.state.NumSpans -= uint64(len(rec.DeprecatedSpans)) + encoded, err := protoutil.Marshal(&ptstorage.Spans{Spans: rec.DeprecatedSpans}) require.NoError(t, err) tCtx.state.TotalBytes -= uint64(len(encoded) + len(rec.Meta) + len(rec.MetaType)) } @@ -392,7 +392,7 @@ func (p protectOp) run(ctx context.Context, t *testing.T, tCtx *testContext) { tCtx.state.Records = append(tCtx.state.Records, tail...) tCtx.state.Version++ tCtx.state.NumRecords++ - tCtx.state.NumSpans += uint64(len(rec.Spans)) + tCtx.state.NumSpans += uint64(len(rec.DeprecatedSpans)) encoded, err := protoutil.Marshal(&ptstorage.Spans{Spans: p.spans}) require.NoError(t, err) tCtx.state.TotalBytes += uint64(len(encoded) + len(p.meta) + len(p.metaType)) @@ -503,12 +503,12 @@ func tableSpans(tableIDs ...uint32) []roachpb.Span { func newRecord(ts hlc.Timestamp, metaType string, meta []byte, spans ...roachpb.Span) ptpb.Record { return ptpb.Record{ - ID: uuid.MakeV4().GetBytes(), - Timestamp: ts, - Mode: ptpb.PROTECT_AFTER, - MetaType: metaType, - Meta: meta, - Spans: spans, + ID: uuid.MakeV4().GetBytes(), + Timestamp: ts, + Mode: ptpb.PROTECT_AFTER, + MetaType: metaType, + Meta: meta, + DeprecatedSpans: spans, } } diff --git a/pkg/kv/kvserver/protectedts/ptstorage/validate_test.go b/pkg/kv/kvserver/protectedts/ptstorage/validate_test.go index 905dd07117ff..99a5a296d003 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/validate_test.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/validate_test.go @@ -34,38 +34,38 @@ func TestValidateRecordForProtect(t *testing.T) { }{ { r: &ptpb.Record{ - ID: uuid.MakeV4().GetBytes(), - Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, - MetaType: "job", - Meta: []byte("junk"), - Spans: spans, + ID: uuid.MakeV4().GetBytes(), + Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, + MetaType: "job", + Meta: []byte("junk"), + DeprecatedSpans: spans, }, err: nil, }, { r: &ptpb.Record{ - Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, - MetaType: "job", - Meta: []byte("junk"), - Spans: spans, + Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, + MetaType: "job", + Meta: []byte("junk"), + DeprecatedSpans: spans, }, err: errZeroID, }, { r: &ptpb.Record{ - ID: uuid.MakeV4().GetBytes(), - MetaType: "job", - Meta: []byte("junk"), - Spans: spans, + ID: uuid.MakeV4().GetBytes(), + MetaType: "job", + Meta: []byte("junk"), + DeprecatedSpans: spans, }, err: errZeroTimestamp, }, { r: &ptpb.Record{ - ID: uuid.MakeV4().GetBytes(), - Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, - Meta: []byte("junk"), - Spans: spans, + ID: uuid.MakeV4().GetBytes(), + Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}, + Meta: []byte("junk"), + DeprecatedSpans: spans, }, err: errInvalidMeta, }, diff --git a/pkg/kv/kvserver/protectedts/ptverifier/verifier.go b/pkg/kv/kvserver/protectedts/ptverifier/verifier.go index 5ead36abb854..26f7d5af6679 100644 --- a/pkg/kv/kvserver/protectedts/ptverifier/verifier.go +++ b/pkg/kv/kvserver/protectedts/ptverifier/verifier.go @@ -81,7 +81,7 @@ func getRecordWithTimestamp( func makeVerificationBatch(r *ptpb.Record, aliveAt hlc.Timestamp) kv.Batch { // Need to perform validation, build a batch and run it. - mergedSpans, _ := roachpb.MergeSpans(&r.Spans) + mergedSpans, _ := roachpb.MergeSpans(&r.DeprecatedSpans) var b kv.Batch for _, s := range mergedSpans { var req roachpb.AdminVerifyProtectedTimestampRequest diff --git a/pkg/kv/kvserver/protectedts/ptverifier/verifier_test.go b/pkg/kv/kvserver/protectedts/ptverifier/verifier_test.go index deb598ff053d..311abbe1032a 100644 --- a/pkg/kv/kvserver/protectedts/ptverifier/verifier_test.go +++ b/pkg/kv/kvserver/protectedts/ptverifier/verifier_test.go @@ -78,10 +78,10 @@ func TestVerifier(t *testing.T) { spans[i] = makeTableSpan(tid) } r := ptpb.Record{ - ID: uuid.MakeV4().GetBytes(), - Timestamp: s.Clock().Now(), - Mode: ptpb.PROTECT_AFTER, - Spans: spans, + ID: uuid.MakeV4().GetBytes(), + Timestamp: s.Clock().Now(), + Mode: ptpb.PROTECT_AFTER, + DeprecatedSpans: spans, } require.Nil(t, s.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { return pts.Protect(ctx, txn, &r) @@ -138,8 +138,8 @@ func TestVerifier(t *testing.T) { resp.Add(&roachpb.AdminVerifyProtectedTimestampResponse{ VerificationFailedRanges: []roachpb.AdminVerifyProtectedTimestampResponse_FailedRange{{ RangeID: 42, - StartKey: roachpb.RKey(r.Spans[0].Key), - EndKey: roachpb.RKey(r.Spans[0].EndKey), + StartKey: roachpb.RKey(r.DeprecatedSpans[0].Key), + EndKey: roachpb.RKey(r.DeprecatedSpans[0].EndKey), }}, }) return &resp, nil @@ -166,16 +166,16 @@ func TestVerifier(t *testing.T) { resp.Add(&roachpb.AdminVerifyProtectedTimestampResponse{ VerificationFailedRanges: []roachpb.AdminVerifyProtectedTimestampResponse_FailedRange{{ RangeID: 42, - StartKey: roachpb.RKey(r.Spans[0].Key), - EndKey: roachpb.RKey(r.Spans[0].EndKey), + StartKey: roachpb.RKey(r.DeprecatedSpans[0].Key), + EndKey: roachpb.RKey(r.DeprecatedSpans[0].EndKey), Reason: "foo", }}, }) resp.Add(&roachpb.AdminVerifyProtectedTimestampResponse{ VerificationFailedRanges: []roachpb.AdminVerifyProtectedTimestampResponse_FailedRange{{ RangeID: 12, - StartKey: roachpb.RKey(r.Spans[1].Key), - EndKey: roachpb.RKey(r.Spans[1].EndKey), + StartKey: roachpb.RKey(r.DeprecatedSpans[1].Key), + EndKey: roachpb.RKey(r.DeprecatedSpans[1].EndKey), Reason: "bar", }}, }) @@ -206,8 +206,8 @@ func TestVerifier(t *testing.T) { resp.Add(&roachpb.AdminVerifyProtectedTimestampResponse{ DeprecatedFailedRanges: []roachpb.RangeDescriptor{{ RangeID: 42, - StartKey: roachpb.RKey(r.Spans[0].Key), - EndKey: roachpb.RKey(r.Spans[0].EndKey), + StartKey: roachpb.RKey(r.DeprecatedSpans[0].Key), + EndKey: roachpb.RKey(r.DeprecatedSpans[0].EndKey), }}, }) return &resp, nil diff --git a/pkg/kv/kvserver/replica_protected_timestamp_test.go b/pkg/kv/kvserver/replica_protected_timestamp_test.go index 77d24440ebaa..3055d3d77551 100644 --- a/pkg/kv/kvserver/replica_protected_timestamp_test.go +++ b/pkg/kv/kvserver/replica_protected_timestamp_test.go @@ -124,7 +124,7 @@ func TestProtectedTimestampRecordApplies(t *testing.T) { mt.records = append(mt.records, &ptpb.Record{ ID: args.RecordID.GetBytes(), Timestamp: ts, - Spans: []roachpb.Span{ + DeprecatedSpans: []roachpb.Span{ { Key: roachpb.Key(r.Desc().StartKey), EndKey: roachpb.Key(r.Desc().StartKey.Next()), @@ -159,7 +159,7 @@ func TestProtectedTimestampRecordApplies(t *testing.T) { mt.records = append(mt.records, &ptpb.Record{ ID: id.GetBytes(), Timestamp: oldTimestamp, - Spans: []roachpb.Span{ + DeprecatedSpans: []roachpb.Span{ { Key: roachpb.Key(r.Desc().StartKey), EndKey: roachpb.Key(r.Desc().StartKey.Next()), @@ -201,7 +201,7 @@ func TestProtectedTimestampRecordApplies(t *testing.T) { mt.records = append(mt.records, &ptpb.Record{ ID: id.GetBytes(), Timestamp: oldTimestamp, - Spans: []roachpb.Span{ + DeprecatedSpans: []roachpb.Span{ { Key: roachpb.Key(r.Desc().StartKey), EndKey: roachpb.Key(r.Desc().StartKey.Next()), @@ -263,7 +263,7 @@ func TestProtectedTimestampRecordApplies(t *testing.T) { mt.records = append(mt.records, &ptpb.Record{ ID: args.RecordID.GetBytes(), Timestamp: ts, - Spans: []roachpb.Span{ + DeprecatedSpans: []roachpb.Span{ { Key: keys.MinKey, EndKey: keys.MaxKey, @@ -434,7 +434,7 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { mt.records = append(mt.records, &ptpb.Record{ ID: uuid.MakeV4().GetBytes(), Timestamp: ts, - Spans: []roachpb.Span{ + DeprecatedSpans: []roachpb.Span{ { Key: keys.MinKey, EndKey: keys.MaxKey, @@ -458,7 +458,7 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { mt.records = append(mt.records, &ptpb.Record{ ID: uuid.MakeV4().GetBytes(), Timestamp: ts, - Spans: []roachpb.Span{ + DeprecatedSpans: []roachpb.Span{ { Key: keys.MinKey, EndKey: keys.MaxKey, @@ -486,7 +486,7 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { mt.records = append(mt.records, &ptpb.Record{ ID: uuid.MakeV4().GetBytes(), Timestamp: th.Next(), - Spans: []roachpb.Span{ + DeprecatedSpans: []roachpb.Span{ { Key: keys.MinKey, EndKey: keys.MaxKey, @@ -514,7 +514,7 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { mt.records = append(mt.records, &ptpb.Record{ ID: id.GetBytes(), Timestamp: ts, - Spans: []roachpb.Span{ + DeprecatedSpans: []roachpb.Span{ { Key: keys.MinKey, EndKey: keys.MaxKey, @@ -551,7 +551,7 @@ func (c *manualCache) Iterate( ) hlc.Timestamp { query := roachpb.Span{Key: start, EndKey: end} for _, r := range c.records { - for _, sp := range r.Spans { + for _, sp := range r.DeprecatedSpans { if query.Overlaps(sp) { it(r) break