Skip to content

Commit

Permalink
storage,backupccl: version gate MVCCValueHeaders in ExportRequest
Browse files Browse the repository at this point in the history
Previously, callers could (and did) assume that the values present in
the SSTs returned by export request could be interpreted directly as
roachpb.Value objects using code like:

    roachpb.Value{RawBytes: valBytes}

For MVCCValueHeaders to be exported by ExportRequest all callers need
to be updated:

1. ExportRequest on system.descriptors in sql/catalog/lease
2. ExportRequest on system.descriptors in ccl/changefeedccl/schemafeed
3. ExportRequest used by `FINGERPRINT`
4. ExportRequest used by old binaries in a mixed-version cluster.

(1) and (2) will be easy to update and likely don't matter in practice
moment as those tables do not include values with exportable value
headers at the moment.

(3) will be easy to update, but we still need an option to exclude
value headers (a) until value headers are included in rangefeeds
and (b) so long as we want to compare fingerprints with 23.2 versions.

(4) is impossible to update so if we want BACKUP/RESTORE to round-trip
in mixed version cluster we must version gate including them in
backups until the cluster is on a single version.

Note that this also marks a hard backward incompatibility for backup
artifacts. Backups for 24.1 cannot be restored on 23.2 or older. This
was already the case by policy. 23.2 backups should still work fine on
24.1 since all roachpb.Value's should properly decode as MVCCValue's.

Epic: none
Release note: None
  • Loading branch information
stevendanna committed Mar 11, 2024
1 parent c1c70e9 commit 8454009
Show file tree
Hide file tree
Showing 5 changed files with 36 additions and 13 deletions.
16 changes: 9 additions & 7 deletions pkg/ccl/backupccl/backup_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
Expand Down Expand Up @@ -481,20 +482,21 @@ func runBackupProcessor(
case spans := <-todo:
for _, span := range spans {
for len(span.span.Key) != 0 {
includeMVCCValueHeader := clusterSettings.Version.IsActive(ctx, clusterversion.V24_1)
splitMidKey := splitKeysOnTimestamps.Get(&clusterSettings.SV)
// If we started splitting already, we must continue until we reach the end
// of split span.
if !span.firstKeyTS.IsEmpty() {
splitMidKey = true
}

req := &kvpb.ExportRequest{
RequestHeader: kvpb.RequestHeaderFromSpan(span.span),
ResumeKeyTS: span.firstKeyTS,
StartTime: span.start,
MVCCFilter: spec.MVCCFilter,
TargetFileSize: batcheval.ExportRequestTargetFileSize.Get(&clusterSettings.SV),
SplitMidKey: splitMidKey,
RequestHeader: kvpb.RequestHeaderFromSpan(span.span),
ResumeKeyTS: span.firstKeyTS,
StartTime: span.start,
MVCCFilter: spec.MVCCFilter,
TargetFileSize: batcheval.ExportRequestTargetFileSize.Get(&clusterSettings.SV),
SplitMidKey: splitMidKey,
IncludeMVCCValueHeader: includeMVCCValueHeader,
}

// If we're doing re-attempts but are not yet in the priority regime,
Expand Down
5 changes: 3 additions & 2 deletions pkg/kv/bulk/sst_batcher_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -397,8 +397,9 @@ func TestImportEpochIngestion(t *testing.T) {
Key: startKey.Key,
EndKey: endKey.Key,
},
MVCCFilter: kvpb.MVCCFilter_All,
StartTime: hlc.Timestamp{},
MVCCFilter: kvpb.MVCCFilter_All,
StartTime: hlc.Timestamp{},
IncludeMVCCValueHeader: true,
}

header := kvpb.Header{Timestamp: s.Clock().Now()}
Expand Down
9 changes: 8 additions & 1 deletion pkg/kv/kvpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -1831,7 +1831,14 @@ message ExportRequest {

FingerprintOptions fingerprint_options = 15 [(gogoproto.nullable) = false];

// Next ID: 16
// IncludeMVCCValueHeader controls whether the MVCCValueHeader is
// included in exported bytes. Callers should only set this when all
// readers of the returned SST are prepared to parse full a
// MVCCValue. Even when set, only fields appropriate for export are
// included. See storage.EncodeMVCCValueForExport for details.
bool include_mvcc_value_header = 16 [(gogoproto.customname) = "IncludeMVCCValueHeader"];

// Next ID: 17
}

message FingerprintOptions {
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/batcheval/cmd_export.go
Original file line number Diff line number Diff line change
Expand Up @@ -197,6 +197,7 @@ func evalExport(
TargetLockConflictBytes: targetLockConflictBytes,
StopMidKey: args.SplitMidKey,
ScanStats: cArgs.ScanStats,
IncludeMVCCValueHeader: args.IncludeMVCCValueHeader,
}
var summary kvpb.BulkOpSummary
var resumeInfo storage.ExportRequestResumeInfo
Expand Down
18 changes: 15 additions & 3 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -7884,9 +7884,13 @@ func mvccExportToWriter(
return kvpb.BulkOpSummary{}, ExportRequestResumeInfo{}, errors.Wrapf(err, "decoding mvcc value %s", unsafeKey)
}

unsafeValue, err = EncodeMVCCValueForExport(mvccValue)
if err != nil {
return kvpb.BulkOpSummary{}, ExportRequestResumeInfo{}, errors.Wrapf(err, "repackaging imported mvcc value %s", unsafeKey)
if opts.IncludeMVCCValueHeader {
unsafeValue, err = EncodeMVCCValueForExport(mvccValue)
if err != nil {
return kvpb.BulkOpSummary{}, ExportRequestResumeInfo{}, errors.Wrapf(err, "repackaging imported mvcc value %s", unsafeKey)
}
} else {
unsafeValue = mvccValue.Value.RawBytes
}
// Skip tombstone records when start time is zero (non-incremental)
// and we are not exporting all versions.
Expand Down Expand Up @@ -8053,6 +8057,14 @@ type MVCCExportOptions struct {
// FingerprintOptions controls how fingerprints are generated
// when using MVCCExportFingerprint.
FingerprintOptions MVCCExportFingerprintOptions

// IncludeMVCCValueHeader controls whether we include
// MVCCValueHeaders in the exported data. When true, the
// portions of the header appropriate for export are included
// in the encoded values. Callers should be ready to decode
// full MVCCValue's in this case.
IncludeMVCCValueHeader bool

// ScanStats, if set, is updated with iterator stats upon export success of
// failure. Non-iterator stats i.e., {NumGets,NumReverseScans} are left
// unchanged, and NumScans is incremented by 1.
Expand Down

0 comments on commit 8454009

Please sign in to comment.