Skip to content

Commit

Permalink
Merge #97221
Browse files Browse the repository at this point in the history
97221: kvpb: extract requests and errors from roachpb into new package r=yuzefovich a=yuzefovich

This commit extracts a new `pkg/kv/kvpb` package out of `pkg/roachpb`.
We've wanted to do this for a while, and what prompted me to do this was
the desire to make the ScanResponse depend on `pkg/col/coldata`.
`api.proto` and `errors.proto` files are moved into the new package
while preserving the `roachpb` as their protobuf package. This required
also the movement of the following:
- `roachpb/gen` and `roachpb/roachpbmock` subpackages
- almost all of `api.go` (except for `roachpb.UserPriority` that was
moved into `roachpb/data.go`)
- miscellaneous error-specific files
- extracting a couple of functions from `data.go`
- `method.go`.

In order to keep the backwards-compatibility for errors that are sent
over the wire in the mixed-version clusters, for all error types that
were moved from `roachpb` to `kv/kvpb` a type migration has been
registered with the `errors` library.

Clearly, essentially all of these changes were done automatically. The
only manual change worth mentioning was when moving `errors.go`. Namely,
type conversion of `ReadWithinUncertaintyIntervalError.ObservedTimestamps`
to `roachpb.observedTimestampSlice` was removed in the loop since it
seems to not do anything.

After running `./dev gen bazel` the proto compiler in
`roachpb/BUILD.bazel` changed. I'm assuming that it is expected.

Epic: None

