Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions pkg/cloud/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_library(
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/sql/isql",
"//pkg/util/buildutil",
"//pkg/util/cidr",
"//pkg/util/ctxgroup",
"//pkg/util/ioctx",
Expand Down
4 changes: 4 additions & 0 deletions pkg/cloud/amazon/s3_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,7 @@ type s3Storage struct {
prefix string
metrics *cloud.Metrics
storageOptions cloud.ExternalStorageOptions
uri string // original URI used to construct this storage

opts s3ClientConfig
cached *s3Client
Expand Down Expand Up @@ -332,6 +333,7 @@ func parseS3URL(uri *url.URL) (cloudpb.ExternalStorage, error) {
}

conf.Provider = cloudpb.ExternalStorageProvider_s3
conf.URI = uri.String()

// TODO(rui): currently the value of AssumeRoleParam is written into both of
// the RoleARN fields and the RoleProvider fields in order to support a mixed
Expand Down Expand Up @@ -519,6 +521,7 @@ func MakeS3Storage(
settings: args.Settings,
opts: clientConfig(conf),
storageOptions: args.ExternalStorageOptions(),
uri: dest.URI,
}

reuse := reuseSession.Get(&args.Settings.SV)
Expand Down Expand Up @@ -747,6 +750,7 @@ func (s *s3Storage) Conf() cloudpb.ExternalStorage {
return cloudpb.ExternalStorage{
Provider: cloudpb.ExternalStorageProvider_s3,
S3Config: s.conf,
URI: s.uri,
}
}

Expand Down
5 changes: 5 additions & 0 deletions pkg/cloud/azure/azure_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,7 @@ func parseAzureURL(uri *url.URL) (cloudpb.ExternalStorage, error) {
azureURL := cloud.ConsumeURL{URL: uri}
conf := cloudpb.ExternalStorage{}
conf.Provider = cloudpb.ExternalStorageProvider_azure
conf.URI = uri.String()
auth, err := azureAuthMethod(uri, &azureURL)
if err != nil {
return conf, err
Expand Down Expand Up @@ -221,6 +222,7 @@ type azureStorage struct {
container *container.Client
prefix string
settings *cluster.Settings
uri string // original URI used to construct this storage
}

type azCacheKey struct {
Expand Down Expand Up @@ -304,6 +306,7 @@ func makeAzureStorage(
container: azClient.NewContainerClient(conf.Container),
prefix: conf.Prefix,
settings: args.Settings,
uri: dest.URI,
}, nil
}
}
Expand Down Expand Up @@ -365,6 +368,7 @@ func makeAzureStorage(
container: azClient.NewContainerClient(conf.Container),
prefix: conf.Prefix,
settings: args.Settings,
uri: dest.URI,
}, nil
}

Expand All @@ -377,6 +381,7 @@ func (s *azureStorage) Conf() cloudpb.ExternalStorage {
return cloudpb.ExternalStorage{
Provider: cloudpb.ExternalStorageProvider_azure,
AzureConfig: s.conf,
URI: s.uri,
}
}

Expand Down
5 changes: 1 addition & 4 deletions pkg/cloud/cloudpb/external_storage.proto
Original file line number Diff line number Diff line change
Expand Up @@ -178,10 +178,7 @@ message ExternalStorage {
ExternalConnectionConfig external_connection_config = 9 [(gogoproto.nullable) = false];

// URI is the string URI from which this encoded external storage config was
// derived, if known. May be empty in most cases unless set explicitly by the
// caller who created the config from a URI.
// TODO(dt): It would be nice if this were always set but we would need every
// implementation of ExternalStorage to do so in its Conf() method.
// derived. May be empty if the storage configuration was not built from a URI.
string URI = 10;
}

29 changes: 24 additions & 5 deletions pkg/cloud/externalconn/connection_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ func parseExternalConnectionURL(
) (cloudpb.ExternalStorage, error) {
conf := cloudpb.ExternalStorage{}
conf.Provider = cloudpb.ExternalStorageProvider_external
conf.URI = uri.String()
var err error
conf.ExternalConnectionConfig, err = makeExternalConnectionConfig(uri, args)
return conf, err
Expand Down Expand Up @@ -79,15 +80,33 @@ func makeExternalConnectionStorage(
switch d := ec.ConnectionProto().Details.(type) {
case *connectionpb.ConnectionDetails_SimpleURI:
// Append the subdirectory that was passed in with the `external` URI to the
// underlying `nodelocal` URI.
// underlying storage URI.
uri, err := url.Parse(d.SimpleURI.URI)
if err != nil {
return nil, errors.Wrap(err, "failed to parse `nodelocal` URI")
return nil, errors.Wrap(err, "failed to parse underlying storage URI")
}
uri.Path = path.Join(uri.Path, cfg.Path)
return cloud.ExternalStorageFromURI(ctx, uri.String(), args.IOConf, args.Settings,
args.BlobClientFactory, username.MakeSQLUsernameFromPreNormalizedString(cfg.User),
args.DB, args.Limiters, args.MetricsRecorder, args.Options...)

// Parse the resolved URI to get the underlying storage configuration.
resolvedConf, err := cloud.ExternalStorageConfFromURI(
uri.String(),
username.MakeSQLUsernameFromPreNormalizedString(cfg.User),
)
if err != nil {
return nil, errors.Wrap(err, "failed to create config for resolved URI")
}

// Override the URI field to preserve the original external:// URI.
// This ensures that the underlying storage returns the external:// URI
// from its Conf() method rather than the resolved storage URI.
resolvedConf.URI = dest.URI

// Create the storage using the modified config.
return cloud.MakeExternalStorage(
ctx, resolvedConf, args.IOConf, args.Settings,
args.BlobClientFactory, args.DB, args.Limiters,
args.MetricsRecorder, args.Options...,
)
default:
return nil, errors.Newf("cannot connect to %T; unsupported resource for an ExternalStorage connection", d)
}
Expand Down
4 changes: 4 additions & 0 deletions pkg/cloud/gcp/gcs_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@ func parseGSURL(uri *url.URL) (cloudpb.ExternalStorage, error) {
gsURL := cloud.ConsumeURL{URL: uri}
conf := cloudpb.ExternalStorage{}
conf.Provider = cloudpb.ExternalStorageProvider_gs
conf.URI = uri.String()
assumeRole, delegateRoles := cloud.ParseRoleString(gsURL.ConsumeParam(AssumeRoleParam))
conf.GoogleCloudConfig = &cloudpb.ExternalStorage_GCS{
Bucket: uri.Host,
Expand Down Expand Up @@ -105,6 +106,7 @@ type gcsStorage struct {
ioConf base.ExternalIODirConfig
prefix string
settings *cluster.Settings
uri string // original URI used to construct this storage
}

var _ cloud.ExternalStorage = &gcsStorage{}
Expand All @@ -113,6 +115,7 @@ func (g *gcsStorage) Conf() cloudpb.ExternalStorage {
return cloudpb.ExternalStorage{
Provider: cloudpb.ExternalStorageProvider_gs,
GoogleCloudConfig: g.conf,
URI: g.uri,
}
}

Expand Down Expand Up @@ -223,6 +226,7 @@ func makeGCSStorage(
ioConf: args.IOConf,
prefix: conf.Prefix,
settings: args.Settings,
uri: dest.URI,
}, nil
}

Expand Down
4 changes: 4 additions & 0 deletions pkg/cloud/httpsink/http_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
func parseHTTPURL(uri *url.URL) (cloudpb.ExternalStorage, error) {
conf := cloudpb.ExternalStorage{}
conf.Provider = cloudpb.ExternalStorageProvider_http
conf.URI = uri.String()
conf.HttpPath.BaseUri = uri.String()
return conf, nil
}
Expand All @@ -41,6 +42,7 @@ type httpStorage struct {
hosts []string
settings *cluster.Settings
ioConf base.ExternalIODirConfig
uri string // original URI used to construct this storage
}

var _ cloud.ExternalStorage = &httpStorage{}
Expand Down Expand Up @@ -86,6 +88,7 @@ func MakeHTTPStorage(
hosts: strings.Split(uri.Host, ","),
settings: args.Settings,
ioConf: args.IOConf,
uri: dest.URI,
}, nil
}

Expand All @@ -95,6 +98,7 @@ func (h *httpStorage) Conf() cloudpb.ExternalStorage {
HttpPath: cloudpb.ExternalStorage_Http{
BaseUri: h.base.String(),
},
URI: h.uri,
}
}

Expand Down
16 changes: 15 additions & 1 deletion pkg/cloud/impl_registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
Expand Down Expand Up @@ -216,8 +217,21 @@ func ExternalStorageFromURI(
if err != nil {
return nil, err
}
return MakeExternalStorage(ctx, conf, externalConfig, settings, blobClientFactory,
es, err := MakeExternalStorage(ctx, conf, externalConfig, settings, blobClientFactory,
db, limiters, metrics, opts...)
if err != nil {
return nil, err
}
if buildutil.CrdbTestBuild {
// Verify that the Conf() method returns the URI field.
returnedConf := es.Conf()
if returnedConf.URI != conf.URI {
return nil, errors.AssertionFailedf(
"ExternalStorage.Conf() did not return the original URI: expected %q, got %q",
uri, returnedConf.URI)
}
}
return es, nil
}

// MakeExternalStorage creates an ExternalStorage from the given config.
Expand Down
13 changes: 11 additions & 2 deletions pkg/cloud/nodelocal/nodelocal_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@ func parseLocalFileURI(
}

conf.Provider = cloudpb.ExternalStorageProvider_nodelocal
conf.URI = uri.String()
var err error
conf.LocalFileConfig, err = makeLocalFileConfig(uri)
return conf, err
Expand All @@ -87,6 +88,7 @@ type localFileStorage struct {
base string // relative filepath prefixed with externalIODir, for I/O ops on this node.
blobClient blobs.BlobClient // inter-node file sharing service
settings *cluster.Settings // cluster settings for the ExternalStorage
uri string // original URI used to construct this storage
}

var _ cloud.ExternalStorage = &localFileStorage{}
Expand Down Expand Up @@ -114,14 +116,21 @@ func makeLocalFileStorage(
if err != nil {
return nil, errors.Wrap(err, "failed to create blob client")
}
return &localFileStorage{base: cfg.Path, cfg: cfg, ioConf: args.IOConf, blobClient: client,
settings: args.Settings}, nil
return &localFileStorage{
base: cfg.Path,
cfg: cfg,
ioConf: args.IOConf,
blobClient: client,
settings: args.Settings,
uri: dest.URI,
}, nil
}

func (l *localFileStorage) Conf() cloudpb.ExternalStorage {
return cloudpb.ExternalStorage{
Provider: cloudpb.ExternalStorageProvider_nodelocal,
LocalFileConfig: l.cfg,
URI: l.uri,
}
}

Expand Down
1 change: 0 additions & 1 deletion pkg/cloud/nullsink/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ go_test(
deps = [
"//pkg/base",
"//pkg/cloud",
"//pkg/cloud/cloudpb",
"//pkg/security/username",
"//pkg/util/leaktest",
"@com_github_cockroachdb_errors//:errors",
Expand Down
19 changes: 14 additions & 5 deletions pkg/cloud/nullsink/nullsink_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,13 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
)

func parseNullURL(_ cloud.ExternalStorageURIContext, _ *url.URL) (cloudpb.ExternalStorage, error) {
return cloudpb.ExternalStorage{Provider: cloudpb.ExternalStorageProvider_null}, nil
func parseNullURL(
_ cloud.ExternalStorageURIContext, uri *url.URL,
) (cloudpb.ExternalStorage, error) {
return cloudpb.ExternalStorage{
Provider: cloudpb.ExternalStorageProvider_null,
URI: uri.String(),
}, nil
}

// NullRequiresExternalIOAccounting is the return falues for
Expand All @@ -33,23 +38,27 @@ func MakeNullSinkStorageURI(path string) string {
}

type nullSinkStorage struct {
uri string // original URI used to construct this storage
}

var _ cloud.ExternalStorage = &nullSinkStorage{}

func makeNullSinkStorage(
_ context.Context, _ cloud.ExternalStorageContext, _ cloudpb.ExternalStorage,
_ context.Context, _ cloud.ExternalStorageContext, dest cloudpb.ExternalStorage,
) (cloud.ExternalStorage, error) {
telemetry.Count("external-io.nullsink")
return &nullSinkStorage{}, nil
return &nullSinkStorage{uri: dest.URI}, nil
}

func (n *nullSinkStorage) Close() error {
return nil
}

func (n *nullSinkStorage) Conf() cloudpb.ExternalStorage {
return cloudpb.ExternalStorage{Provider: cloudpb.ExternalStorageProvider_null}
return cloudpb.ExternalStorage{
Provider: cloudpb.ExternalStorageProvider_null,
URI: n.uri,
}
}

func (n *nullSinkStorage) ExternalIOConf() base.ExternalIODirConfig {
Expand Down
7 changes: 5 additions & 2 deletions pkg/cloud/nullsink/nullsink_storage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/cloud/cloudpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -43,7 +42,11 @@ func TestNullSinkReadAndWrite(t *testing.T) {
}
defer s.Close()

require.Equal(t, cloudpb.ExternalStorage{Provider: cloudpb.ExternalStorageProvider_null}, s.Conf())
// Test that URI round-trips through Conf().
returnedConf := s.Conf()
require.Equal(t, dest, returnedConf.URI, "URI should round-trip through Conf()")

// Nullsink-specific behavior: writes are no-ops, reads return EOF, size is 0.
require.NoError(t, cloud.WriteFile(ctx, s, "", bytes.NewReader([]byte("abc"))))
sz, err := s.Size(ctx, "")
require.NoError(t, err)
Expand Down
5 changes: 5 additions & 0 deletions pkg/cloud/userfile/file_table_storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ func parseUserfileURL(
}

conf.Provider = cloudpb.ExternalStorageProvider_userfile
conf.URI = uri.String()
conf.FileTableConfig.User = normUser
conf.FileTableConfig.QualifiedTableName = qualifiedTableName
conf.FileTableConfig.Path = uri.Path
Expand All @@ -77,6 +78,7 @@ type fileTableStorage struct {
ioConf base.ExternalIODirConfig
prefix string // relative filepath
settings *cluster.Settings
uri string // original URI used to construct this storage
}

var _ cloud.ExternalStorage = &fileTableStorage{}
Expand Down Expand Up @@ -126,6 +128,7 @@ func makeFileTableStorage(
ioConf: args.IOConf,
prefix: cfg.Path,
settings: args.Settings,
uri: dest.URI,
}, nil
}

Expand Down Expand Up @@ -156,6 +159,7 @@ func MakeSQLConnFileTableStorage(
ioConf: base.ExternalIODirConfig{},
prefix: prefix,
settings: nil,
uri: "", // CLI path doesn't have access to original URI
}, nil
}

Expand All @@ -176,6 +180,7 @@ func (f *fileTableStorage) Conf() cloudpb.ExternalStorage {
return cloudpb.ExternalStorage{
Provider: cloudpb.ExternalStorageProvider_userfile,
FileTableConfig: f.cfg,
URI: f.uri,
}
}

Expand Down