From 9c0bf630773f88a24aee26f1fbedfcd5925d08e3 Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Tue, 9 Aug 2022 16:11:58 -0400 Subject: [PATCH 1/7] externalconn: write,list,read from ExternalStorage on `CREATE EXTERNAL CONNECTION` Previously, `CREATE EXTERNAL CONNECTION` for an `ExternalStorage` would only parse the URI and perform basic validation on the URI construct. This change teaches the factory to actually check the ExternalStorage is reachable, by writing a sentinel file, listing it, and reading the file back from it. By doing this any configuration errors will be surfaced to the user at the time the external connection is being created rather than when it is going to be used in a backup or changefeed etc. This offers a better UX and stronger guarantees that the underlying resource is valid. This change adds a testing knob to skip this validation step for certain datadriven tests that are only concerned with the creation and dropping of the external connection object, and do not have the correct credentials to read,write and list from the external storage. The provider specific unit tests in `pkg/ccl/cloudccl` however will perfrom this validation. Release note (sql change): Creating an external connection to represent and ExternalStorage will now write a sentinel file, list it, and read the contents back to validate the underlying resource. --- pkg/BUILD.bazel | 2 + pkg/base/testing_knobs.go | 1 + .../changefeedccl/sink_kafka_connection.go | 3 +- pkg/ccl/cloudccl/externalconn/BUILD.bazel | 3 +- .../cloudccl/externalconn/datadriven_test.go | 27 +++++- .../testdata/create_drop_external_connection | 20 ++++- .../create_drop_external_connection | 11 ++- pkg/cloud/amazon/BUILD.bazel | 2 + pkg/cloud/amazon/s3_connection.go | 11 +-- pkg/cloud/externalconn/BUILD.bazel | 1 + pkg/cloud/externalconn/connection.go | 4 +- pkg/cloud/externalconn/impl_registry.go | 22 ++++- pkg/cloud/externalconn/testutils/BUILD.bazel | 23 +++++ .../{utils => testutils}/cluster.go | 2 +- .../{utils => testutils}/tenant_state.go | 2 +- pkg/cloud/externalconn/utils/BUILD.bazel | 15 ++-- .../externalconn/utils/connection_utils.go | 84 +++++++++++++++++++ pkg/cloud/gcp/BUILD.bazel | 1 + pkg/cloud/gcp/gcs_kms.go | 5 +- pkg/cloud/gcp/gcs_kms_connection.go | 5 +- pkg/cloud/nodelocal/BUILD.bazel | 2 + pkg/cloud/nodelocal/nodelocal_connection.go | 8 +- pkg/server/BUILD.bazel | 1 + pkg/server/server_sql.go | 4 + pkg/sql/BUILD.bazel | 3 - pkg/sql/create_external_connection.go | 2 +- pkg/sql/exec_util.go | 2 + pkg/sql/logictest/logic.go | 3 + pkg/sql/tests/BUILD.bazel | 3 + pkg/sql/{ => tests}/copy_file_upload_test.go | 32 +++---- 30 files changed, 243 insertions(+), 61 deletions(-) create mode 100644 pkg/cloud/externalconn/testutils/BUILD.bazel rename pkg/cloud/externalconn/{utils => testutils}/cluster.go (99%) rename pkg/cloud/externalconn/{utils => testutils}/tenant_state.go (98%) create mode 100644 pkg/cloud/externalconn/utils/connection_utils.go rename pkg/sql/{ => tests}/copy_file_upload_test.go (91%) diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 178177aeff7..5413bf87709 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -799,6 +799,7 @@ GO_TARGETS = [ "//pkg/cloud/cloudtestutils:cloudtestutils", "//pkg/cloud/externalconn/connectionpb:connectionpb", "//pkg/cloud/externalconn/providers:providers", + "//pkg/cloud/externalconn/testutils:testutils", "//pkg/cloud/externalconn/utils:utils", "//pkg/cloud/externalconn:externalconn", "//pkg/cloud/gcp:gcp", @@ -2180,6 +2181,7 @@ GET_X_DATA_TARGETS = [ "//pkg/cloud/externalconn:get_x_data", "//pkg/cloud/externalconn/connectionpb:get_x_data", "//pkg/cloud/externalconn/providers:get_x_data", + "//pkg/cloud/externalconn/testutils:get_x_data", "//pkg/cloud/externalconn/utils:get_x_data", "//pkg/cloud/gcp:get_x_data", "//pkg/cloud/httpsink:get_x_data", diff --git a/pkg/base/testing_knobs.go b/pkg/base/testing_knobs.go index 893356cf2dd..5bd6902a35e 100644 --- a/pkg/base/testing_knobs.go +++ b/pkg/base/testing_knobs.go @@ -51,4 +51,5 @@ type TestingKnobs struct { CapturedIndexUsageStatsKnobs ModuleTestingKnobs AdmissionControl ModuleTestingKnobs UnusedIndexRecommendKnobs ModuleTestingKnobs + ExternalConnection ModuleTestingKnobs } diff --git a/pkg/ccl/changefeedccl/sink_kafka_connection.go b/pkg/ccl/changefeedccl/sink_kafka_connection.go index 03ea49e20d4..00b7ca6b288 100644 --- a/pkg/ccl/changefeedccl/sink_kafka_connection.go +++ b/pkg/ccl/changefeedccl/sink_kafka_connection.go @@ -15,11 +15,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase" "github.com/cockroachdb/cockroach/pkg/cloud/externalconn" "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb" + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/errors" ) func parseAndValidateKafkaSinkURI( - ctx context.Context, uri *url.URL, + ctx context.Context, _ interface{}, _ username.SQLUsername, uri *url.URL, ) (externalconn.ExternalConnection, error) { // Validate the kafka URI by creating a kafka sink and throwing it away. // diff --git a/pkg/ccl/cloudccl/externalconn/BUILD.bazel b/pkg/ccl/cloudccl/externalconn/BUILD.bazel index e9c5d68e93c..d99abb58536 100644 --- a/pkg/ccl/cloudccl/externalconn/BUILD.bazel +++ b/pkg/ccl/cloudccl/externalconn/BUILD.bazel @@ -12,8 +12,9 @@ go_test( "//pkg/base", "//pkg/ccl/changefeedccl", "//pkg/ccl/kvccl/kvtenantccl", + "//pkg/cloud/externalconn", "//pkg/cloud/externalconn/providers", - "//pkg/cloud/externalconn/utils", + "//pkg/cloud/externalconn/testutils", "//pkg/jobs", "//pkg/roachpb", "//pkg/security/securityassets", diff --git a/pkg/ccl/cloudccl/externalconn/datadriven_test.go b/pkg/ccl/cloudccl/externalconn/datadriven_test.go index 5ea9113c010..6c103443828 100644 --- a/pkg/ccl/cloudccl/externalconn/datadriven_test.go +++ b/pkg/ccl/cloudccl/externalconn/datadriven_test.go @@ -15,9 +15,10 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" - _ "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl" // register the sink External Connection implementations + _ "github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl" + "github.com/cockroachdb/cockroach/pkg/cloud/externalconn" _ "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/providers" // register all the concrete External Connection implementations - "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/utils" + ectestutils "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/testutils" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -33,16 +34,28 @@ func TestDataDriven(t *testing.T) { ctx := context.Background() datadriven.Walk(t, testutils.TestDataPath(t), func(t *testing.T, path string) { + dir, dirCleanupFn := testutils.TempDir(t) + defer dirCleanupFn() + + var skipCheckExternalStorageConnection bool + ecTestingKnobs := &externalconn.TestingKnobs{ + SkipCheckingExternalStorageConnection: func() bool { + return skipCheckExternalStorageConnection + }, + } tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ Knobs: base.TestingKnobs{ - JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speeds up test + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), // speeds up test + ExternalConnection: ecTestingKnobs, }, + ExternalIODirConfig: base.ExternalIODirConfig{}, + ExternalIODir: dir, }, }) defer tc.Stopper().Stop(ctx) - externalConnTestCluster := utils.NewHandle(t, tc) + externalConnTestCluster := ectestutils.NewHandle(t, tc) defer externalConnTestCluster.Cleanup() externalConnTestCluster.InitializeTenant(ctx, roachpb.SystemTenantID) @@ -63,6 +76,12 @@ func TestDataDriven(t *testing.T) { case "initialize": externalConnTestCluster.InitializeTenant(ctx, tenantID) + case "enable-check-external-storage": + skipCheckExternalStorageConnection = false + + case "disable-check-external-storage": + skipCheckExternalStorageConnection = true + case "exec-sql": if d.HasArg("user") { var user string diff --git a/pkg/ccl/cloudccl/externalconn/testdata/create_drop_external_connection b/pkg/ccl/cloudccl/externalconn/testdata/create_drop_external_connection index 2aae1d87d31..bbf36c927fb 100644 --- a/pkg/ccl/cloudccl/externalconn/testdata/create_drop_external_connection +++ b/pkg/ccl/cloudccl/externalconn/testdata/create_drop_external_connection @@ -12,12 +12,12 @@ foo STORAGE {"provider": "nodelocal", "simpleUri": {"uri": "nodelocal://1/foo/ba exec-sql CREATE EXTERNAL CONNECTION "missing-node-id" AS 'nodelocal:///foo'; ---- -pq: failed to construct External Connection details: invalid `nodelocal` URI: host component of nodelocal URI must be a node ID (use 'self' to specify each node should access its own local filesystem): nodelocal:///foo +pq: failed to construct External Connection details: failed to create nodelocal external connection: invalid `nodelocal` URI: host component of nodelocal URI must be a node ID (use 'self' to specify each node should access its own local filesystem): nodelocal:///foo exec-sql CREATE EXTERNAL CONNECTION "invalid-nodeid-nodelocal" AS 'nodelocal://a/foo'; ---- -pq: failed to construct External Connection details: invalid `nodelocal` URI: host component of nodelocal URI must be a node ID: nodelocal://a/foo +pq: failed to construct External Connection details: failed to create nodelocal external connection: invalid `nodelocal` URI: host component of nodelocal URI must be a node ID: nodelocal://a/foo # Try to create another External Connection with the same name. exec-sql @@ -142,6 +142,9 @@ subtest end subtest basic-gs-kms +disable-check-external-storage +---- + exec-sql CREATE EXTERNAL CONNECTION "foo-kms" AS 'gs:///cmk?AUTH=implicit&CREDENTIALS=baz&ASSUME_ROLE=ronaldo,rashford,bruno&BEARER_TOKEN=foo'; ---- @@ -168,10 +171,16 @@ DROP EXTERNAL CONNECTION "foo-kms"; inspect-system-table ---- +enable-check-external-storage +---- + subtest end subtest basic-s3 +disable-check-external-storage +---- + exec-sql CREATE EXTERNAL CONNECTION "foo-s3" AS 's3://foo/bar?AUTH=implicit&AWS_ACCESS_KEY_ID=123&AWS_SECRET_ACCESS_KEY=456&ASSUME_ROLE=ronaldo,rashford,bruno'; ---- @@ -180,12 +189,12 @@ CREATE EXTERNAL CONNECTION "foo-s3" AS 's3://foo/bar?AUTH=implicit&AWS_ACCESS_KE exec-sql CREATE EXTERNAL CONNECTION "missing-host-s3" AS 's3:///?AUTH=implicit'; ---- -pq: failed to construct External Connection details: empty host component; s3 URI must specify a target bucket +pq: failed to construct External Connection details: failed to create s3 external connection: empty host component; s3 URI must specify a target bucket exec-sql CREATE EXTERNAL CONNECTION "invalid-params-s3" AS 's3://foo/bar?AUTH=implicit&INVALIDPARAM=baz'; ---- -pq: failed to construct External Connection details: unknown S3 query parameters: INVALIDPARAM +pq: failed to construct External Connection details: failed to create s3 external connection: unknown S3 query parameters: INVALIDPARAM inspect-system-table ---- @@ -198,6 +207,9 @@ DROP EXTERNAL CONNECTION "foo-s3"; inspect-system-table ---- +enable-check-external-storage +---- + subtest end subtest basic-kafka-sink diff --git a/pkg/ccl/cloudccl/externalconn/testdata/multi-tenant/create_drop_external_connection b/pkg/ccl/cloudccl/externalconn/testdata/multi-tenant/create_drop_external_connection index b3aba38f0ea..f47c1024976 100644 --- a/pkg/ccl/cloudccl/externalconn/testdata/multi-tenant/create_drop_external_connection +++ b/pkg/ccl/cloudccl/externalconn/testdata/multi-tenant/create_drop_external_connection @@ -15,12 +15,12 @@ foo STORAGE {"provider": "nodelocal", "simpleUri": {"uri": "nodelocal://1/foo/ba exec-sql CREATE EXTERNAL CONNECTION "missing-node-id" AS 'nodelocal:///foo'; ---- -pq: failed to construct External Connection details: invalid `nodelocal` URI: host component of nodelocal URI must be a node ID (use 'self' to specify each node should access its own local filesystem): nodelocal:///foo +pq: failed to construct External Connection details: failed to create nodelocal external connection: invalid `nodelocal` URI: host component of nodelocal URI must be a node ID (use 'self' to specify each node should access its own local filesystem): nodelocal:///foo exec-sql CREATE EXTERNAL CONNECTION "invalid-nodeid-nodelocal" AS 'nodelocal://a/foo'; ---- -pq: failed to construct External Connection details: invalid `nodelocal` URI: host component of nodelocal URI must be a node ID: nodelocal://a/foo +pq: failed to construct External Connection details: failed to create nodelocal external connection: invalid `nodelocal` URI: host component of nodelocal URI must be a node ID: nodelocal://a/foo # Try to create another External Connection with the same name. exec-sql @@ -145,6 +145,9 @@ subtest end subtest basic-gs-kms +disable-check-external-storage +---- + exec-sql CREATE EXTERNAL CONNECTION "foo-kms" AS 'gs:///cmk?AUTH=implicit&CREDENTIALS=baz&ASSUME_ROLE=ronaldo,rashford,bruno&BEARER_TOKEN=foo'; ---- @@ -172,12 +175,12 @@ CREATE EXTERNAL CONNECTION "foo-s3" AS 's3://foo/bar?AUTH=implicit&AWS_ACCESS_KE exec-sql CREATE EXTERNAL CONNECTION "missing-host-s3" AS 's3:///?AUTH=implicit'; ---- -pq: failed to construct External Connection details: empty host component; s3 URI must specify a target bucket +pq: failed to construct External Connection details: failed to create s3 external connection: empty host component; s3 URI must specify a target bucket exec-sql CREATE EXTERNAL CONNECTION "invalid-params-s3" AS 's3://foo/bar?AUTH=implicit&INVALIDPARAM=baz'; ---- -pq: failed to construct External Connection details: unknown S3 query parameters: INVALIDPARAM +pq: failed to construct External Connection details: failed to create s3 external connection: unknown S3 query parameters: INVALIDPARAM inspect-system-table ---- diff --git a/pkg/cloud/amazon/BUILD.bazel b/pkg/cloud/amazon/BUILD.bazel index d6ca46f02ba..ef2c62bc46a 100644 --- a/pkg/cloud/amazon/BUILD.bazel +++ b/pkg/cloud/amazon/BUILD.bazel @@ -16,6 +16,8 @@ go_library( "//pkg/cloud/cloudpb", "//pkg/cloud/externalconn", "//pkg/cloud/externalconn/connectionpb", + "//pkg/cloud/externalconn/utils", + "//pkg/security/username", "//pkg/server/telemetry", "//pkg/settings", "//pkg/settings/cluster", diff --git a/pkg/cloud/amazon/s3_connection.go b/pkg/cloud/amazon/s3_connection.go index ff875ca7917..87b4f16c50a 100644 --- a/pkg/cloud/amazon/s3_connection.go +++ b/pkg/cloud/amazon/s3_connection.go @@ -14,17 +14,18 @@ import ( "context" "net/url" - "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/cloud/externalconn" "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb" + "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/utils" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/errors" ) func parseAndValidateS3ConnectionURI( - _ context.Context, uri *url.URL, + ctx context.Context, execCfg interface{}, user username.SQLUsername, uri *url.URL, ) (externalconn.ExternalConnection, error) { - // Parse and validate the S3 URL. - if _, err := parseS3URL(cloud.ExternalStorageURIContext{}, uri); err != nil { - return nil, err + if err := utils.CheckExternalStorageConnection(ctx, execCfg, user, uri.String()); err != nil { + return nil, errors.Wrap(err, "failed to create s3 external connection") } connDetails := connectionpb.ConnectionDetails{ diff --git a/pkg/cloud/externalconn/BUILD.bazel b/pkg/cloud/externalconn/BUILD.bazel index 111775e3bd5..0ad707343f1 100644 --- a/pkg/cloud/externalconn/BUILD.bazel +++ b/pkg/cloud/externalconn/BUILD.bazel @@ -13,6 +13,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/cloud/externalconn", visibility = ["//visibility:public"], deps = [ + "//pkg/base", "//pkg/cloud", "//pkg/cloud/cloudpb", "//pkg/cloud/externalconn/connectionpb", diff --git a/pkg/cloud/externalconn/connection.go b/pkg/cloud/externalconn/connection.go index f0272527942..0cec91885e7 100644 --- a/pkg/cloud/externalconn/connection.go +++ b/pkg/cloud/externalconn/connection.go @@ -15,6 +15,7 @@ import ( "net/url" "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb" + "github.com/cockroachdb/cockroach/pkg/security/username" ) // ExternalConnection is the interface to the external resource represented by @@ -34,4 +35,5 @@ type ExternalConnection interface { // connectionParserFactory is the factory method that takes in an endpoint URI // for an external resource, and returns the ExternalConnection representation // of that URI. -type connectionParserFactory func(ctx context.Context, uri *url.URL) (ExternalConnection, error) +type connectionParserFactory func(ctx context.Context, execCfg interface{}, + user username.SQLUsername, uri *url.URL) (ExternalConnection, error) diff --git a/pkg/cloud/externalconn/impl_registry.go b/pkg/cloud/externalconn/impl_registry.go index 59e9d019404..93f0c9875fe 100644 --- a/pkg/cloud/externalconn/impl_registry.go +++ b/pkg/cloud/externalconn/impl_registry.go @@ -15,6 +15,8 @@ import ( "fmt" "net/url" + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/errors" ) @@ -34,7 +36,9 @@ func RegisterConnectionDetailsFromURIFactory( } // ExternalConnectionFromURI returns a ExternalConnection for the given URI. -func ExternalConnectionFromURI(ctx context.Context, uri string) (ExternalConnection, error) { +func ExternalConnectionFromURI( + ctx context.Context, execCfg interface{}, user username.SQLUsername, uri string, +) (ExternalConnection, error) { externalConnectionURI, err := url.Parse(uri) if err != nil { return nil, err @@ -46,5 +50,19 @@ func ExternalConnectionFromURI(ctx context.Context, uri string) (ExternalConnect return nil, errors.Newf("no parseFn found for external connection provider %s", externalConnectionURI.Scheme) } - return parseFn(ctx, externalConnectionURI) + return parseFn(ctx, execCfg, user, externalConnectionURI) } + +// TestingKnobs provide fine-grained control over the external connection +// components for testing. +type TestingKnobs struct { + // SkipCheckingExternalStorageConnection returns whether `CREATE EXTERNAL + // CONNECTION` should skip the step that writes, lists and reads a sentinel + // file from the underlying ExternalStorage. + SkipCheckingExternalStorageConnection func() bool +} + +// ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. +func (t *TestingKnobs) ModuleTestingKnobs() {} + +var _ base.ModuleTestingKnobs = (*TestingKnobs)(nil) diff --git a/pkg/cloud/externalconn/testutils/BUILD.bazel b/pkg/cloud/externalconn/testutils/BUILD.bazel new file mode 100644 index 00000000000..697392933bc --- /dev/null +++ b/pkg/cloud/externalconn/testutils/BUILD.bazel @@ -0,0 +1,23 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "testutils", + srcs = [ + "cluster.go", + "tenant_state.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/testutils", + visibility = ["//visibility:public"], + deps = [ + "//pkg/base", + "//pkg/roachpb", + "//pkg/security/username", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "@com_github_stretchr_testify//require", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/cloud/externalconn/utils/cluster.go b/pkg/cloud/externalconn/testutils/cluster.go similarity index 99% rename from pkg/cloud/externalconn/utils/cluster.go rename to pkg/cloud/externalconn/testutils/cluster.go index 716126e9f2c..28dbfde8ab2 100644 --- a/pkg/cloud/externalconn/utils/cluster.go +++ b/pkg/cloud/externalconn/testutils/cluster.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package utils +package testutils import ( "context" diff --git a/pkg/cloud/externalconn/utils/tenant_state.go b/pkg/cloud/externalconn/testutils/tenant_state.go similarity index 98% rename from pkg/cloud/externalconn/utils/tenant_state.go rename to pkg/cloud/externalconn/testutils/tenant_state.go index bb47cc34866..3ac69329459 100644 --- a/pkg/cloud/externalconn/utils/tenant_state.go +++ b/pkg/cloud/externalconn/testutils/tenant_state.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package utils +package testutils import ( "context" diff --git a/pkg/cloud/externalconn/utils/BUILD.bazel b/pkg/cloud/externalconn/utils/BUILD.bazel index 7344e85f143..a0327c0b892 100644 --- a/pkg/cloud/externalconn/utils/BUILD.bazel +++ b/pkg/cloud/externalconn/utils/BUILD.bazel @@ -3,20 +3,15 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "utils", - srcs = [ - "cluster.go", - "tenant_state.go", - ], + srcs = ["connection_utils.go"], importpath = "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/utils", visibility = ["//visibility:public"], deps = [ - "//pkg/base", - "//pkg/roachpb", + "//pkg/cloud", "//pkg/security/username", - "//pkg/testutils/serverutils", - "//pkg/testutils/sqlutils", - "//pkg/testutils/testcluster", - "@com_github_stretchr_testify//require", + "//pkg/sql", + "//pkg/util/ioctx", + "@com_github_cockroachdb_errors//:errors", ], ) diff --git a/pkg/cloud/externalconn/utils/connection_utils.go b/pkg/cloud/externalconn/utils/connection_utils.go new file mode 100644 index 00000000000..ebbfba53d06 --- /dev/null +++ b/pkg/cloud/externalconn/utils/connection_utils.go @@ -0,0 +1,84 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package utils + +import ( + "bytes" + "context" + "io/ioutil" + "strings" + + "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/util/ioctx" + "github.com/cockroachdb/errors" +) + +const markerFile = "crdb_external_storage_location" + +// CheckExternalStorageConnection writes a sentinel file, lists the file, and reads the file +// back. This serves as a sanity check that the external connection represents +// an ExternalStorage resource that can be connected and interacted with.. +func CheckExternalStorageConnection( + ctx context.Context, execCfg interface{}, user username.SQLUsername, uri string, +) error { + cfg := execCfg.(*sql.ExecutorConfig) + es, err := cfg.DistSQLSrv.ExternalStorageFromURI(ctx, uri, user) + if err != nil { + return err + } + defer es.Close() + + if cfg.ExternalConnectionTestingKnobs != nil && + cfg.ExternalConnectionTestingKnobs.SkipCheckingExternalStorageConnection != nil { + if cfg.ExternalConnectionTestingKnobs.SkipCheckingExternalStorageConnection() { + return nil + } + } + + // Write a sentinel file. + markerContent := "a CockroachDB cluster has been configured to read and write to this location" + if err := cloud.WriteFile(ctx, es, markerFile, bytes.NewReader([]byte(markerContent))); err != nil { + return errors.Wrap(err, "failed to write a sentinel ExternalStorage file") + } + + // List the sentinel file. + var foundFile bool + if err := es.List(ctx, "", "", func(s string) error { + paths := strings.Split(s, "/") + s = paths[len(paths)-1] + if match := strings.HasPrefix(s, markerFile); match { + foundFile = true + } + return err + }); err != nil { + return errors.Wrap(err, "failed to list sentinel ExternalStorage file") + } else if !foundFile { + return errors.Newf("failed to find sentinel ExternalStorage file '%s'", markerFile) + } + + // Read the sentinel file. + reader, err := es.ReadFile(ctx, markerFile) + if err != nil { + return errors.Wrap(err, "failed to read sentinel ExternalStorage file") + } + + content, err := ioutil.ReadAll(ioctx.ReaderCtxAdapter(ctx, reader)) + if err != nil { + return errors.Wrap(err, "failed to read sentinel ExternalStorage file content") + } + if markerContent != string(content) { + return errors.Newf("content mismatch, expected: %s but found: %s", markerContent, string(content)) + } + + return nil +} diff --git a/pkg/cloud/gcp/BUILD.bazel b/pkg/cloud/gcp/BUILD.bazel index 8e01b1db71a..b9947b4027c 100644 --- a/pkg/cloud/gcp/BUILD.bazel +++ b/pkg/cloud/gcp/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/cloud/cloudpb", "//pkg/cloud/externalconn", "//pkg/cloud/externalconn/connectionpb", + "//pkg/security/username", "//pkg/server/telemetry", "//pkg/settings", "//pkg/settings/cluster", diff --git a/pkg/cloud/gcp/gcs_kms.go b/pkg/cloud/gcp/gcs_kms.go index 7f4c873df39..7c4d2e2313f 100644 --- a/pkg/cloud/gcp/gcs_kms.go +++ b/pkg/cloud/gcp/gcs_kms.go @@ -67,7 +67,8 @@ func resolveKMSURIParams(kmsURI url.URL) kmsURIParams { return params } -func validateKMSURI(uri url.URL) error { +// ValidateKMSURI validates the passed in URI. +func ValidateKMSURI(uri url.URL) error { if uri.Path == "/" { return errors.Newf("host component of the GCS KMS cannot be empty; must contain the Customer Managed Key") } @@ -86,7 +87,7 @@ func MakeGCSKMS(ctx context.Context, uri string, env cloud.KMSEnv) (cloud.KMS, e } // Validate the URI parameters. - if err := validateKMSURI(*kmsURI); err != nil { + if err := ValidateKMSURI(*kmsURI); err != nil { return nil, err } diff --git a/pkg/cloud/gcp/gcs_kms_connection.go b/pkg/cloud/gcp/gcs_kms_connection.go index 8f13eb92f55..268ac653bd7 100644 --- a/pkg/cloud/gcp/gcs_kms_connection.go +++ b/pkg/cloud/gcp/gcs_kms_connection.go @@ -16,12 +16,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud/externalconn" "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb" + "github.com/cockroachdb/cockroach/pkg/security/username" ) func parseAndValidateGCSKMSConnectionURI( - _ context.Context, uri *url.URL, + _ context.Context, _ interface{}, _ username.SQLUsername, uri *url.URL, ) (externalconn.ExternalConnection, error) { - if err := validateKMSURI(*uri); err != nil { + if err := ValidateKMSURI(*uri); err != nil { return nil, err } diff --git a/pkg/cloud/nodelocal/BUILD.bazel b/pkg/cloud/nodelocal/BUILD.bazel index 22312234fae..f57283e407c 100644 --- a/pkg/cloud/nodelocal/BUILD.bazel +++ b/pkg/cloud/nodelocal/BUILD.bazel @@ -16,7 +16,9 @@ go_library( "//pkg/cloud/cloudpb", "//pkg/cloud/externalconn", "//pkg/cloud/externalconn/connectionpb", + "//pkg/cloud/externalconn/utils", "//pkg/roachpb", + "//pkg/security/username", "//pkg/server/telemetry", "//pkg/settings/cluster", "//pkg/util/ioctx", diff --git a/pkg/cloud/nodelocal/nodelocal_connection.go b/pkg/cloud/nodelocal/nodelocal_connection.go index a714066f675..a1c68b5a08d 100644 --- a/pkg/cloud/nodelocal/nodelocal_connection.go +++ b/pkg/cloud/nodelocal/nodelocal_connection.go @@ -16,14 +16,16 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud/externalconn" "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/connectionpb" + "github.com/cockroachdb/cockroach/pkg/cloud/externalconn/utils" + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/errors" ) func parseAndValidateLocalFileConnectionURI( - _ context.Context, uri *url.URL, + ctx context.Context, execCfg interface{}, user username.SQLUsername, uri *url.URL, ) (externalconn.ExternalConnection, error) { - if err := validateLocalFileURI(uri); err != nil { - return nil, errors.Wrap(err, "invalid `nodelocal` URI") + if err := utils.CheckExternalStorageConnection(ctx, execCfg, user, uri.String()); err != nil { + return nil, errors.Wrap(err, "failed to create nodelocal external connection") } connDetails := connectionpb.ConnectionDetails{ diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index dc765f99c9f..376d3750523 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -80,6 +80,7 @@ go_library( "//pkg/build", "//pkg/cloud", "//pkg/cloud/cloudpb", + "//pkg/cloud/externalconn", "//pkg/clusterversion", "//pkg/config", "//pkg/config/zonepb", diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 427121f8907..e12f5924867 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/blobs" "github.com/cockroachdb/cockroach/pkg/blobs/blobspb" "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/cloud/externalconn" "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/jobs" @@ -901,6 +902,9 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { if unusedIndexRecommendationsKnobs := cfg.TestingKnobs.UnusedIndexRecommendKnobs; unusedIndexRecommendationsKnobs != nil { execCfg.UnusedIndexRecommendationsKnobs = unusedIndexRecommendationsKnobs.(*idxusage.UnusedIndexRecommendationTestingKnobs) } + if externalConnKnobs := cfg.TestingKnobs.ExternalConnection; externalConnKnobs != nil { + execCfg.ExternalConnectionTestingKnobs = externalConnKnobs.(*externalconn.TestingKnobs) + } statsRefresher := stats.MakeRefresher( cfg.AmbientCtx, diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 26a94b9db3d..3112e96c38a 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -531,7 +531,6 @@ go_test( "conn_executor_test.go", "conn_io_test.go", "constraint_test.go", - "copy_file_upload_test.go", "copy_in_test.go", "copy_test.go", "crdb_internal_test.go", @@ -636,7 +635,6 @@ go_test( "//pkg/base", "//pkg/build/bazel", "//pkg/ccl/kvccl/kvtenantccl", - "//pkg/cloud/impl:cloudimpl", "//pkg/clusterversion", "//pkg/config", "//pkg/config/zonepb", @@ -746,7 +744,6 @@ go_test( "//pkg/util/fsm", "//pkg/util/hlc", "//pkg/util/httputil", - "//pkg/util/ioctx", "//pkg/util/json", "//pkg/util/leaktest", "//pkg/util/log", diff --git a/pkg/sql/create_external_connection.go b/pkg/sql/create_external_connection.go index af34adf6547..96782d0c024 100644 --- a/pkg/sql/create_external_connection.go +++ b/pkg/sql/create_external_connection.go @@ -102,7 +102,7 @@ func (p *planner) createExternalConnection( if err != nil { return errors.Wrap(err, "failed to resolve External Connection endpoint") } - exConn, err := externalconn.ExternalConnectionFromURI(params.ctx, as) + exConn, err := externalconn.ExternalConnectionFromURI(params.ctx, params.ExecCfg(), p.User(), as) if err != nil { return errors.Wrap(err, "failed to construct External Connection details") } diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 0b4e6de41b1..1e8cdddfcca 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/apd/v3" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/cloud/externalconn" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/config" @@ -1212,6 +1213,7 @@ type ExecutorConfig struct { SpanConfigTestingKnobs *spanconfig.TestingKnobs CaptureIndexUsageStatsKnobs *scheduledlogging.CaptureIndexUsageStatsTestingKnobs UnusedIndexRecommendationsKnobs *idxusage.UnusedIndexRecommendationTestingKnobs + ExternalConnectionTestingKnobs *externalconn.TestingKnobs // HistogramWindowInterval is (server.Config).HistogramWindowInterval. HistogramWindowInterval time.Duration diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go index 8e30021d774..8a294fe7dc0 100644 --- a/pkg/sql/logictest/logic.go +++ b/pkg/sql/logictest/logic.go @@ -22,6 +22,7 @@ import ( "math/rand" "net/url" "os" + "path" "path/filepath" "reflect" "regexp" @@ -1298,6 +1299,8 @@ func (t *logicTest) newCluster( TempStorageConfig: ¶ms.ServerArgs.TempStorageConfig, Locality: paramsPerNode[i].Locality, TracingDefault: params.ServerArgs.TracingDefault, + // Give every tenant its own ExternalIO directory. + ExternalIODir: path.Join(t.sharedIODir, strconv.Itoa(i)), } tenant, err := t.cluster.Server(i).StartTenant(context.Background(), tenantArgs) diff --git a/pkg/sql/tests/BUILD.bazel b/pkg/sql/tests/BUILD.bazel index 2ac8f075ccb..dfeb40b24f6 100644 --- a/pkg/sql/tests/BUILD.bazel +++ b/pkg/sql/tests/BUILD.bazel @@ -29,6 +29,7 @@ go_test( srcs = [ "autocommit_extended_protocol_test.go", "bank_test.go", + "copy_file_upload_test.go", "empty_query_test.go", "enum_test.go", "hash_sharded_test.go", @@ -56,6 +57,7 @@ go_test( "//pkg/bench", "//pkg/ccl", "//pkg/ccl/utilccl", + "//pkg/cloud/impl:cloudimpl", "//pkg/clusterversion", "//pkg/config/zonepb", "//pkg/internal/rsg", @@ -99,6 +101,7 @@ go_test( "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util/ctxgroup", + "//pkg/util/ioctx", "//pkg/util/json", "//pkg/util/leaktest", "//pkg/util/log", diff --git a/pkg/sql/copy_file_upload_test.go b/pkg/sql/tests/copy_file_upload_test.go similarity index 91% rename from pkg/sql/copy_file_upload_test.go rename to pkg/sql/tests/copy_file_upload_test.go index 8db261e139d..56cafce87a2 100644 --- a/pkg/sql/copy_file_upload_test.go +++ b/pkg/sql/tests/copy_file_upload_test.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package sql +package tests import ( "bytes" @@ -25,7 +25,7 @@ import ( _ "github.com/cockroachdb/cockroach/pkg/cloud/impl" // register cloud storage providers "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -39,7 +39,7 @@ import ( const defaultQualifiedDBSchemaName = "defaultdb.public." const filename = "/test/test_file_upload.csv" -var fileUploadModes = []string{NodelocalFileUploadTable, UserFileUploadTable} +var fileUploadModes = []string{sql.NodelocalFileUploadTable, sql.UserFileUploadTable} func writeFile(t *testing.T, testSendFile string, fileContent []byte) { err := os.MkdirAll(filepath.Dir(testSendFile), 0755) @@ -57,10 +57,10 @@ func prepareFileUploadURI( ) (string, error) { var uri string switch copyInternalTable { - case NodelocalFileUploadTable: + case sql.NodelocalFileUploadTable: testSendFile = strings.TrimPrefix(testSendFile, "/") uri = fmt.Sprintf("nodelocal://self/%s", testSendFile) - case UserFileUploadTable: + case sql.UserFileUploadTable: if !strings.HasPrefix(testSendFile, "/") { return "", errors.New("userfile destination must start with a /") } @@ -99,7 +99,7 @@ func runCopyFile( if err != nil { return err } - stmt, err := txn.Prepare(CopyInFileStmt(fileUploadURI, CrdbInternalName, copyInternalTable)) + stmt, err := txn.Prepare(sql.CopyInFileStmt(fileUploadURI, sql.CrdbInternalName, copyInternalTable)) if err != nil { return err } @@ -144,9 +144,9 @@ func checkUserFileContent( filename string, expectedContent []byte, ) { - uri, err := prepareFileUploadURI(user, filename, UserFileUploadTable) + uri, err := prepareFileUploadURI(user, filename, sql.UserFileUploadTable) require.NoError(t, err) - store, err := s.ExecutorConfig().(ExecutorConfig).DistSQLSrv.ExternalStorageFromURI(ctx, uri, + store, err := s.ExecutorConfig().(sql.ExecutorConfig).DistSQLSrv.ExternalStorageFromURI(ctx, uri, user) require.NoError(t, err) reader, err := store.ReadFile(ctx, "") @@ -161,7 +161,7 @@ func TestFileUpload(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - params, _ := tests.CreateTestServerParams() + params, _ := CreateTestServerParams() localExternalDir, cleanup := testutils.TempDir(t) defer cleanup() params.ExternalIODir = localExternalDir @@ -190,7 +190,7 @@ func TestUploadEmptyFile(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - params, _ := tests.CreateTestServerParams() + params, _ := CreateTestServerParams() localExternalDir, cleanup := testutils.TempDir(t) defer cleanup() params.ExternalIODir = localExternalDir @@ -217,7 +217,7 @@ func TestFileNotExist(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - params, _ := tests.CreateTestServerParams() + params, _ := CreateTestServerParams() localExternalDir, cleanup := testutils.TempDir(t) defer cleanup() params.ExternalIODir = localExternalDir @@ -235,7 +235,7 @@ func TestFileExist(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - params, _ := tests.CreateTestServerParams() + params, _ := CreateTestServerParams() localExternalDir, cleanup := testutils.TempDir(t) defer cleanup() params.ExternalIODir = localExternalDir @@ -266,7 +266,7 @@ func TestNodelocalNotAdmin(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - params, _ := tests.CreateTestServerParams() + params, _ := CreateTestServerParams() localExternalDir, cleanup := testutils.TempDir(t) defer cleanup() params.ExternalIODir = localExternalDir @@ -295,7 +295,7 @@ func TestNodelocalNotAdmin(t *testing.T) { fileContent := []byte("hello \n blah 1@#% some data hello \n @#%^&&*") writeFile(t, testSendFile, fileContent) - err = runCopyFile(t, userDB, smithUserName, testSendFile, NodelocalFileUploadTable) + err = runCopyFile(t, userDB, smithUserName, testSendFile, sql.NodelocalFileUploadTable) expectedErr := "only users with the admin role are allowed to upload" require.True(t, testutils.IsError(err, expectedErr)) } @@ -306,7 +306,7 @@ func TestUserfileNotAdmin(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - params, _ := tests.CreateTestServerParams() + params, _ := CreateTestServerParams() localExternalDir, cleanup := testutils.TempDir(t) defer cleanup() params.ExternalIODir = localExternalDir @@ -337,7 +337,7 @@ func TestUserfileNotAdmin(t *testing.T) { fileContent := []byte("hello \n blah 1@#% some data hello \n @#%^&&*") writeFile(t, testSendFile, fileContent) - err = runCopyFile(t, userDB, smithUserName, testSendFile, UserFileUploadTable) + err = runCopyFile(t, userDB, smithUserName, testSendFile, sql.UserFileUploadTable) require.NoError(t, err) checkUserFileContent(context.Background(), t, s, smithUserName, testSendFile, fileContent) } From 5f083703b9a0c28a9626a805984e6c7b16506339 Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Mon, 8 Aug 2022 17:52:33 -0400 Subject: [PATCH 2/7] sql: enable GRANT/REVOKE syntax support for UDF This commits extends existing privilege statement syntax to support user-defined functions. Release note: None --- .../sql/bnf/alter_default_privileges_stmt.bnf | 8 +-- docs/generated/sql/bnf/grant_stmt.bnf | 6 ++ docs/generated/sql/bnf/revoke_stmt.bnf | 6 ++ docs/generated/sql/bnf/stmt_block.bnf | 59 ++++++++++--------- pkg/sql/parser/sql.y | 42 ++++++++++++- pkg/sql/parser/testdata/drop_function | 8 +-- pkg/sql/parser/testdata/grant_revoke | 48 +++++++++++++++ pkg/sql/sem/tree/grant.go | 9 +++ pkg/sql/sem/tree/udf.go | 2 +- 9 files changed, 151 insertions(+), 37 deletions(-) diff --git a/docs/generated/sql/bnf/alter_default_privileges_stmt.bnf b/docs/generated/sql/bnf/alter_default_privileges_stmt.bnf index 69860ec5dec..bb59d244e43 100644 --- a/docs/generated/sql/bnf/alter_default_privileges_stmt.bnf +++ b/docs/generated/sql/bnf/alter_default_privileges_stmt.bnf @@ -1,5 +1,5 @@ alter_default_privileges_stmt ::= - 'ALTER' 'DEFAULT' 'PRIVILEGES' ( 'FOR' ( 'ROLE' | 'USER' ) role_spec_list | ) ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'GRANT' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' ) 'TO' role_spec_list ( 'WITH' 'GRANT' 'OPTION' | ) ) - | 'ALTER' 'DEFAULT' 'PRIVILEGES' ( 'FOR' ( 'ROLE' | 'USER' ) role_spec_list | ) ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'REVOKE' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) ) - | 'ALTER' 'DEFAULT' 'PRIVILEGES' 'FOR' 'ALL' 'ROLES' ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'GRANT' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' ) 'TO' role_spec_list ( 'WITH' 'GRANT' 'OPTION' | ) ) - | 'ALTER' 'DEFAULT' 'PRIVILEGES' 'FOR' 'ALL' 'ROLES' ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'REVOKE' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) ) + 'ALTER' 'DEFAULT' 'PRIVILEGES' ( 'FOR' ( 'ROLE' | 'USER' ) role_spec_list | ) ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'GRANT' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'TO' role_spec_list ( 'WITH' 'GRANT' 'OPTION' | ) ) + | 'ALTER' 'DEFAULT' 'PRIVILEGES' ( 'FOR' ( 'ROLE' | 'USER' ) role_spec_list | ) ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'REVOKE' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) ) + | 'ALTER' 'DEFAULT' 'PRIVILEGES' 'FOR' 'ALL' 'ROLES' ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'GRANT' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'TO' role_spec_list ( 'WITH' 'GRANT' 'OPTION' | ) ) + | 'ALTER' 'DEFAULT' 'PRIVILEGES' 'FOR' 'ALL' 'ROLES' ( 'IN' 'SCHEMA' ( ( qualifiable_schema_name ) ( ( ',' qualifiable_schema_name ) )* ) | ) ( 'REVOKE' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' ( 'TABLES' | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' | 'FUNCTIONS' ) 'FROM' role_spec_list ( 'CASCADE' | 'RESTRICT' | ) ) diff --git a/docs/generated/sql/bnf/grant_stmt.bnf b/docs/generated/sql/bnf/grant_stmt.bnf index 069004d1595..453d07ac9bf 100644 --- a/docs/generated/sql/bnf/grant_stmt.bnf +++ b/docs/generated/sql/bnf/grant_stmt.bnf @@ -31,6 +31,12 @@ grant_stmt ::= | 'GRANT' 'ALL' 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list | 'GRANT' privilege_list 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION' | 'GRANT' privilege_list 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list + | 'GRANT' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION' + | 'GRANT' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list + | 'GRANT' 'ALL' 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION' + | 'GRANT' 'ALL' 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list + | 'GRANT' privilege_list 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION' + | 'GRANT' privilege_list 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list | 'GRANT' 'SYSTEM' 'ALL' 'PRIVILEGES' 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION' | 'GRANT' 'SYSTEM' 'ALL' 'PRIVILEGES' 'TO' role_spec_list | 'GRANT' 'SYSTEM' 'ALL' 'TO' role_spec_list 'WITH' 'GRANT' 'OPTION' diff --git a/docs/generated/sql/bnf/revoke_stmt.bnf b/docs/generated/sql/bnf/revoke_stmt.bnf index 16225afc192..82bdc8a9e28 100644 --- a/docs/generated/sql/bnf/revoke_stmt.bnf +++ b/docs/generated/sql/bnf/revoke_stmt.bnf @@ -28,6 +28,12 @@ revoke_stmt ::= | 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list | 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'ALL' 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privilege_list 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list + | 'REVOKE' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list + | 'REVOKE' 'ALL' 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list + | 'REVOKE' privilege_list 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list + | 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'ALL' 'PRIVILEGES' 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list + | 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'ALL' 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list + | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privilege_list 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list | 'REVOKE' 'SYSTEM' 'ALL' 'PRIVILEGES' 'FROM' role_spec_list | 'REVOKE' 'SYSTEM' 'ALL' 'FROM' role_spec_list | 'REVOKE' 'SYSTEM' privilege_list 'FROM' role_spec_list diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 1a3606cad42..4761f1af596 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -92,6 +92,7 @@ grant_stmt ::= | 'GRANT' privileges 'ON' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option | 'GRANT' privileges 'ON' 'ALL' 'SEQUENCES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option | 'GRANT' privileges 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option + | 'GRANT' privileges 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'TO' role_spec_list opt_with_grant_option | 'GRANT' 'SYSTEM' privileges 'TO' role_spec_list opt_with_grant_option prepare_stmt ::= @@ -109,6 +110,8 @@ revoke_stmt ::= | 'REVOKE' privileges 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list | 'REVOKE' privileges 'ON' 'ALL' 'SEQUENCES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' 'ALL' 'TABLES' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list + | 'REVOKE' privileges 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list + | 'REVOKE' 'GRANT' 'OPTION' 'FOR' privileges 'ON' 'ALL' 'FUNCTIONS' 'IN' 'SCHEMA' schema_name_list 'FROM' role_spec_list | 'REVOKE' 'SYSTEM' privileges 'FROM' role_spec_list | 'REVOKE' 'GRANT' 'OPTION' 'FOR' 'SYSTEM' privileges 'FROM' role_spec_list @@ -360,6 +363,7 @@ grant_targets ::= | 'TABLE' table_pattern_list | 'DATABASE' name_list | 'EXTERNAL' 'CONNECTION' name_list + | 'FUNCTION' function_with_argtypes_list role_spec_list ::= ( role_spec ) ( ( ',' role_spec ) )* @@ -1447,6 +1451,9 @@ table_pattern ::= simple_db_object_name | complex_table_pattern +function_with_argtypes_list ::= + ( function_with_argtypes ) ( ( ',' function_with_argtypes ) )* + privilege ::= name | 'CREATE' @@ -1963,6 +1970,10 @@ unrestricted_name ::= | type_func_name_keyword | reserved_keyword +function_with_argtypes ::= + db_object_name func_args + | db_object_name + type_name ::= db_object_name @@ -2465,9 +2476,6 @@ table_index_name_list ::= table_name_list ::= ( table_name ) ( ( ',' table_name ) )* -function_with_argtypes_list ::= - ( function_with_argtypes ) ( ( ',' function_with_argtypes ) )* - non_reserved_word ::= 'identifier' | unreserved_keyword @@ -2657,6 +2665,10 @@ reserved_keyword ::= | 'WITH' | cockroachdb_extra_reserved_keyword +func_args ::= + '(' func_args_list ')' + | '(' ')' + simple_typename ::= general_type_name | '@' iconst32 @@ -2737,6 +2749,7 @@ target_object_type ::= | 'SEQUENCES' | 'TYPES' | 'SCHEMAS' + | 'FUNCTIONS' alter_changefeed_cmd ::= 'ADD' changefeed_targets opt_with_options @@ -2747,10 +2760,6 @@ alter_changefeed_cmd ::= alter_backup_cmd ::= 'ADD' backup_kms -function_with_argtypes ::= - db_object_name func_args - | db_object_name - alter_func_opt_list ::= ( common_func_opt_item ) ( ( common_func_opt_item ) )* @@ -3116,6 +3125,9 @@ type_func_name_no_crdb_extra_keyword ::= | 'RIGHT' | 'SIMILAR' +func_args_list ::= + ( func_arg ) ( ( ',' func_arg ) )* + general_type_name ::= type_function_name_no_crdb_extra @@ -3195,10 +3207,6 @@ sequence_option_elem ::= backup_kms ::= 'NEW_KMS' '=' string_or_placeholder_opt_list 'WITH' 'OLD_KMS' '=' string_or_placeholder_opt_list -func_args ::= - '(' func_args_list ')' - | '(' ')' - common_func_opt_item ::= 'CALLED' 'ON' 'NULL' 'INPUT' | 'RETURNS' 'NULL' 'ON' 'NULL' 'INPUT' @@ -3474,6 +3482,13 @@ join_qual ::= rowsfrom_list ::= ( rowsfrom_item ) ( ( ',' rowsfrom_item ) )* +func_arg ::= + func_arg_class param_name func_arg_type + | param_name func_arg_class func_arg_type + | param_name func_arg_type + | func_arg_class func_arg_type + | func_arg_type + opt_varying ::= 'VARYING' | @@ -3539,9 +3554,6 @@ storage_parameter_key_list ::= partition_by_index ::= partition_by -func_args_list ::= - ( func_arg ) ( ( ',' func_arg ) )* - opt_float ::= '(' 'ICONST' ')' | @@ -3677,13 +3689,6 @@ opt_family_name ::= create_as_constraint_elem ::= 'PRIMARY' 'KEY' '(' create_as_params ')' opt_with_storage_parameter_list -func_arg ::= - func_arg_class param_name func_arg_type - | param_name func_arg_class func_arg_type - | param_name func_arg_type - | func_arg_class func_arg_type - | func_arg_type - func_as ::= 'SCONST' @@ -3700,6 +3705,12 @@ join_outer ::= rowsfrom_item ::= func_expr_windowless +func_arg_class ::= + 'IN' + +param_name ::= + type_function_name + char_aliases ::= 'CHAR' | 'CHARACTER' @@ -3768,12 +3779,6 @@ create_as_col_qualification_elem ::= create_as_params ::= ( create_as_param ) ( ( ',' create_as_param ) )* -func_arg_class ::= - 'IN' - -param_name ::= - type_function_name - col_qualification ::= 'CONSTRAINT' constraint_name col_qualification_elem | col_qualification_elem diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index f0676658fb2..b47371f950a 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -5423,6 +5423,18 @@ grant_stmt: WithGrantOption: $11.bool(), } } +| GRANT privileges ON ALL FUNCTIONS IN SCHEMA schema_name_list TO role_spec_list opt_with_grant_option + { + $$.val = &tree.Grant{ + Privileges: $2.privilegeList(), + Targets: tree.GrantTargetList{ + Schemas: $8.objectNamePrefixList(), + AllFunctionsInSchema: true, + }, + Grantees: $10.roleSpecList(), + WithGrantOption: $11.bool(), + } + } | GRANT SYSTEM privileges TO role_spec_list opt_with_grant_option { $$.val = &tree.Grant{ @@ -5538,6 +5550,30 @@ revoke_stmt: GrantOptionFor: true, } } +| REVOKE privileges ON ALL FUNCTIONS IN SCHEMA schema_name_list FROM role_spec_list + { + $$.val = &tree.Revoke{ + Privileges: $2.privilegeList(), + Targets: tree.GrantTargetList{ + Schemas: $8.objectNamePrefixList(), + AllFunctionsInSchema: true, + }, + Grantees: $10.roleSpecList(), + GrantOptionFor: false, + } + } +| REVOKE GRANT OPTION FOR privileges ON ALL FUNCTIONS IN SCHEMA schema_name_list FROM role_spec_list + { + $$.val = &tree.Revoke{ + Privileges: $5.privilegeList(), + Targets: tree.GrantTargetList{ + Schemas: $11.objectNamePrefixList(), + AllFunctionsInSchema: true, + }, + Grantees: $13.roleSpecList(), + GrantOptionFor: true, + } + } | REVOKE SYSTEM privileges FROM role_spec_list { $$.val = &tree.Revoke{ @@ -7620,6 +7656,10 @@ grant_targets: { $$.val = tree.GrantTargetList{ExternalConnections: $3.nameList()} } +| FUNCTION function_with_argtypes_list + { + $$.val = tree.GrantTargetList{Functions: $2.functionObjs()} + } // backup_targets is similar to grant_targets but used by backup and restore, and thus // supports tenants, but does not support sequences, types, or other SQL nouns @@ -9989,7 +10029,7 @@ target_object_type: { $$.val = privilege.Schemas } -| FUNCTIONS error +| FUNCTIONS { $$.val = privilege.Functions } diff --git a/pkg/sql/parser/testdata/drop_function b/pkg/sql/parser/testdata/drop_function index 1053b79ad70..1c08b95fa64 100644 --- a/pkg/sql/parser/testdata/drop_function +++ b/pkg/sql/parser/testdata/drop_function @@ -26,10 +26,10 @@ DROP FUNCTION _ CASCADE -- identifiers removed parse DROP FUNCTION f, g ---- -DROP FUNCTION f ,g -- normalized! -DROP FUNCTION f ,g -- fully parenthesized -DROP FUNCTION f ,g -- literals removed -DROP FUNCTION _ ,_ -- identifiers removed +DROP FUNCTION f, g +DROP FUNCTION f, g -- fully parenthesized +DROP FUNCTION f, g -- literals removed +DROP FUNCTION _, _ -- identifiers removed parse DROP FUNCTION f(int) diff --git a/pkg/sql/parser/testdata/grant_revoke b/pkg/sql/parser/testdata/grant_revoke index d21f87f493b..21b71f5f5bc 100644 --- a/pkg/sql/parser/testdata/grant_revoke +++ b/pkg/sql/parser/testdata/grant_revoke @@ -280,6 +280,30 @@ GRANT SELECT ON ALL SEQUENCES IN SCHEMA s1, s2 TO root, bar -- fully parenthesiz GRANT SELECT ON ALL SEQUENCES IN SCHEMA s1, s2 TO root, bar -- literals removed GRANT SELECT ON ALL SEQUENCES IN SCHEMA _, _ TO _, _ -- identifiers removed +parse +GRANT EXECUTE ON FUNCTION f1 TO root, bar +---- +GRANT EXECUTE ON FUNCTION f1 TO root, bar +GRANT EXECUTE ON FUNCTION f1 TO root, bar -- fully parenthesized +GRANT EXECUTE ON FUNCTION f1 TO root, bar -- literals removed +GRANT EXECUTE ON FUNCTION _ TO _, _ -- identifiers removed + +parse +GRANT EXECUTE ON FUNCTION f1, f2(INT) TO root, bar +---- +GRANT EXECUTE ON FUNCTION f1, f2(IN INT8) TO root, bar -- normalized! +GRANT EXECUTE ON FUNCTION f1, f2(IN INT8) TO root, bar -- fully parenthesized +GRANT EXECUTE ON FUNCTION f1, f2(IN INT8) TO root, bar -- literals removed +GRANT EXECUTE ON FUNCTION _, _(IN INT8) TO _, _ -- identifiers removed + +parse +GRANT EXECUTE ON ALL FUNCTIONS IN SCHEMA s1, s2 TO root, bar +---- +GRANT EXECUTE ON ALL FUNCTIONS IN SCHEMA s1, s2 TO root, bar +GRANT EXECUTE ON ALL FUNCTIONS IN SCHEMA s1, s2 TO root, bar -- fully parenthesized +GRANT EXECUTE ON ALL FUNCTIONS IN SCHEMA s1, s2 TO root, bar -- literals removed +GRANT EXECUTE ON ALL FUNCTIONS IN SCHEMA _, _ TO _, _ -- identifiers removed + parse REVOKE SELECT ON SEQUENCE s1 FROM root, bar ---- @@ -529,6 +553,30 @@ REVOKE SELECT, UPDATE ON ALL TABLES IN SCHEMA s1, s2 FROM root, bar -- fully par REVOKE SELECT, UPDATE ON ALL TABLES IN SCHEMA s1, s2 FROM root, bar -- literals removed REVOKE SELECT, UPDATE ON ALL TABLES IN SCHEMA _, _ FROM _, _ -- identifiers removed +parse +REVOKE EXECUTE ON FUNCTION f1 FROM root, bar +---- +REVOKE EXECUTE ON FUNCTION f1 FROM root, bar +REVOKE EXECUTE ON FUNCTION f1 FROM root, bar -- fully parenthesized +REVOKE EXECUTE ON FUNCTION f1 FROM root, bar -- literals removed +REVOKE EXECUTE ON FUNCTION _ FROM _, _ -- identifiers removed + +parse +REVOKE EXECUTE ON FUNCTION f1, f2(INT) FROM root, bar +---- +REVOKE EXECUTE ON FUNCTION f1, f2(IN INT8) FROM root, bar -- normalized! +REVOKE EXECUTE ON FUNCTION f1, f2(IN INT8) FROM root, bar -- fully parenthesized +REVOKE EXECUTE ON FUNCTION f1, f2(IN INT8) FROM root, bar -- literals removed +REVOKE EXECUTE ON FUNCTION _, _(IN INT8) FROM _, _ -- identifiers removed + +parse +REVOKE EXECUTE ON ALL FUNCTIONS IN SCHEMA s1, s2 FROM root, bar +---- +REVOKE EXECUTE ON ALL FUNCTIONS IN SCHEMA s1, s2 FROM root, bar +REVOKE EXECUTE ON ALL FUNCTIONS IN SCHEMA s1, s2 FROM root, bar -- fully parenthesized +REVOKE EXECUTE ON ALL FUNCTIONS IN SCHEMA s1, s2 FROM root, bar -- literals removed +REVOKE EXECUTE ON ALL FUNCTIONS IN SCHEMA _, _ FROM _, _ -- identifiers removed + # Ensure that the support for ON ROLE doesn't leak # where it should not be recognized. error diff --git a/pkg/sql/sem/tree/grant.go b/pkg/sql/sem/tree/grant.go index f481f59ed79..06f0003707d 100644 --- a/pkg/sql/sem/tree/grant.go +++ b/pkg/sql/sem/tree/grant.go @@ -36,10 +36,13 @@ type GrantTargetList struct { Schemas ObjectNamePrefixList Tables TableAttrs Types []*UnresolvedObjectName + Functions FuncObjs // If the target is for all sequences in a set of schemas. AllSequencesInSchema bool // If the target is for all tables in a set of schemas. AllTablesInSchema bool + // If the target is for all functions in a set of schemas. + AllFunctionsInSchema bool // If the target is system. System bool // If the target is External Connection. @@ -63,6 +66,9 @@ func (tl *GrantTargetList) Format(ctx *FmtCtx) { } else if tl.AllTablesInSchema { ctx.WriteString("ALL TABLES IN SCHEMA ") ctx.FormatNode(&tl.Schemas) + } else if tl.AllFunctionsInSchema { + ctx.WriteString("ALL FUNCTIONS IN SCHEMA ") + ctx.FormatNode(&tl.Schemas) } else if tl.Schemas != nil { ctx.WriteString("SCHEMA ") ctx.FormatNode(&tl.Schemas) @@ -77,6 +83,9 @@ func (tl *GrantTargetList) Format(ctx *FmtCtx) { } else if tl.ExternalConnections != nil { ctx.WriteString("EXTERNAL CONNECTION ") ctx.FormatNode(&tl.ExternalConnections) + } else if tl.Functions != nil { + ctx.WriteString("FUNCTION ") + ctx.FormatNode(tl.Functions) } else { if tl.Tables.SequenceOnly { ctx.WriteString("SEQUENCE ") diff --git a/pkg/sql/sem/tree/udf.go b/pkg/sql/sem/tree/udf.go index 2aa2f0cf94a..50e1d94ce71 100644 --- a/pkg/sql/sem/tree/udf.go +++ b/pkg/sql/sem/tree/udf.go @@ -337,7 +337,7 @@ type FuncObjs []FuncObj func (node FuncObjs) Format(ctx *FmtCtx) { for i, f := range node { if i > 0 { - ctx.WriteString(" ,") + ctx.WriteString(", ") } ctx.FormatNode(f) } From 310825c89d81ddbc9f943929b888cbcdcc1f8970 Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Mon, 8 Aug 2022 23:39:40 -0400 Subject: [PATCH 3/7] sql: grant/revole support for udf This commit implements support for GRANT...ON FUNCTION statements and REVOKE...ON FUNCTION statements for udfs. Release note: None --- docs/generated/eventlog.md | 25 +++++++ pkg/sql/descriptor.go | 1 + pkg/sql/grant_revoke.go | 28 +++++++- pkg/sql/resolver.go | 65 +++++++++++++++++++ pkg/sql/sqltelemetry/iam.go | 5 ++ .../eventpb/eventlog_channels_generated.go | 3 + pkg/util/log/eventpb/json_encode_generated.go | 24 +++++++ pkg/util/log/eventpb/privilege_events.proto | 8 +++ 8 files changed, 156 insertions(+), 3 deletions(-) diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index d6165bbcae7..fa0d632d422 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -1757,6 +1757,31 @@ added to / removed from a user for a database object. | `DatabaseName` | The name of the affected database. | yes | +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `Statement` | A normalized copy of the SQL statement that triggered the event. The statement string contains a mix of sensitive and non-sensitive details (it is redactable). | partially | +| `Tag` | The statement tag. This is separate from the statement string, since the statement string can contain sensitive information. The tag is guaranteed not to. | no | +| `User` | The user account that triggered the event. The special usernames `root` and `node` are not considered sensitive. | depends | +| `DescriptorID` | The primary object descriptor affected by the operation. Set to zero for operations that don't affect descriptors. | no | +| `ApplicationName` | The application name for the session where the event was emitted. This is included in the event to ease filtering of logging output by application. Application names starting with a dollar sign (`$`) are not considered sensitive. | no | +| `PlaceholderValues` | The mapping of SQL placeholders to their values, for prepared statements. | yes | +| `Grantee` | The user/role affected by the grant or revoke operation. | yes | +| `GrantedPrivileges` | The privileges being granted to the grantee. | no | +| `RevokedPrivileges` | The privileges being revoked from the grantee. | no | + +### `change_function_privilege` + + + +| Field | Description | Sensitive | +|--|--|--| +| `FuncName` | The name of the affected function. | yes | + + #### Common fields | Field | Description | Sensitive | diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index 3505312e338..fc9f4828677 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -58,6 +58,7 @@ var ( errNoSchema = pgerror.Newf(pgcode.InvalidName, "no schema specified") errNoTable = pgerror.New(pgcode.InvalidName, "no table specified") errNoType = pgerror.New(pgcode.InvalidName, "no type specified") + errNoFunction = pgerror.New(pgcode.InvalidName, "no function specified") errNoMatch = pgerror.New(pgcode.UndefinedObject, "no object matched") ) diff --git a/pkg/sql/grant_revoke.go b/pkg/sql/grant_revoke.go index f78939fc9bc..e1ff353d2b4 100644 --- a/pkg/sql/grant_revoke.go +++ b/pkg/sql/grant_revoke.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" @@ -227,10 +228,10 @@ func (n *changeDescriptorBackedPrivilegesNode) startExec(params runParams) error // First, update the descriptors. We want to catch all errors before // we update them in KV below. b := p.txn.NewBatch() - for _, descriptorWithTypes := range descriptorsWithTypes { + for _, descriptorWithType := range descriptorsWithTypes { // Disallow privilege changes on system objects. For more context, see #43842. - descriptor := descriptorWithTypes.descriptor - objType := descriptorWithTypes.objectType + descriptor := descriptorWithType.descriptor + objType := descriptorWithType.objectType if catalog.IsSystemDescriptor(descriptor) { @@ -396,6 +397,21 @@ func (n *changeDescriptorBackedPrivilegesNode) startExec(params runParams) error SchemaName: d.Name, // FIXME }}) } + case *funcdesc.Mutable: + if err := p.writeFuncSchemaChange(ctx, d); err != nil { + return err + } + for _, grantee := range n.grantees { + privs := eventDetails // copy the granted/revoked privilege list. + privs.Grantee = grantee.Normalized() + events = append(events, eventLogEntry{ + targetID: int32(d.ID), + event: &eventpb.ChangeFunctionPrivilege{ + CommonSQLPrivilegeEventDetails: privs, + FuncName: d.Name, // FIXME + }}) + } + // TODO(chengxiong): add eventlog for function privilege changes. } } @@ -440,12 +456,18 @@ func (p *planner) getGrantOnObject( case targets.AllTablesInSchema: incIAMFunc(sqltelemetry.OnAllTablesInSchema) return privilege.Table, nil + case targets.AllFunctionsInSchema: + incIAMFunc(sqltelemetry.OnAllFunctionsInSchema) + return privilege.Function, nil case targets.Schemas != nil: incIAMFunc(sqltelemetry.OnSchema) return privilege.Schema, nil case targets.Types != nil: incIAMFunc(sqltelemetry.OnType) return privilege.Type, nil + case targets.Functions != nil: + incIAMFunc(sqltelemetry.OnFunction) + return privilege.Function, nil case targets.System: incIAMFunc(sqltelemetry.OnSystem) return privilege.Global, nil diff --git a/pkg/sql/resolver.go b/pkg/sql/resolver.go index 6c932c0346c..932d77b1ad3 100644 --- a/pkg/sql/resolver.go +++ b/pkg/sql/resolver.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descbuilder" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" @@ -447,6 +448,37 @@ func (p *planner) getDescriptorsFromTargetListForPrivilegeChange( return descs, nil } + if targets.Functions != nil { + if len(targets.Functions) == 0 { + return nil, errNoFunction + } + descs := make([]DescriptorWithObjectType, 0, len(targets.Functions)) + fnResolved := catalog.DescriptorIDSet{} + for _, f := range targets.Functions { + overload, err := p.matchUDF(ctx, &f, true /* required */) + if err != nil { + return nil, err + } + fnID, err := funcdesc.UserDefinedFunctionOIDToID(overload.Oid) + if err != nil { + return nil, err + } + if fnResolved.Contains(fnID) { + continue + } + fnResolved.Add(fnID) + fnDesc, err := p.Descriptors().GetMutableFunctionByID(ctx, p.txn, fnID, tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return nil, err + } + descs = append(descs, DescriptorWithObjectType{ + descriptor: fnDesc, + objectType: privilege.Function, + }) + } + return descs, nil + } + if targets.Schemas != nil { if len(targets.Schemas) == 0 { return nil, errNoSchema @@ -505,6 +537,39 @@ func (p *planner) getDescriptorsFromTargetListForPrivilegeChange( } } + return descs, nil + } else if targets.AllFunctionsInSchema { + var descs []DescriptorWithObjectType + for _, scName := range targets.Schemas { + dbName := p.CurrentDatabase() + if scName.ExplicitCatalog { + dbName = scName.Catalog() + } + db, err := p.Descriptors().GetMutableDatabaseByName(ctx, p.txn, dbName, flags) + if err != nil { + return nil, err + } + sc, err := p.Descriptors().GetMutableSchemaByName( + ctx, p.txn, db, scName.Schema(), tree.SchemaLookupFlags{Required: true}, + ) + if err != nil { + return nil, err + } + err = sc.ForEachFunctionOverload(func(overload descpb.SchemaDescriptor_FunctionOverload) error { + fn, err := p.Descriptors().GetMutableFunctionByID(ctx, p.txn, overload.ID, tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return err + } + descs = append(descs, DescriptorWithObjectType{ + descriptor: fn, + objectType: privilege.Function, + }) + return nil + }) + if err != nil { + return nil, err + } + } return descs, nil } diff --git a/pkg/sql/sqltelemetry/iam.go b/pkg/sql/sqltelemetry/iam.go index 7a0eb56b5c9..e7aa37d2f08 100644 --- a/pkg/sql/sqltelemetry/iam.go +++ b/pkg/sql/sqltelemetry/iam.go @@ -36,12 +36,17 @@ const ( OnSequence = "on_sequence" // OnType is used when a GRANT/REVOKE is happening on a type. OnType = "on_type" + // OnFunction is used when a GRANT/REVOKE is happening on a function. + OnFunction = "on_function" // OnAllTablesInSchema is used when a GRANT/REVOKE is happening on // all tables in a set of schemas. OnAllTablesInSchema = "on_all_tables_in_schemas" // OnAllSequencesInSchema is used when a GRANT/REVOKE is happening on // all sequences in a set of schemas. OnAllSequencesInSchema = "on_all_sequences_in_schemas" + // OnAllFunctionsInSchema is used when a GRANT/REVOKE is happening on + // all functions in a set of schemas. + OnAllFunctionsInSchema = "on_all_functions_in_schemas" // OnSystem is used when a GRANT/REVOKE is happening on system. OnSystem = "on_system" // OnExternalConnection is used when a GRANT/REVOKE is happening on an diff --git a/pkg/util/log/eventpb/eventlog_channels_generated.go b/pkg/util/log/eventpb/eventlog_channels_generated.go index b9d9f95b67c..98e67187900 100644 --- a/pkg/util/log/eventpb/eventlog_channels_generated.go +++ b/pkg/util/log/eventpb/eventlog_channels_generated.go @@ -210,6 +210,9 @@ func (m *AlterTypeOwner) LoggingChannel() logpb.Channel { return logpb.Channel_P // LoggingChannel implements the EventPayload interface. func (m *ChangeDatabasePrivilege) LoggingChannel() logpb.Channel { return logpb.Channel_PRIVILEGES } +// LoggingChannel implements the EventPayload interface. +func (m *ChangeFunctionPrivilege) LoggingChannel() logpb.Channel { return logpb.Channel_PRIVILEGES } + // LoggingChannel implements the EventPayload interface. func (m *ChangeSchemaPrivilege) LoggingChannel() logpb.Channel { return logpb.Channel_PRIVILEGES } diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index 1526ad0a24f..152b2a24c16 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -747,6 +747,30 @@ func (m *ChangeDatabasePrivilege) AppendJSONFields(printComma bool, b redact.Red return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *ChangeFunctionPrivilege) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonSQLEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonSQLPrivilegeEventDetails.AppendJSONFields(printComma, b) + + if m.FuncName != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"FuncName\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.FuncName))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *ChangeSchemaPrivilege) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { diff --git a/pkg/util/log/eventpb/privilege_events.proto b/pkg/util/log/eventpb/privilege_events.proto index e3683d14d0d..30bacfe7c50 100644 --- a/pkg/util/log/eventpb/privilege_events.proto +++ b/pkg/util/log/eventpb/privilege_events.proto @@ -83,6 +83,14 @@ message ChangeTypePrivilege { string type_name = 4 [(gogoproto.jsontag) = ",omitempty"]; } +message ChangeFunctionPrivilege { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonSQLEventDetails sql = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonSQLPrivilegeEventDetails privs = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + // The name of the affected function. + string func_name = 4 [(gogoproto.jsontag) = ",omitempty"]; +} + // AlterDatabaseOwner is recorded when a database's owner is changed. message AlterDatabaseOwner { From 83ca23415f93def2c00216f662dcbce09579c8ab Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Tue, 9 Aug 2022 11:54:35 -0400 Subject: [PATCH 4/7] sql: fetch immutable with AvoidLease when hydrating mutables With UDF introduced, it's getting more often than before that we need to hydrate a table implicit type. Before this commit we always try to get a mutable type when hydrating a mutable descriptor. However, getting a mutable table implicit type is not allowed (which is good). But this defense breaks hydration of mutable function which use a table implict type. In this commit, we change it to always fetch immutable types but with `AvoidedLease` flag. Database and Schema are also changed to fetch a fresh immutable. Release note: None --- pkg/sql/catalog/descs/hydrate.go | 34 +++++++++++++++++++---- pkg/sql/catalog/descs/type.go | 6 +++- pkg/sql/logictest/testdata/logic_test/udf | 8 ++++++ 3 files changed, 41 insertions(+), 7 deletions(-) diff --git a/pkg/sql/catalog/descs/hydrate.go b/pkg/sql/catalog/descs/hydrate.go index 3fdd2a7398b..d42a9754bec 100644 --- a/pkg/sql/catalog/descs/hydrate.go +++ b/pkg/sql/catalog/descs/hydrate.go @@ -264,11 +264,33 @@ func getMutableTypeLookupFunc( tc *Collection, txn *kv.Txn, schema catalog.SchemaDescriptor, ) typedesc.TypeLookupFunc { return func(ctx context.Context, id descpb.ID) (tree.TypeName, catalog.TypeDescriptor, error) { - desc, err := tc.GetMutableTypeVersionByID(ctx, txn, id) + // Note that getting mutable table implicit type is not allowed. To + // hydrate table implicit types, we don't really need a mutable type + // descriptor since we are not going to mutate the table because we simply + // need the tuple type and some metadata. So it's adequate here to get a + // fresh immutable. + flags := tree.ObjectLookupFlags{ + CommonLookupFlags: tree.CommonLookupFlags{ + Required: true, + IncludeDropped: true, + IncludeOffline: true, + AvoidLeased: true, + }, + } + typDesc, err := tc.GetImmutableTypeByID(ctx, txn, id, flags) if err != nil { return tree.TypeName{}, nil, err } - dbDesc, err := tc.GetMutableDescriptorByID(ctx, txn, desc.ParentID) + + _, dbDesc, err := tc.GetImmutableDatabaseByID( + ctx, txn, typDesc.GetParentID(), + tree.DatabaseLookupFlags{ + Required: true, + IncludeDropped: true, + IncludeOffline: true, + AvoidLeased: true, + }, + ) if err != nil { return tree.TypeName{}, nil, err } @@ -278,12 +300,12 @@ func getMutableTypeLookupFunc( scName = schema.GetName() } else { sc, err := tc.getSchemaByID( - ctx, txn, desc.ParentSchemaID, + ctx, txn, typDesc.GetParentSchemaID(), tree.SchemaLookupFlags{ Required: true, IncludeDropped: true, IncludeOffline: true, - RequireMutable: true, + AvoidLeased: true, }, ) if err != nil { @@ -291,8 +313,8 @@ func getMutableTypeLookupFunc( } scName = sc.GetName() } - name := tree.MakeQualifiedTypeName(dbDesc.GetName(), scName, desc.Name) - return name, desc, nil + name := tree.MakeQualifiedTypeName(dbDesc.GetName(), scName, typDesc.GetName()) + return name, typDesc, nil } } diff --git a/pkg/sql/catalog/descs/type.go b/pkg/sql/catalog/descs/type.go index d35ca2155b9..99da5b6a0d3 100644 --- a/pkg/sql/catalog/descs/type.go +++ b/pkg/sql/catalog/descs/type.go @@ -23,6 +23,10 @@ import ( "github.com/cockroachdb/errors" ) +// ErrMutableTableImplicitType indicates that a table implicit type was fetched +// as a mutable, which is not allowed. +var ErrMutableTableImplicitType = pgerror.Newf(pgcode.DependentObjectsStillExist, "table implicit type not mutable") + // GetMutableTypeByName returns a mutable type descriptor with properties // according to the provided lookup flags. RequireMutable is ignored. func (tc *Collection) GetMutableTypeByName( @@ -91,7 +95,7 @@ func (tc *Collection) GetMutableTypeByID( case *typedesc.Mutable: return t, nil case *typedesc.TableImplicitRecordType: - return nil, pgerror.Newf(pgcode.DependentObjectsStillExist, "cannot modify table record type %q", desc.GetName()) + return nil, errors.Wrapf(ErrMutableTableImplicitType, "cannot modify table record type %q", desc.GetName()) } return nil, errors.AssertionFailedf("unhandled type descriptor type %T during GetMutableTypeByID", desc) diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index cdb6fafb7a9..ffbb0d92cba 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -705,6 +705,14 @@ SET search_path = public statement ok DROP SCHEMA sc1; +subtest drop_table_using_implicit_type + +statement ok +CREATE FUNCTION test_implicit_f() RETURNS t_implicit_type LANGUAGE SQL AS $$ SELECT * FROM t_implicit_type $$ + +statement ok +DROP FUNCTION test_implicit_f; + subtest disallow_udf_in_table statement ok From 8e1013a8f4b664192bdff1718d627e8b7332ddd4 Mon Sep 17 00:00:00 2001 From: Chengxiong Ruan Date: Tue, 9 Aug 2022 13:37:43 -0400 Subject: [PATCH 5/7] sql: populate information_schema.role_routine_grants with udf Previously the role_routine_grants table was unimplemented. This commit populates the table with user-defined functions privileges and leaves a TODO for all builtin functions. Release note: None --- pkg/sql/information_schema.go | 75 +++++++- .../logictest/testdata/logic_test/pg_catalog | 2 +- pkg/sql/logictest/testdata/logic_test/udf | 179 +++++++++++++++++- 3 files changed, 251 insertions(+), 5 deletions(-) diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 2da65245aea..4a3a26ab1b5 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/types" @@ -1589,12 +1590,80 @@ var informationSchemaRoutinePrivilegesTable = virtualSchemaTable{ } var informationSchemaRoleRoutineGrantsTable = virtualSchemaTable{ - comment: "role_routine_grants was created for compatibility and is currently unimplemented", + // TODO(chengxiong): add builtin function privileges as well. + comment: "privileges granted on functions (incomplete; only contains privileges of user-defined functions)", schema: vtable.InformationSchemaRoleRoutineGrants, - populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { + populate: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { + var dbDescs []catalog.DatabaseDescriptor + if db == nil { + var err error + dbDescs, err = p.Descriptors().GetAllDatabaseDescriptors(ctx, p.Txn()) + if err != nil { + return err + } + } else { + dbDescs = append(dbDescs, db) + } + for _, db := range dbDescs { + err := db.ForEachSchema(func(id descpb.ID, name string) error { + sc, err := p.Descriptors().GetImmutableSchemaByID(ctx, p.txn, id, tree.SchemaLookupFlags{Required: true}) + if err != nil { + return err + } + return sc.ForEachFunctionOverload(func(overload descpb.SchemaDescriptor_FunctionOverload) error { + fn, err := p.Descriptors().GetMutableFunctionByID(ctx, p.txn, overload.ID, tree.ObjectLookupFlagsWithRequired()) + if err != nil { + return err + } + privs := fn.GetPrivileges() + dbNameStr := tree.NewDString(db.GetName()) + scNameStr := tree.NewDString(sc.GetName()) + fnSpecificName := tree.NewDString(fmt.Sprintf("%s_%d", fn.GetName(), catid.FuncIDToOID(fn.GetID()))) + fnName := tree.NewDString(fn.GetName()) + // EXECUTE is the only privilege kind relevant to functions. + exPriv := tree.NewDString(privilege.EXECUTE.String()) + if err := addRow( + tree.DNull, // grantor + tree.NewDString(privs.Owner().Normalized()), // grantee + dbNameStr, // specific_catalog + scNameStr, // specific_schema + fnSpecificName, // specific_name + dbNameStr, // routine_catalog + scNameStr, // routine_schema + fnName, // routine_name + exPriv, // privilege_type + yesString, // is_grantable + ); err != nil { + return err + } + for _, user := range privs.Users { + if !privilege.EXECUTE.IsSetIn(user.Privileges) { + continue + } + if err := addRow( + tree.DNull, // grantor + tree.NewDString(user.User().Normalized()), // grantee + dbNameStr, // specific_catalog + scNameStr, // specific_schema + fnSpecificName, // specific_name + dbNameStr, // routine_catalog + scNameStr, // routine_schema + fnName, // routine_name + exPriv, // privilege_type + yesOrNoDatum(privilege.EXECUTE.IsSetIn(user.WithGrantOption)), // is_grantable + ); err != nil { + return err + } + } + return nil + }) + }) + if err != nil { + return err + } + } return nil }, - unimplemented: true, } var informationSchemaElementTypesTable = virtualSchemaTable{ diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 56b92f683e2..4ad85c18c32 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -3906,7 +3906,7 @@ objoid classoid objsubid description 4294967179 4294967123 0 foreign key constraints 4294967178 4294967123 0 resource_groups was created for compatibility and is currently unimplemented 4294967177 4294967123 0 role_column_grants was created for compatibility and is currently unimplemented -4294967176 4294967123 0 role_routine_grants was created for compatibility and is currently unimplemented +4294967176 4294967123 0 privileges granted on functions (incomplete; only contains privileges of user-defined functions) 4294967175 4294967123 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) 4294967174 4294967123 0 role_udt_grants was created for compatibility and is currently unimplemented 4294967173 4294967123 0 role_usage_grants was created for compatibility and is currently unimplemented diff --git a/pkg/sql/logictest/testdata/logic_test/udf b/pkg/sql/logictest/testdata/logic_test/udf index ffbb0d92cba..36f1887cee1 100644 --- a/pkg/sql/logictest/testdata/logic_test/udf +++ b/pkg/sql/logictest/testdata/logic_test/udf @@ -787,7 +787,6 @@ CREATE FUNCTION public.test_vf_f() SELECT lower('hello'); $$ - subtest execution statement ok @@ -993,3 +992,181 @@ l1 l2 i1 i2 s1 s2 v1 v2 1 1 1 1 1 1 11 11 2 2 2 2 2 2 12 12 3 3 3 3 3 3 13 13 + +subtest grant_revoke + +statement ok +CREATE SCHEMA test_priv_sc1; +CREATE FUNCTION test_priv_f1() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; +CREATE FUNCTION test_priv_f2(int) RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; +CREATE FUNCTION test_priv_sc1.test_priv_f3() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; +CREATE USER udf_test_user; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100163 test public test_priv_f1 EXECUTE YES +NULL root test public test_priv_f2_100164 test public test_priv_f2 EXECUTE YES +NULL root test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE YES + +statement ok +GRANT EXECUTE ON FUNCTION test_priv_f1(), test_priv_f2(int), test_priv_sc1.test_priv_f3 TO udf_test_user WITH GRANT OPTION; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100163 test public test_priv_f1 EXECUTE YES +NULL root test public test_priv_f2_100164 test public test_priv_f2 EXECUTE YES +NULL root test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE YES +NULL udf_test_user test public test_priv_f1_100163 test public test_priv_f1 EXECUTE YES +NULL udf_test_user test public test_priv_f2_100164 test public test_priv_f2 EXECUTE YES +NULL udf_test_user test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE YES + +statement ok +REVOKE GRANT OPTION FOR EXECUTE ON FUNCTION test_priv_f1(), test_priv_f2(int), test_priv_sc1.test_priv_f3 FROM udf_test_user; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100163 test public test_priv_f1 EXECUTE YES +NULL root test public test_priv_f2_100164 test public test_priv_f2 EXECUTE YES +NULL root test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE YES +NULL udf_test_user test public test_priv_f1_100163 test public test_priv_f1 EXECUTE NO +NULL udf_test_user test public test_priv_f2_100164 test public test_priv_f2 EXECUTE NO +NULL udf_test_user test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE NO + +statement ok +REVOKE EXECUTE ON FUNCTION test_priv_f1(), test_priv_f2(int), test_priv_sc1.test_priv_f3 FROM udf_test_user; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100163 test public test_priv_f1 EXECUTE YES +NULL root test public test_priv_f2_100164 test public test_priv_f2 EXECUTE YES +NULL root test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE YES + +statement ok +GRANT EXECUTE ON ALL FUNCTIONS IN SCHEMA public, test_priv_sc1 TO udf_test_user WITH GRANT OPTION; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100163 test public test_priv_f1 EXECUTE YES +NULL root test public test_priv_f2_100164 test public test_priv_f2 EXECUTE YES +NULL root test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE YES +NULL udf_test_user test public test_priv_f1_100163 test public test_priv_f1 EXECUTE YES +NULL udf_test_user test public test_priv_f2_100164 test public test_priv_f2 EXECUTE YES +NULL udf_test_user test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE YES + +statement ok +REVOKE GRANT OPTION FOR EXECUTE ON ALL FUNCTIONS in schema public, test_priv_sc1 FROM udf_test_user; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100163 test public test_priv_f1 EXECUTE YES +NULL root test public test_priv_f2_100164 test public test_priv_f2 EXECUTE YES +NULL root test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE YES +NULL udf_test_user test public test_priv_f1_100163 test public test_priv_f1 EXECUTE NO +NULL udf_test_user test public test_priv_f2_100164 test public test_priv_f2 EXECUTE NO +NULL udf_test_user test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE NO + +statement ok +REVOKE EXECUTE ON ALL FUNCTIONS IN SCHEMA public, test_priv_sc1 FROM udf_test_user; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100163 test public test_priv_f1 EXECUTE YES +NULL root test public test_priv_f2_100164 test public test_priv_f2 EXECUTE YES +NULL root test test_priv_sc1 test_priv_f3_100165 test test_priv_sc1 test_priv_f3 EXECUTE YES + +statement ok +DROP FUNCTION test_priv_f1; +DROP FUNCTION test_priv_f2; +DROP FUNCTION test_priv_sc1.test_priv_f3; +DROP USER udf_test_user; + +subtest default_privileges + +statement ok +CREATE USER udf_test_user; +CREATE FUNCTION test_priv_f1() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100166 test public test_priv_f1 EXECUTE YES + +# Add default privilege and make sure new function +statement ok +ALTER DEFAULT PRIVILEGES IN SCHEMA public, test_priv_sc1 GRANT EXECUTE ON FUNCTIONS TO udf_test_user WITH GRANT OPTION; + +statement ok +CREATE FUNCTION test_priv_f2(int) RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; +CREATE FUNCTION test_priv_sc1.test_priv_f3() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100166 test public test_priv_f1 EXECUTE YES +NULL root test public test_priv_f2_100167 test public test_priv_f2 EXECUTE YES +NULL root test test_priv_sc1 test_priv_f3_100168 test test_priv_sc1 test_priv_f3 EXECUTE YES +NULL udf_test_user test public test_priv_f2_100167 test public test_priv_f2 EXECUTE YES +NULL udf_test_user test test_priv_sc1 test_priv_f3_100168 test test_priv_sc1 test_priv_f3 EXECUTE YES + +statement ok +DROP FUNCTION test_priv_f2; +DROP FUNCTION test_priv_sc1.test_priv_f3; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100166 test public test_priv_f1 EXECUTE YES + +statement ok +ALTER DEFAULT PRIVILEGES IN SCHEMA public, test_priv_sc1 REVOKE EXECUTE ON FUNCTIONS FROM udf_test_user; + +statement ok +CREATE FUNCTION test_priv_f2(int) RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; +CREATE FUNCTION test_priv_sc1.test_priv_f3() RETURNS INT LANGUAGE SQL AS $$ SELECT 1 $$; + +query TTTTTTTTTT colnames +SELECT * FROM information_schema.role_routine_grants +WHERE routine_name IN ('test_priv_f1', 'test_priv_f2', 'test_priv_f3') +ORDER BY grantee, routine_name; +---- +grantor grantee specific_catalog specific_schema specific_name routine_catalog routine_schema routine_name privilege_type is_grantable +NULL root test public test_priv_f1_100166 test public test_priv_f1 EXECUTE YES +NULL root test public test_priv_f2_100169 test public test_priv_f2 EXECUTE YES +NULL root test test_priv_sc1 test_priv_f3_100170 test test_priv_sc1 test_priv_f3 EXECUTE YES From 7c8d747816e6c675ce4eeb1afea208b6b90c4114 Mon Sep 17 00:00:00 2001 From: Steven Danna Date: Fri, 12 Aug 2022 13:46:22 +0100 Subject: [PATCH 6/7] kvserver: use static tracing span name for AddSSTable Including the span bounds in the name of the tracing span means that we can't aggregate information across all AddSSTable calls. Release note: None --- pkg/kv/kvserver/batcheval/cmd_add_sstable.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index 3fd05266cea..5ab1e4df231 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -125,7 +125,7 @@ func EvalAddSSTable( var span *tracing.Span var err error - ctx, span = tracing.ChildSpan(ctx, fmt.Sprintf("AddSSTable [%s,%s)", start.Key, end.Key)) + ctx, span = tracing.ChildSpan(ctx, "AddSSTable") defer span.Finish() log.Eventf(ctx, "evaluating AddSSTable [%s,%s)", start.Key, end.Key) From ff7462411a99a87f11e42f34a7c5044900538ff3 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Wed, 10 Aug 2022 16:09:04 +0200 Subject: [PATCH 7/7] rangefeed: correctly handle intent outside of time-bound Fixes https://github.com/cockroachdb/cockroach/issues/85886. Release note (bug fix): Changefeed jobs undergoing catch-up scans could fail with an error "expected provisional value for intent with ts X, found Y". The problem would either spontaneously resolve or be rectified after a high-priority scan of the affected index. This bug is now fixed. --- pkg/kv/kvserver/rangefeed/catchup_scan.go | 23 +++++++--- .../kvserver/rangefeed/catchup_scan_test.go | 46 +++++++++++++++++++ 2 files changed, 62 insertions(+), 7 deletions(-) diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan.go b/pkg/kv/kvserver/rangefeed/catchup_scan.go index 40db0f58796..3c0ec0c91e2 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan.go @@ -194,13 +194,17 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err return errors.AssertionFailedf("unexpected inline key %s", unsafeKey) } - // This is an MVCCMetadata key for an intent. The catchUp scan only cares - // about committed values, so ignore this and skip past the corresponding - // provisional key-value. To do this, iterate to the provisional - // key-value, validate its timestamp, then iterate again. When using - // MVCCIncrementalIterator we know that the provisional value will also be - // within the time bounds so we use Next. - i.Next() + // This is an MVCCMetadata key for an intent. The catchUp scan + // only cares about committed values, so ignore this and skip past + // the corresponding provisional key-value. To do this, iterate to + // the provisional key-value, validate its timestamp, then iterate + // again. If we arrived here with a preceding call to NextIgnoringTime + // (in the with-diff case), it's possible that the intent is not within + // the time bounds. Using `NextIgnoringTime` on the next line makes sure + // that we are guaranteed to validate the version that belongs to the + // intent. + i.NextIgnoringTime() + if ok, err := i.Valid(); err != nil { return errors.Wrap(err, "iterating to provisional value for intent") } else if !ok { @@ -210,6 +214,11 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err return errors.Errorf("expected provisional value for intent with ts %s, found %s", meta.Timestamp, i.UnsafeKey().Timestamp) } + // Now move to the next key of interest. Note that if in the last + // iteration of the loop we called `NextIgnoringTime`, the fact that we + // hit an intent proves that there wasn't a previous value, so we can + // (in fact, have to, to avoid surfacing unwanted keys) unconditionally + // enforce time bounds. i.Next() continue } diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go index bf0b7ff6689..2ef24d5362e 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go @@ -149,3 +149,49 @@ func TestCatchupScanInlineError(t *testing.T) { require.Error(t, err) require.Contains(t, err.Error(), "unexpected inline value") } + +func TestCatchupScanSeesOldIntent(t *testing.T) { + defer leaktest.AfterTest(t)() + // Regression test for [#85886]. When with-diff is specified, the iterator may + // be positioned on an intent that is outside the time bounds. When we read + // the intent and want to load the version, we must make sure to ignore time + // bounds, or we'll see a wholly unrelated version. + // + // [#85886]: https://github.com/cockroachdb/cockroach/issues/85886 + + ctx := context.Background() + eng := storage.NewDefaultInMemForTesting() + defer eng.Close() + + // b -> version @ 1100 (visible) + // d -> intent @ 990 (iterator will be positioned here because of with-diff option) + // e -> version @ 1100 + tsCutoff := hlc.Timestamp{WallTime: 1000} // the lower bound of the catch-up scan + tsIntent := tsCutoff.Add(-10, 0) // the intent is below the lower bound + tsVersionInWindow := tsCutoff.Add(10, 0) // an unrelated version is above the lower bound + + require.NoError(t, storage.MVCCPut(ctx, eng, nil, roachpb.Key("b"), + tsVersionInWindow, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("foo"), nil)) + + txn := roachpb.MakeTransaction("foo", roachpb.Key("d"), roachpb.NormalUserPriority, tsIntent, 100, 0) + require.NoError(t, storage.MVCCPut(ctx, eng, nil, roachpb.Key("d"), + tsIntent, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("intent"), &txn)) + + require.NoError(t, storage.MVCCPut(ctx, eng, nil, roachpb.Key("e"), + tsVersionInWindow, hlc.ClockTimestamp{}, roachpb.MakeValueFromString("bar"), nil)) + + // Run a catchup scan across the span and watch it succeed. + span := roachpb.Span{Key: keys.LocalMax, EndKey: keys.MaxKey} + iter := NewCatchUpIterator(eng, span, tsCutoff, nil) + defer iter.Close() + + keys := map[string]struct{}{} + require.NoError(t, iter.CatchUpScan(func(e *roachpb.RangeFeedEvent) error { + keys[string(e.Val.Key)] = struct{}{} + return nil + }, true /* withDiff */)) + require.Equal(t, map[string]struct{}{ + "b": {}, + "e": {}, + }, keys) +}