Skip to content

Commit

Permalink
storage: Split pkg cloud into cloud interface and cloud impl
Browse files Browse the repository at this point in the history
Previously, both the ExternalStorage interface and all the specialized
implementations were in `pkg/storage/cloud`.

This PR moves the interface and factory method signatures to
`pkg/storage/cloud` and all the implementations to
`pkg/storage/cloudimpl`.  The motivation behind this is to ensure that
other packages such `pkg/sql` and `pkg/kv` depend on just the
interface/factory signatures (not the concrete impl) to prevent
dependency cycles.

More concreteley, we need to plumb an InternalExecutor and kv.DB for our
new user scoped file-table storage. This was previously not possible
because of a `pkg/sql -> pkg/storage/cloud ->pkg/sql` cycle. Now, since
the executor will be used in `cloudimpl` this cycle will not occur.

Release note: None
  • Loading branch information
adityamaru committed Jun 25, 2020
1 parent d14bf5d commit 26aa14e
Show file tree
Hide file tree
Showing 38 changed files with 464 additions and 429 deletions.
10 changes: 5 additions & 5 deletions pkg/ccl/backupccl/backup_cloud_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ import (

"github.com/aws/aws-sdk-go/aws/credentials"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/lease"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -53,8 +53,8 @@ func TestCloudBackupRestoreS3(t *testing.T) {
prefix := fmt.Sprintf("TestBackupRestoreS3-%d", timeutil.Now().UnixNano())
uri := url.URL{Scheme: "s3", Host: bucket, Path: prefix}
values := uri.Query()
values.Add(cloud.S3AccessKeyParam, creds.AccessKeyID)
values.Add(cloud.S3SecretParam, creds.SecretAccessKey)
values.Add(cloudimpl.S3AccessKeyParam, creds.AccessKeyID)
values.Add(cloudimpl.S3SecretParam, creds.SecretAccessKey)
uri.RawQuery = values.Encode()

backupAndRestore(ctx, t, tc, []string{uri.String()}, []string{uri.String()}, numAccounts)
Expand Down Expand Up @@ -104,8 +104,8 @@ func TestCloudBackupRestoreAzure(t *testing.T) {
prefix := fmt.Sprintf("TestBackupRestoreAzure-%d", timeutil.Now().UnixNano())
uri := url.URL{Scheme: "azure", Host: bucket, Path: prefix}
values := uri.Query()
values.Add(cloud.AzureAccountNameParam, accountName)
values.Add(cloud.AzureAccountKeyParam, accountKey)
values.Add(cloudimpl.AzureAccountNameParam, accountName)
values.Add(cloudimpl.AzureAccountKeyParam, accountKey)
uri.RawQuery = values.Encode()

backupAndRestore(ctx, t, tc, []string{uri.String()}, []string{uri.String()}, numAccounts)
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 @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -407,7 +408,7 @@ func (b *backupResumer) Resume(
}
// For all backups, partitioned or not, the main BACKUP manifest is stored at
// details.URI.
defaultConf, err := cloud.ExternalStorageConfFromURI(details.URI)
defaultConf, err := cloudimpl.ExternalStorageConfFromURI(details.URI)
if err != nil {
return errors.Wrapf(err, "export configuration")
}
Expand All @@ -417,7 +418,7 @@ func (b *backupResumer) Resume(
}
storageByLocalityKV := make(map[string]*roachpb.ExternalStorage)
for kv, uri := range details.URIsByLocalityKV {
conf, err := cloud.ExternalStorageConfFromURI(uri)
conf, err := cloudimpl.ExternalStorageConfFromURI(uri)
if err != nil {
return err
}
Expand Down Expand Up @@ -554,7 +555,7 @@ func (b *backupResumer) deleteCheckpoint(ctx context.Context, cfg *sql.ExecutorC
details := b.job.Details().(jobspb.BackupDetails)
// For all backups, partitioned or not, the main BACKUP manifest is stored at
// details.URI.
conf, err := cloud.ExternalStorageConfFromURI(details.URI)
conf, err := cloudimpl.ExternalStorageConfFromURI(details.URI)
if err != nil {
return err
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/interval"
Expand Down Expand Up @@ -244,15 +244,15 @@ func backupJobDescription(
}

for _, t := range to {
sanitizedTo, err := cloud.SanitizeExternalStorageURI(t, nil /* extraParams */)
sanitizedTo, err := cloudimpl.SanitizeExternalStorageURI(t, nil /* extraParams */)
if err != nil {
return "", err
}
b.To = append(b.To, tree.NewDString(sanitizedTo))
}

for _, from := range incrementalFrom {
sanitizedFrom, err := cloud.SanitizeExternalStorageURI(from, nil /* extraParams */)
sanitizedFrom, err := cloudimpl.SanitizeExternalStorageURI(from, nil /* extraParams */)
if err != nil {
return "", err
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/backupccl/backup_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
hlc "github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -114,7 +114,7 @@ func runBackupProcessor(

// For all backups, partitioned or not, the main BACKUP manifest is stored at
// details.URI.
defaultConf, err := cloud.ExternalStorageConfFromURI(spec.DefaultURI)
defaultConf, err := cloudimpl.ExternalStorageConfFromURI(spec.DefaultURI)
if err != nil {
return errors.Wrapf(err, "export configuration")
}
Expand All @@ -124,7 +124,7 @@ func runBackupProcessor(
}
storageByLocalityKV := make(map[string]*roachpb.ExternalStorage)
for kv, uri := range spec.URIsByLocalityKV {
conf, err := cloud.ExternalStorageConfFromURI(uri)
conf, err := cloudimpl.ExternalStorageConfFromURI(uri)
if err != nil {
return err
}
Expand Down
9 changes: 5 additions & 4 deletions pkg/ccl/backupccl/manifest_handling.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
Expand Down Expand Up @@ -103,7 +104,7 @@ func readBackupManifestFromStore(
func containsManifest(ctx context.Context, exportStore cloud.ExternalStorage) (bool, error) {
r, err := exportStore.ReadFile(ctx, BackupManifestName)
if err != nil {
if errors.Is(err, cloud.ErrFileDoesNotExist) {
if errors.Is(err, cloudimpl.ErrFileDoesNotExist) {
return false, nil
}
return false, err
Expand Down Expand Up @@ -437,7 +438,7 @@ func resolveBackupManifests(
// automatically created incremental layers inside the base layer.
prev, err := findPriorBackups(ctx, baseStores[0])
if err != nil {
if errors.Is(err, cloud.ErrListingUnsupported) {
if errors.Is(err, cloudimpl.ErrListingUnsupported) {
log.Warningf(ctx, "storage sink %T does not support listing, only resolving the base backup", baseStores[0])
// If we do not support listing, we have to just assume there are none
// and restore the specified base.
Expand Down Expand Up @@ -677,7 +678,7 @@ func VerifyUsableExportTarget(
readable, BackupManifestName)
}

if !errors.Is(err, cloud.ErrFileDoesNotExist) {
if !errors.Is(err, cloudimpl.ErrFileDoesNotExist) {
return errors.Wrapf(err, "%s returned an unexpected error when checking for the existence of %s file", readable, BackupManifestName)
}

Expand All @@ -689,7 +690,7 @@ func VerifyUsableExportTarget(
readable, BackupManifestCheckpointName)
}

if !errors.Is(err, cloud.ErrFileDoesNotExist) {
if !errors.Is(err, cloudimpl.ErrFileDoesNotExist) {
return errors.Wrapf(err, "%s returned an unexpected error when checking for the existence of %s file", readable, BackupManifestCheckpointName)
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/restore_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/interval"
Expand Down Expand Up @@ -189,7 +189,7 @@ func makeImportSpans(
if backupLocalityInfo != nil && backupLocalityInfo[i].URIsByOriginalLocalityKV != nil {
storesByLocalityKV = make(map[string]roachpb.ExternalStorage)
for kv, uri := range backupLocalityInfo[i].URIsByOriginalLocalityKV {
conf, err := cloud.ExternalStorageConfFromURI(uri)
conf, err := cloudimpl.ExternalStorageConfFromURI(uri)
if err != nil {
return nil, hlc.Timestamp{}, err
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
Expand Down Expand Up @@ -756,7 +757,7 @@ func restoreJobDescription(
for i, backup := range from {
r.From[i] = make(tree.PartitionedBackup, len(backup))
for j, uri := range backup {
sf, err := cloud.SanitizeExternalStorageURI(uri, nil /* extraParams */)
sf, err := cloudimpl.SanitizeExternalStorageURI(uri, nil /* extraParams */)
if err != nil {
return "", err
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -104,7 +104,7 @@ func showBackupPlanHook(

incPaths, err := findPriorBackups(ctx, store)
if err != nil {
if errors.Is(err, cloud.ErrListingUnsupported) {
if errors.Is(err, cloudimpl.ErrListingUnsupported) {
// If we do not support listing, we have to just assume there are none
// and show the specified base.
log.Warningf(ctx, "storage sink %T does not support listing, only resolving the base backup", store)
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/changefeed_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
Expand Down Expand Up @@ -365,7 +365,7 @@ func changefeedPlanHook(
func changefeedJobDescription(
p sql.PlanHookState, changefeed *tree.CreateChangefeed, sinkURI string, opts map[string]string,
) (string, error) {
cleanedSinkURI, err := cloud.SanitizeExternalStorageURI(sinkURI, []string{changefeedbase.SinkParamSASLPassword})
cleanedSinkURI, err := cloudimpl.SanitizeExternalStorageURI(sinkURI, []string{changefeedbase.SinkParamSASLPassword})
if err != nil {
return "", err
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/changefeedccl/sink_cloudstorage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down Expand Up @@ -98,7 +99,7 @@ func TestCloudStorageSink(t *testing.T) {

clientFactory := blobs.TestBlobServiceClient(settings.ExternalIODir)
externalStorageFromURI := func(ctx context.Context, uri string) (cloud.ExternalStorage, error) {
return cloud.ExternalStorageFromURI(ctx, uri, base.ExternalIODirConfig{}, settings, clientFactory)
return cloudimpl.ExternalStorageFromURI(ctx, uri, base.ExternalIODirConfig{}, settings, clientFactory)
}

t.Run(`golden`, func(t *testing.T) {
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/cliccl/load.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/cli"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -55,11 +56,11 @@ func runLoadShow(cmd *cobra.Command, args []string) error {
ctx := context.Background()
basepath := args[0]
if !strings.Contains(basepath, "://") {
basepath = cloud.MakeLocalStorageURI(basepath)
basepath = cloudimpl.MakeLocalStorageURI(basepath)
}

externalStorageFromURI := func(ctx context.Context, uri string) (cloud.ExternalStorage, error) {
return cloud.ExternalStorageFromURI(ctx, uri, base.ExternalIODirConfig{},
return cloudimpl.ExternalStorageFromURI(ctx, uri, base.ExternalIODirConfig{},
cluster.NoSettings, blobs.TestEmptyBlobClientFactory)
}
// This reads the raw backup descriptor (with table descriptors possibly not
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/importccl/exportcsv.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/encoding/csv"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -238,7 +238,7 @@ func (sp *csvWriter) Run(ctx context.Context) {
return errors.Wrap(err, "failed to flush csv writer")
}

conf, err := cloud.ExternalStorageConfFromURI(sp.spec.Destination)
conf, err := cloudimpl.ExternalStorageConfFromURI(sp.spec.Destination)
if err != nil {
return err
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/importccl/import_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
Expand Down Expand Up @@ -166,7 +166,7 @@ func makeInputConverter(
case roachpb.IOFileFormat_CSV:
isWorkload := true
for _, file := range spec.Uri {
if conf, err := cloud.ExternalStorageConfFromURI(file); err != nil || conf.Provider != roachpb.ExternalStorageProvider_Workload {
if conf, err := cloudimpl.ExternalStorageConfFromURI(file); err != nil || conf.Provider != roachpb.ExternalStorageProvider_Workload {
isWorkload = false
break
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/importccl/import_processor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
Expand Down Expand Up @@ -857,7 +858,7 @@ func externalStorageFactory(
if err != nil {
return nil, err
}
return cloud.MakeExternalStorage(ctx, dest, base.ExternalIODirConfig{},
return cloudimpl.MakeExternalStorage(ctx, dest, base.ExternalIODirConfig{},
nil, blobs.TestBlobServiceClient(workdir))
}

Expand Down Expand Up @@ -902,7 +903,7 @@ func newTestSpec(t *testing.T, format roachpb.IOFileFormat, inputs ...string) te
}

for id, path := range inputs {
spec.inputs[int32(id)] = cloud.MakeLocalStorageURI(path)
spec.inputs[int32(id)] = cloudimpl.MakeLocalStorageURI(path)
}

return spec
Expand Down
6 changes: 3 additions & 3 deletions pkg/ccl/importccl/import_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -173,7 +173,7 @@ func importJobDescription(
stmt.CreateDefs = defs
stmt.Files = nil
for _, file := range files {
clean, err := cloud.SanitizeExternalStorageURI(file, nil /* extraParams */)
clean, err := cloudimpl.SanitizeExternalStorageURI(file, nil /* extraParams */)
if err != nil {
return "", err
}
Expand Down Expand Up @@ -255,7 +255,7 @@ func importPlanHook(
files = filenamePatterns
} else {
for _, file := range filenamePatterns {
if cloud.URINeedsGlobExpansion(file) {
if cloudimpl.URINeedsGlobExpansion(file) {
s, err := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI(ctx, file)
if err != nil {
return err
Expand Down
5 changes: 3 additions & 2 deletions pkg/ccl/importccl/load.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -122,11 +123,11 @@ func Load(
semaCtx := tree.MakeSemaContext()

blobClientFactory := blobs.TestBlobServiceClient(writeToDir)
conf, err := cloud.ExternalStorageConfFromURI(uri)
conf, err := cloudimpl.ExternalStorageConfFromURI(uri)
if err != nil {
return backupccl.BackupManifest{}, err
}
dir, err := cloud.MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{},
dir, err := cloudimpl.MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{},
cluster.NoSettings, blobClientFactory)
if err != nil {
return backupccl.BackupManifest{}, errors.Wrap(err, "export storage from URI")
Expand Down
Loading

0 comments on commit 26aa14e

Please sign in to comment.