Release note: None

Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
craig[bot] and yuzefovich committed Feb 19, 2023
2 parents 3d054f3 + 736a67e commit e9c96e7
Show file tree
Hide file tree
Showing 731 changed files with 10,599 additions and 9,828 deletions.
27 changes: 14 additions & 13 deletions .github/CODEOWNERS
Validating CODEOWNERS rules …
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@
/pkg/sql/job_exec_context* @cockroachdb/sql-queries @cockroachdb/jobs-prs
/pkg/sql/delegate/*job*.go @cockroachdb/jobs-prs

/pkg/sql/appstatspb @cockroachdb/sql-observability
/pkg/sql/execstats/ @cockroachdb/sql-observability
/pkg/sql/scheduledlogging/ @cockroachdb/sql-observability
/pkg/sql/sqlstats/ @cockroachdb/sql-observability
Expand Down Expand Up @@ -196,6 +197,19 @@
/pkg/kv/kvclient/kvstreamer @cockroachdb/sql-queries
/pkg/kv/kvclient/rangefeed/ @cockroachdb/repl-prs
/pkg/kv/kvnemesis/ @cockroachdb/kv-prs
/pkg/kv/kvpb/ @cockroachdb/kv-prs
/pkg/kv/kvpb/.gitattributes @cockroachdb/dev-inf
/pkg/kv/kvpb/ambiguous_* @cockroachdb/kv-prs
/pkg/kv/kvpb/api* @cockroachdb/kv-prs
/pkg/kv/kvpb/batch* @cockroachdb/kv-prs
/pkg/kv/kvpb/error* @cockroachdb/kv-prs
/pkg/kv/kvpb/gen @cockroachdb/dev-inf
/pkg/kv/kvpb/gen.bzl @cockroachdb/dev-inf
/pkg/kv/kvpb/kvpbmock/ @cockroachdb/kv-prs
/pkg/kv/kvpb/method* @cockroachdb/kv-prs
/pkg/kv/kvpb/string_test.go @cockroachdb/kv-prs
/pkg/kv/kvpb/testdata/ambi* @cockroachdb/kv-prs
/pkg/kv/kvpb/testdata/repl* @cockroachdb/kv-prs
/pkg/kv/kvprober/ @cockroachdb/kv-prs
# Same subdirectory rule as above for `/pkg/kv`
/pkg/kv/kvserver/*.* @cockroachdb/kv-prs
Expand Down Expand Up @@ -393,30 +407,17 @@
/pkg/multitenant/ @cockroachdb/multi-tenant
/pkg/release/ @cockroachdb/dev-inf
/pkg/roachpb/.gitattributes @cockroachdb/dev-inf
/pkg/roachpb/ambiguous_* @cockroachdb/kv-prs
/pkg/roachpb/api* @cockroachdb/kv-prs
/pkg/roachpb/batch* @cockroachdb/kv-prs
#!/pkg/roachpb/BUILD.bazel @cockroachdb/kv-prs-noreview
/pkg/roachpb/data* @cockroachdb/kv-prs
/pkg/roachpb/dep_test.go @cockroachdb/dev-inf
/pkg/roachpb/error* @cockroachdb/kv-prs
/pkg/roachpb/gen @cockroachdb/dev-inf
/pkg/roachpb/gen.bzl @cockroachdb/dev-inf
/pkg/roachpb/app* @cockroachdb/sql-observability
/pkg/roachpb/index* @cockroachdb/sql-observability
/pkg/roachpb/internal* @cockroachdb/kv-prs
/pkg/roachpb/io-formats* @cockroachdb/disaster-recovery
#!/pkg/roachpb/main_test.go @cockroachdb/kv-prs-noreview
/pkg/roachpb/merge_spans* @cockroachdb/kv-prs
/pkg/roachpb/metadata* @cockroachdb/kv-prs
/pkg/roachpb/method* @cockroachdb/kv-prs
/pkg/roachpb/roachpbmock/ @cockroachdb/kv-prs
/pkg/roachpb/replica_* @cockroachdb/kv-prs
/pkg/roachpb/span* @cockroachdb/kv-prs
/pkg/roachpb/string_test.go @cockroachdb/kv-prs
/pkg/roachpb/tenant* @cockroachdb/kv-prs
/pkg/roachpb/testdata/ambi* @cockroachdb/kv-prs
/pkg/roachpb/testdata/repl* @cockroachdb/kv-prs
#!/pkg/roachpb/version* @cockroachdb/unowned
/pkg/roachprod/ @cockroachdb/test-eng
/pkg/rpc/ @cockroachdb/kv-prs
Expand Down
8 changes: 4 additions & 4 deletions BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -58,18 +58,18 @@ exports_files([
# gazelle:resolve go google.golang.org/genproto/googleapis/pubsub/v1 @org_golang_google_genproto//googleapis/pubsub/v1:pubsub
# gazelle:resolve go google.golang.org/genproto/googleapis/cloud/kms/v1 @org_golang_google_genproto//googleapis/cloud/kms/v1:kms

# See pkg/roachpb/gen/BUILD.bazel for more details.
# See pkg/kv/kvpb/gen/BUILD.bazel for more details.
#
# gazelle:resolve proto go roachpb/api.proto //pkg/roachpb
# gazelle:resolve proto go kv/kvpb/api.proto //pkg/kv/kvpb
# gazelle:resolve proto go kv/kvpb/errors.proto //pkg/kv/kvpb
# gazelle:resolve proto go roachpb/data.proto //pkg/roachpb
# gazelle:resolve proto go roachpb/errors.proto //pkg/roachpb
# gazelle:resolve proto go roachpb/index_usage_stats.proto //pkg/roachpb
# gazelle:resolve proto go roachpb/internal.proto //pkg/roachpb
# gazelle:resolve proto go roachpb/internal_raft.proto //pkg/roachpb
# gazelle:resolve proto go roachpb/io-formats.proto //pkg/roachpb
# gazelle:resolve proto go roachpb/metadata.proto //pkg/roachpb
# gazelle:resolve proto go roachpb/span_config.proto //pkg/roachpb
# gazelle:exclude pkg/roachpb/batch_generated.go
# gazelle:exclude pkg/kv/kvpb/batch_generated.go

# See pkg/sql/opt/optgen/cmd/langgen/BUILD.bazel for more details.
#
Expand Down
4 changes: 2 additions & 2 deletions build/bazelutil/check.sh
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,8 @@ pkg/kv/kvclient/rangecache/range_cache.go://go:generate mockgen -package=rangeca
pkg/kv/kvclient/rangefeed/rangefeed.go://go:generate mockgen -destination=mocks_generated_test.go --package=rangefeed . DB
pkg/kv/kvserver/concurrency/lock_table.go://go:generate ../../../util/interval/generic/gen.sh *lockState concurrency
pkg/kv/kvserver/spanlatch/manager.go://go:generate ../../../util/interval/generic/gen.sh *latch spanlatch
pkg/roachpb/api.go://go:generate mockgen -package=roachpbmock -destination=roachpbmock/mocks_generated.go . InternalClient,Internal_RangeFeedClient,Internal_MuxRangeFeedClient
pkg/roachpb/batch.go://go:generate go run gen/main.go --filename batch_generated.go *.pb.go
pkg/kv/kvpb/api.go://go:generate mockgen -package=kvpbmock -destination=kvpbmock/mocks_generated.go . InternalClient,Internal_RangeFeedClient,Internal_MuxRangeFeedClient
pkg/kv/kvpb/batch.go://go:generate go run gen/main.go --filename batch_generated.go *.pb.go
pkg/security/certmgr/cert.go://go:generate mockgen -package=certmgr -destination=mocks_generated_test.go . Cert
pkg/security/securitytest/securitytest.go://go:generate go-bindata -mode 0600 -modtime 1400000000 -pkg securitytest -o embedded.go -ignore README.md -ignore regenerate.sh test_certs
pkg/security/securitytest/securitytest.go://go:generate gofmt -s -w embedded.go
Expand Down
6 changes: 3 additions & 3 deletions build/bazelutil/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -47,8 +47,8 @@
".*\\.pb\\.gw\\.go$": "generated code",
"cockroach/pkg/.*_generated\\.go$": "generated code",
"cockroach/pkg/kv/kvclient/kvcoord/lock_spans_over_budget_error\\.go$": "invalid direct cast on error object",
"cockroach/pkg/roachpb/batch_generated-gen\\.go$": "invalid direct cast on error object",
"cockroach/pkg/roachpb/errors\\.go$": "invalid direct cast on error object",
"cockroach/pkg/kv/kvpb/batch_generated-gen\\.go$": "invalid direct cast on error object",
"cockroach/pkg/kv/kvpb/errors\\.go$": "invalid direct cast on error object",
"cockroach/pkg/sql/pgwire/pgerror/constraint_name\\.go$": "invalid direct cast on error object",
"cockroach/pkg/sql/pgwire/pgerror/pgcode\\.go$": "invalid direct cast on error object",
"cockroach/pkg/testutils/lint/lint_test\\.go$": "invalid direct cast on error object",
Expand Down Expand Up @@ -719,7 +719,7 @@
},
"SA1019": {
"exclude_files": {
"cockroach/pkg/roachpb/api_test.go$": "same package that grpc-go imports",
"cockroach/pkg/kv/kvpb/api_test.go$": "same package that grpc-go imports",
"cockroach/pkg/rpc/codec.go$": "rpc/codec.go imports the same proto package that grpc-go imports (as of crdb@dd87d1145 and grpc-go@7b167fd6).",
"cockroach/pkg/rpc/stats_handler.go$": "Using deprecated WireLength call",
"cockroach/pkg/.*\\.eg\\.go$": "generated code",
Expand Down
1 change: 1 addition & 0 deletions docs/generated/http/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ genrule(
"//pkg/geo/geopb:geopb_proto",
"//pkg/gossip:gossip_proto",
"//pkg/jobs/jobspb:jobspb_proto",
"//pkg/kv/kvpb:kvpb_proto",
"//pkg/kv/kvserver/concurrency/lock:lock_proto",
"//pkg/kv/kvserver/kvserverpb:kvserverpb_proto",
"//pkg/kv/kvserver/liveness/livenesspb:livenesspb_proto",
Expand Down
15 changes: 10 additions & 5 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,8 @@ ALL_TESTS = [
"//pkg/kv/kvclient/rangefeed/rangefeedcache:rangefeedcache_test",
"//pkg/kv/kvclient/rangefeed:rangefeed_test",
"//pkg/kv/kvnemesis:kvnemesis_test",
"//pkg/kv/kvpb:kvpb_disallowed_imports_test",
"//pkg/kv/kvpb:kvpb_test",
"//pkg/kv/kvprober:kvprober_test",
"//pkg/kv/kvserver/abortspan:abortspan_test",
"//pkg/kv/kvserver/allocator/allocatorimpl:allocatorimpl_test",
Expand Down Expand Up @@ -1180,6 +1182,11 @@ GO_TARGETS = [
"//pkg/kv/kvnemesis/kvnemesisutil:kvnemesisutil",
"//pkg/kv/kvnemesis:kvnemesis",
"//pkg/kv/kvnemesis:kvnemesis_test",
"//pkg/kv/kvpb/gen:gen",
"//pkg/kv/kvpb/gen:gen_lib",
"//pkg/kv/kvpb/kvpbmock:kvpbmock",
"//pkg/kv/kvpb:kvpb",
"//pkg/kv/kvpb:kvpb_test",
"//pkg/kv/kvprober:kvprober",
"//pkg/kv/kvprober:kvprober_test",
"//pkg/kv/kvserver/abortspan:abortspan",
Expand Down Expand Up @@ -1345,9 +1352,6 @@ GO_TARGETS = [
"//pkg/release:release",
"//pkg/repstream/streampb:streampb",
"//pkg/repstream:repstream",
"//pkg/roachpb/gen:gen",
"//pkg/roachpb/gen:gen_lib",
"//pkg/roachpb/roachpbmock:roachpbmock",
"//pkg/roachpb:roachpb",
"//pkg/roachpb:roachpb_test",
"//pkg/roachprod/cloud:cloud",
Expand Down Expand Up @@ -2620,6 +2624,9 @@ GET_X_DATA_TARGETS = [
"//pkg/kv/kvclient/rangestats:get_x_data",
"//pkg/kv/kvnemesis:get_x_data",
"//pkg/kv/kvnemesis/kvnemesisutil:get_x_data",
"//pkg/kv/kvpb:get_x_data",
"//pkg/kv/kvpb/gen:get_x_data",
"//pkg/kv/kvpb/kvpbmock:get_x_data",
"//pkg/kv/kvprober:get_x_data",
"//pkg/kv/kvserver:get_x_data",
"//pkg/kv/kvserver/abortspan:get_x_data",
Expand Down Expand Up @@ -2722,8 +2729,6 @@ GET_X_DATA_TARGETS = [
"//pkg/repstream:get_x_data",
"//pkg/repstream/streampb:get_x_data",
"//pkg/roachpb:get_x_data",
"//pkg/roachpb/gen:get_x_data",
"//pkg/roachpb/roachpbmock:get_x_data",
"//pkg/roachprod:get_x_data",
"//pkg/roachprod/cloud:get_x_data",
"//pkg/roachprod/config:get_x_data",
Expand Down
2 changes: 1 addition & 1 deletion pkg/base/serverident/server_ident.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ type ServerIdentificationPayload interface {
// sense for this interface to live in log).
//
// Note that this tenant ID should not be confused with the one put in the
// context by roachpb.NewContextForTenant(): that one is used by a server
// context by roachpb.ContextWithClientTenant(): that one is used by a server
// handling an RPC call, referring to the tenant that's the client of the RPC.
TenantID() interface{}
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/backupccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ go_library(
"//pkg/kv",
"//pkg/kv/bulk",
"//pkg/kv/kvclient",
"//pkg/kv/kvpb",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/batcheval",
"//pkg/kv/kvserver/concurrency/lock",
Expand Down Expand Up @@ -226,6 +227,7 @@ go_test(
"//pkg/kv",
"//pkg/kv/bulk",
"//pkg/kv/kvclient/kvcoord",
"//pkg/kv/kvpb",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/closedts",
"//pkg/kv/kvserver/kvserverbase",
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backup_intents_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,8 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/ccl"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand All @@ -43,7 +43,7 @@ func TestCleanupIntentsDuringBackupPerformanceRegression(t *testing.T) {

// Interceptor catches requests that cleanup transactions of size 1000 which are
// test data transactions. All other transaction commits pass though.
interceptor := func(ctx context.Context, req *roachpb.BatchRequest) *roachpb.Error {
interceptor := func(ctx context.Context, req *kvpb.BatchRequest) *kvpb.Error {
endTxn := req.Requests[0].GetEndTxn()
if endTxn != nil && !endTxn.Commit && len(endTxn.LockSpans) == perTransactionRowCount {
// If this is a rollback of one the test's SQL transactions, allow the
Expand Down
7 changes: 4 additions & 3 deletions pkg/ccl/backupccl/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/joberror"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/scheduledjobs"
Expand Down Expand Up @@ -72,7 +73,7 @@ var BackupCheckpointInterval = settings.RegisterDurationSetting(

var forceReadBackupManifest = util.ConstantWithMetamorphicTestBool("backup-read-manifest", false)

func countRows(raw roachpb.BulkOpSummary, pkIDs map[uint64]bool) roachpb.RowCount {
func countRows(raw kvpb.BulkOpSummary, pkIDs map[uint64]bool) roachpb.RowCount {
res := roachpb.RowCount{DataSize: raw.DataSize}
for id, count := range raw.EntryCounts {
if _, ok := pkIDs[id]; ok {
Expand Down Expand Up @@ -179,7 +180,7 @@ func backup(
pkIDs := make(map[uint64]bool)
for i := range backupManifest.Descriptors {
if t, _, _, _, _ := descpb.GetDescriptors(&backupManifest.Descriptors[i]); t != nil {
pkIDs[roachpb.BulkOpSummaryID(uint64(t.ID), uint64(t.PrimaryIndex.ID))] = true
pkIDs[kvpb.BulkOpSummaryID(uint64(t.ID), uint64(t.PrimaryIndex.ID))] = true
}
}

Expand All @@ -206,7 +207,7 @@ func backup(
urisByLocalityKV,
encryption,
&kmsEnv,
roachpb.MVCCFilter(backupManifest.MVCCFilter),
kvpb.MVCCFilter(backupManifest.MVCCFilter),
backupManifest.StartTime,
backupManifest.EndTime,
)
Expand Down
23 changes: 12 additions & 11 deletions pkg/ccl/backupccl/backup_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
Expand Down Expand Up @@ -364,8 +365,8 @@ func runBackupProcessor(
splitMidKey = true
}

req := &roachpb.ExportRequest{
RequestHeader: roachpb.RequestHeaderFromSpan(span.span),
req := &kvpb.ExportRequest{
RequestHeader: kvpb.RequestHeaderFromSpan(span.span),
ResumeKeyTS: span.firstKeyTS,
StartTime: span.start,
MVCCFilter: spec.MVCCFilter,
Expand Down Expand Up @@ -394,7 +395,7 @@ func runBackupProcessor(
priority = timeutil.Since(readTime) > priorityAfter.Get(&clusterSettings.SV)
}

header := roachpb.Header{
header := kvpb.Header{
// We set the DistSender response target bytes field to a sentinel
// value. The sentinel value of 1 forces the ExportRequest to paginate
// after creating a single SST.
Expand All @@ -416,19 +417,19 @@ func runBackupProcessor(
header.WaitPolicy = lock.WaitPolicy_Error
}

admissionHeader := roachpb.AdmissionHeader{
admissionHeader := kvpb.AdmissionHeader{
// Export requests are currently assigned BulkNormalPri.
//
// TODO(dt): Consider linking this to/from the UserPriority field.
Priority: int32(admissionpb.BulkNormalPri),
CreateTime: timeutil.Now().UnixNano(),
Source: roachpb.AdmissionHeader_FROM_SQL,
Source: kvpb.AdmissionHeader_FROM_SQL,
NoMemoryReservedAtSource: true,
}
log.VEventf(ctx, 1, "sending ExportRequest for span %s (attempt %d, priority %s)",
span.span, span.attempts+1, header.UserPriority.String())
var rawResp roachpb.Response
var pErr *roachpb.Error
var rawResp kvpb.Response
var pErr *kvpb.Error
requestSentAt := timeutil.Now()
exportRequestErr := contextutil.RunWithTimeout(ctx,
fmt.Sprintf("ExportRequest for span %s", span.span),
Expand All @@ -441,7 +442,7 @@ func runBackupProcessor(
return nil
})
if exportRequestErr != nil {
if intentErr, ok := pErr.GetDetail().(*roachpb.WriteIntentError); ok {
if intentErr, ok := pErr.GetDetail().(*kvpb.WriteIntentError); ok {
span.lastTried = timeutil.Now()
span.attempts++
todo <- span
Expand All @@ -458,7 +459,7 @@ func runBackupProcessor(
}
// BatchTimestampBeforeGCError is returned if the ExportRequest
// attempts to read below the range's GC threshold.
if batchTimestampBeforeGCError, ok := pErr.GetDetail().(*roachpb.BatchTimestampBeforeGCError); ok {
if batchTimestampBeforeGCError, ok := pErr.GetDetail().(*kvpb.BatchTimestampBeforeGCError); ok {
// If the range we are exporting is marked to be excluded from
// backup, it is safe to ignore the error. It is likely that the
// table has been configured with a low GC TTL, and so the data
Expand All @@ -471,7 +472,7 @@ func runBackupProcessor(
return errors.Wrapf(exportRequestErr, "exporting %s", span.span)
}

resp := rawResp.(*roachpb.ExportResponse)
resp := rawResp.(*kvpb.ExportResponse)

// If the reply has a resume span, we process it immediately.
var resumeSpan spanAndTime
Expand Down Expand Up @@ -558,7 +559,7 @@ func runBackupProcessor(

// recordExportStats emits a StructuredEvent containing the stats about the
// evaluated ExportRequest.
func recordExportStats(sp *tracing.Span, resp *roachpb.ExportResponse, requestSentAt time.Time) {
func recordExportStats(sp *tracing.Span, resp *kvpb.ExportResponse, requestSentAt time.Time) {
if sp == nil {
return
}
Expand Down
7 changes: 4 additions & 3 deletions pkg/ccl/backupccl/backup_processor_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
Expand All @@ -41,7 +42,7 @@ func distBackupPlanSpecs(
urisByLocalityKV map[string]string,
encryption *jobspb.BackupEncryptionOptions,
kmsEnv cloud.KMSEnv,
mvccFilter roachpb.MVCCFilter,
mvccFilter kvpb.MVCCFilter,
startTime, endTime hlc.Timestamp,
) (map[base.SQLInstanceID]*execinfrapb.BackupDataSpec, error) {
var span *tracing.Span
Expand Down Expand Up @@ -85,9 +86,9 @@ func distBackupPlanSpecs(
}
// Wrap the relevant BackupEncryptionOptions to be used by the Backup
// processor and KV ExportRequest.
var fileEncryption *roachpb.FileEncryptionOptions
var fileEncryption *kvpb.FileEncryptionOptions
if encryption != nil {
fileEncryption = &roachpb.FileEncryptionOptions{Key: encryption.Key}
fileEncryption = &kvpb.FileEncryptionOptions{Key: encryption.Key}
}

// First construct spans based on span partitions. Then add on
Expand Down
Loading

0 comments on commit e9c96e7

Please sign in to comment.