diff --git a/pkg/ccl/serverccl/BUILD.bazel b/pkg/ccl/serverccl/BUILD.bazel index 687dc94989b6..f9c45ab00880 100644 --- a/pkg/ccl/serverccl/BUILD.bazel +++ b/pkg/ccl/serverccl/BUILD.bazel @@ -51,6 +51,7 @@ go_test( "//pkg/sql", "//pkg/sql/catalog/catconstants", "//pkg/sql/catalog/descpb", + "//pkg/sql/idxusage", "//pkg/sql/pgwire/pgcode", "//pkg/sql/sqlstats", "//pkg/sql/tests", diff --git a/pkg/ccl/serverccl/tenant_status_test.go b/pkg/ccl/serverccl/tenant_status_test.go index ab0a23b3bb27..96c3acfd6011 100644 --- a/pkg/ccl/serverccl/tenant_status_test.go +++ b/pkg/ccl/serverccl/tenant_status_test.go @@ -17,6 +17,7 @@ import ( "strconv" "strings" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/base" _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl" @@ -25,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -208,7 +210,7 @@ func TestResetSQLStatsRPCForTenant(t *testing.T) { "SELECT 1, 1, 1", } - testHelper := newTestTenantHelper(t, 3 /* tenantClusterSize */) + testHelper := newTestTenantHelper(t, 3 /* tenantClusterSize */, tests.CreateTestingKnobs()) defer testHelper.cleanup(ctx, t) testCluster := testHelper.testCluster() @@ -318,7 +320,8 @@ func TestContentionEventsForTenant(t *testing.T) { ctx := context.Background() - testHelper := newTestTenantHelper(t, 3 /* tenantClusterSize */) + testHelper := + newTestTenantHelper(t, 3 /* tenantClusterSize */, tests.CreateTestingKnobs()) defer testHelper.cleanup(ctx, t) testingCluster := testHelper.testCluster() @@ -383,3 +386,86 @@ SET TRACING=off; } } } + +func TestIndexUsageForTenants(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + skip.UnderStressRace(t, "expensive tests") + + ctx := context.Background() + + statsIngestionCb, statsIngestionNotifier := idxusage.CreateIndexStatsIngestedCallbackForTest() + + knobs := tests.CreateTestingKnobs() + knobs.IndexUsageStatsKnobs = &idxusage.TestingKnobs{ + OnIndexUsageStatsProcessedCallback: statsIngestionCb, + } + testHelper := newTestTenantHelper(t, 3 /* tenantClusterSize */, knobs) + defer testHelper.cleanup(ctx, t) + + testingCluster := testHelper.testCluster() + controlledCluster := testHelper.controlCluster() + + testingCluster.tenantConn(0).Exec(t, ` +CREATE TABLE test ( + k INT PRIMARY KEY, + a INT, + b INT, + INDEX(a) +) +`) + + testingCluster.tenantConn(0).Exec(t, ` +INSERT INTO test +VALUES (1, 10, 100), (2, 20, 200), (3, 30, 300) +`) + + // Record scan on primary index. + testingCluster.tenantConn(0).Exec(t, "SELECT * FROM test") + + // Record scan on secondary index. + testingCluster.tenantConn(1).Exec(t, "SELECT * FROM test@test_a_idx") + testTableIDStr := testingCluster.tenantConn(2).QueryStr(t, "SELECT 'test'::regclass::oid")[0][0] + testTableID, err := strconv.Atoi(testTableIDStr) + require.NoError(t, err) + + // Wait for the stats to be ingested. + require.NoError(t, + idxusage.WaitForIndexStatsIngestionForTest(statsIngestionNotifier, map[roachpb.IndexUsageKey]struct{}{ + { + TableID: roachpb.TableID(testTableID), + IndexID: 1, + }: {}, + { + TableID: roachpb.TableID(testTableID), + IndexID: 2, + }: {}, + }, 2 /* expectedEventCnt*/, 5*time.Second /* timeout */), + ) + + query := ` +SELECT + table_id, + index_id, + total_reads, + extract_duration('second', now() - last_read) < 5 +FROM + crdb_internal.index_usage_statistics +WHERE + table_id = $1 +` + actual := testingCluster.tenantConn(2).QueryStr(t, query, testTableID) + expected := [][]string{ + {testTableIDStr, "1", "1", "true"}, + {testTableIDStr, "2", "1", "true"}, + } + + require.Equal(t, expected, actual) + + // Ensure tenant data isolation. + actual = controlledCluster.tenantConn(2).QueryStr(t, query, testTableID) + expected = [][]string{} + + require.Equal(t, expected, actual) +} diff --git a/pkg/ccl/serverccl/tenant_test_utils.go b/pkg/ccl/serverccl/tenant_test_utils.go index 9d0f3e6e82ea..02267bca8334 100644 --- a/pkg/ccl/serverccl/tenant_test_utils.go +++ b/pkg/ccl/serverccl/tenant_test_utils.go @@ -35,12 +35,17 @@ type testTenant struct { } func newTestTenant( - t *testing.T, server serverutils.TestServerInterface, existing bool, tenantID roachpb.TenantID, + t *testing.T, + server serverutils.TestServerInterface, + existing bool, + tenantID roachpb.TenantID, + knobs base.TestingKnobs, ) *testTenant { t.Helper() tenantParams := tests.CreateTestTenantParams(tenantID) tenantParams.Existing = existing + tenantParams.TestingKnobs = knobs log.TestingClearServerIdentifiers() tenant, tenantConn := serverutils.StartTenant(t, server, tenantParams) @@ -71,7 +76,9 @@ type tenantTestHelper struct { tenantControlCluster tenantCluster } -func newTestTenantHelper(t *testing.T, tenantClusterSize int) *tenantTestHelper { +func newTestTenantHelper( + t *testing.T, tenantClusterSize int, knobs base.TestingKnobs, +) *tenantTestHelper { t.Helper() params, _ := tests.CreateTestServerParams() @@ -87,12 +94,14 @@ func newTestTenantHelper(t *testing.T, tenantClusterSize int) *tenantTestHelper server, tenantClusterSize, security.EmbeddedTenantIDs()[0], + knobs, ), tenantControlCluster: newTenantCluster( t, server, tenantClusterSize, security.EmbeddedTenantIDs()[1], + knobs, ), } } @@ -115,14 +124,19 @@ func (h *tenantTestHelper) cleanup(ctx context.Context, t *testing.T) { type tenantCluster []*testTenant func newTenantCluster( - t *testing.T, server serverutils.TestServerInterface, tenantClusterSize int, tenantID uint64, + t *testing.T, + server serverutils.TestServerInterface, + tenantClusterSize int, + tenantID uint64, + knobs base.TestingKnobs, ) tenantCluster { t.Helper() cluster := make([]*testTenant, tenantClusterSize) existing := false for i := 0; i < tenantClusterSize; i++ { - cluster[i] = newTestTenant(t, server, existing, roachpb.MakeTenantID(tenantID)) + cluster[i] = + newTestTenant(t, server, existing, roachpb.MakeTenantID(tenantID), knobs) existing = true } diff --git a/pkg/cmd/reduce/main.go b/pkg/cmd/reduce/main.go index 124e90920fb4..8307676169e4 100644 --- a/pkg/cmd/reduce/main.go +++ b/pkg/cmd/reduce/main.go @@ -52,7 +52,7 @@ var ( flags = flag.NewFlagSet(os.Args[0], flag.ExitOnError) binary = flags.String("binary", "./cockroach", "path to cockroach binary") file = flags.String("file", "", "the path to a file containing a SQL query to reduce") - verbose = flags.Bool("v", false, "print progress to standard output") + verbose = flags.Bool("v", false, "print progress to standard output and the original test case output if it is not interesting") contains = flags.String("contains", "", "error regex to search for") unknown = flags.Bool("unknown", false, "print unknown types during walk") workers = flags.Int("goroutines", goroutines, "number of worker goroutines (defaults to NumCPU/3") @@ -141,7 +141,7 @@ func reduceSQL( chunkReducer = reducesql.NewSQLChunkReducer(chunkReductions) } - interesting := func(ctx context.Context, sql string) bool { + isInteresting := func(ctx context.Context, sql string) (interesting bool, logOriginalHint func()) { // Disable telemetry and license generation. cmd := exec.CommandContext(ctx, binary, "demo", @@ -162,13 +162,18 @@ func reduceSQL( case errors.HasType(err, (*os.PathError)(nil)): log.Fatal(err) } - return containsRE.Match(out) + if verbose { + logOriginalHint = func() { + fmt.Fprintf(logger, "output did not match regex %s:\n\n%s", contains, string(out)) + } + } + return containsRE.Match(out), logOriginalHint } out, err := reduce.Reduce( logger, inputSQL, - interesting, + isInteresting, workers, reduce.ModeInteresting, chunkReducer, diff --git a/pkg/cmd/roachtest/tests/sqlsmith.go b/pkg/cmd/roachtest/tests/sqlsmith.go index 5571c1bb15d2..782faef3bae7 100644 --- a/pkg/cmd/roachtest/tests/sqlsmith.go +++ b/pkg/cmd/roachtest/tests/sqlsmith.go @@ -211,10 +211,11 @@ func registerSQLSmith(r registry.Registry) { es := err.Error() if strings.Contains(es, "internal error") { // TODO(yuzefovich): we temporarily ignore internal errors - // that are because of #40929. + // that are because of #40929 and #70831. var expectedError bool for _, exp := range []string{ "could not parse \"0E-2019\" as type decimal", + "no volatility for cast tuple", } { expectedError = expectedError || strings.Contains(es, exp) } diff --git a/pkg/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go index d0aa6f62c548..021d4f70c3f9 100644 --- a/pkg/rpc/auth_tenant.go +++ b/pkg/rpc/auth_tenant.go @@ -83,6 +83,9 @@ func (a tenantAuthorizer) authorize( case "/cockroach.server.serverpb.Status/ListLocalSessions": return a.authTenant(tenID) + case "/cockroach.server.serverpb.Status/IndexUsageStatistics": + return a.authTenant(tenID) + case "/cockroach.roachpb.Internal/GetSpanConfigs": return a.authGetSpanConfigs(tenID, req.(*roachpb.GetSpanConfigsRequest)) diff --git a/pkg/server/index_usage_stats_test.go b/pkg/server/index_usage_stats_test.go index 438efa6fa95e..694e242cf13c 100644 --- a/pkg/server/index_usage_stats_test.go +++ b/pkg/server/index_usage_stats_test.go @@ -54,51 +54,11 @@ func compareStatsHelper( require.Equal(t, expected, actual) } -func createIndexStatsIngestedCallback() ( - func(key roachpb.IndexUsageKey), - chan roachpb.IndexUsageKey, -) { - // Create a buffered channel so the callback is non-blocking. - notify := make(chan roachpb.IndexUsageKey, 100) - - cb := func(key roachpb.IndexUsageKey) { - notify <- key - } - - return cb, notify -} - -func waitForStatsIngestion( - t *testing.T, - notify chan roachpb.IndexUsageKey, - expectedKeys map[roachpb.IndexUsageKey]struct{}, - expectedEventCnt int, - timeout time.Duration, -) { - var timer timeutil.Timer - eventCnt := 0 - - timer.Reset(timeout) - - for eventCnt < expectedEventCnt { - select { - case key := <-notify: - if _, ok := expectedKeys[key]; ok { - eventCnt++ - } - continue - case <-timer.C: - timer.Read = true - t.Fatalf("expected stats ingestion to complete within %s, but it timed out", timeout) - } - } -} - func TestStatusAPIIndexUsage(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - statsIngestionCb, statsIngestionNotifier := createIndexStatsIngestedCallback() + statsIngestionCb, statsIngestionNotifier := idxusage.CreateIndexStatsIngestedCallbackForTest() testCluster := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{ ServerArgs: base.TestServerArgs{ @@ -214,10 +174,12 @@ func TestStatusAPIIndexUsage(t *testing.T) { thirdLocalStatsReader := thirdServer.SQLServer().(*sql.Server).GetLocalIndexStatistics() // Wait for the stats to be ingested. - waitForStatsIngestion(t, statsIngestionNotifier, map[roachpb.IndexUsageKey]struct{}{ - indexKeyA: {}, - indexKeyB: {}, - }, /* expectedKeys */ 4 /* expectedEventCnt*/, 5*time.Second /* timeout */) + require.NoError(t, + idxusage.WaitForIndexStatsIngestionForTest(statsIngestionNotifier, map[roachpb.IndexUsageKey]struct{}{ + indexKeyA: {}, + indexKeyB: {}, + }, /* expectedKeys */ 4 /* expectedEventCnt*/, 5*time.Second /* timeout */), + ) // First node should have nothing. stats := firstLocalStatsReader.Get(indexKeyA.TableID, indexKeyA.IndexID) diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index 5ae05930aa24..46a566d2c7ad 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -247,6 +247,7 @@ func StartTenant( goroutineDumpDirName: args.GoroutineDumpDirName, heapProfileDirName: args.HeapProfileDirName, runtime: args.runtime, + sessionRegistry: args.sessionRegistry, }); err != nil { return nil, "", "", err } diff --git a/pkg/server/tenant_status.go b/pkg/server/tenant_status.go index 1ce404da9d14..3ba0adef7e9d 100644 --- a/pkg/server/tenant_status.go +++ b/pkg/server/tenant_status.go @@ -582,12 +582,68 @@ func (t *tenantStatusServer) Profile( } func (t *tenantStatusServer) IndexUsageStatistics( - ctx context.Context, request *serverpb.IndexUsageStatisticsRequest, + ctx context.Context, req *serverpb.IndexUsageStatisticsRequest, ) (*serverpb.IndexUsageStatisticsResponse, error) { + ctx = propagateGatewayMetadata(ctx) + ctx = t.AnnotateCtx(ctx) + if _, err := t.privilegeChecker.requireViewActivityPermission(ctx); err != nil { return nil, err } - idxUsageStats := t.sqlServer.pgServer.SQLServer.GetLocalIndexStatistics() - return indexUsageStatsLocal(idxUsageStats) + localReq := &serverpb.IndexUsageStatisticsRequest{ + NodeID: "local", + } + + if len(req.NodeID) > 0 { + parsedInstanceID, local, err := t.parseInstanceID(req.NodeID) + if err != nil { + return nil, status.Error(codes.InvalidArgument, err.Error()) + } + if local { + statsReader := t.sqlServer.pgServer.SQLServer.GetLocalIndexStatistics() + return indexUsageStatsLocal(statsReader) + } + + instance, err := t.sqlServer.sqlInstanceProvider.GetInstance(ctx, parsedInstanceID) + if err != nil { + return nil, err + } + statusClient, err := t.dialPod(ctx, parsedInstanceID, instance.InstanceAddr) + if err != nil { + return nil, err + } + + // We issue a localReq instead of the incoming req to other nodes. This is + // to instruct other nodes to only return us their node-local stats and + // do not further propagates the RPC call. + return statusClient.IndexUsageStatistics(ctx, localReq) + } + + fetchIndexUsageStats := func(ctx context.Context, client interface{}, _ base.SQLInstanceID) (interface{}, error) { + statusClient := client.(serverpb.StatusClient) + return statusClient.IndexUsageStatistics(ctx, localReq) + } + + resp := &serverpb.IndexUsageStatisticsResponse{} + aggFn := func(_ base.SQLInstanceID, nodeResp interface{}) { + stats := nodeResp.(*serverpb.IndexUsageStatisticsResponse) + resp.Statistics = append(resp.Statistics, stats.Statistics...) + } + + var combinedError error + errFn := func(_ base.SQLInstanceID, nodeFnError error) { + combinedError = errors.CombineErrors(combinedError, nodeFnError) + } + + if err := t.iteratePods(ctx, fmt.Sprintf("requesting index usage stats for instance %s", req.NodeID), + t.dialCallback, + fetchIndexUsageStats, + aggFn, + errFn, + ); err != nil { + return nil, err + } + + return resp, nil } diff --git a/pkg/sql/delegate/BUILD.bazel b/pkg/sql/delegate/BUILD.bazel index b5fae2c87a00..de248e693472 100644 --- a/pkg/sql/delegate/BUILD.bazel +++ b/pkg/sql/delegate/BUILD.bazel @@ -40,6 +40,7 @@ go_library( "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/keys", + "//pkg/security", "//pkg/settings", "//pkg/sql/catalog/catconstants", "//pkg/sql/catalog/colinfo", diff --git a/pkg/sql/delegate/show_grants.go b/pkg/sql/delegate/show_grants.go index d61e20813f72..7cb5e3fe1909 100644 --- a/pkg/sql/delegate/show_grants.go +++ b/pkg/sql/delegate/show_grants.go @@ -15,8 +15,11 @@ import ( "fmt" "strings" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" ) @@ -233,5 +236,21 @@ FROM "".information_schema.type_privileges` query := fmt.Sprintf(` SELECT * FROM (%s) %s ORDER BY %s `, source.String(), cond.String(), orderBy) + + // Terminate on invalid users. + for _, p := range n.Grantees { + user, err := security.MakeSQLUsernameFromUserInput(string(p), security.UsernameValidation) + if err != nil { + return nil, err + } + userExists, err := d.catalog.RoleExists(d.ctx, user) + if err != nil { + return nil, err + } + if !userExists { + return nil, pgerror.Newf(pgcode.UndefinedObject, "role/user %q does not exist", user) + } + } + return parse(query) } diff --git a/pkg/sql/idxusage/test_utils.go b/pkg/sql/idxusage/test_utils.go index 1da46fc34826..007d5e8a7542 100644 --- a/pkg/sql/idxusage/test_utils.go +++ b/pkg/sql/idxusage/test_utils.go @@ -11,8 +11,12 @@ package idxusage import ( + "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" ) // TestingKnobs is the testing knobs that provides callbacks that unit tests @@ -27,3 +31,51 @@ var _ base.ModuleTestingKnobs = &TestingKnobs{} // ModuleTestingKnobs implements the base.ModuleTestingKnobs interface. func (t *TestingKnobs) ModuleTestingKnobs() {} + +// CreateIndexStatsIngestedCallbackForTest creates a pair of callbacks and +// notification channel for unit tests. The callback is injected into +// IndexUsageStats struct through testing knobs and the channel can be used by +// WaitForStatsIngestionForTest. +func CreateIndexStatsIngestedCallbackForTest() ( + func(key roachpb.IndexUsageKey), + chan roachpb.IndexUsageKey, +) { + // Create a buffered channel so the callback is non-blocking. + notify := make(chan roachpb.IndexUsageKey, 100) + + cb := func(key roachpb.IndexUsageKey) { + notify <- key + } + + return cb, notify +} + +// WaitForIndexStatsIngestionForTest waits for a map of expected events +// to occur for expectedEventCnt number of times from the event channel. +// If expected events did not occur, an error is returned. +func WaitForIndexStatsIngestionForTest( + notify chan roachpb.IndexUsageKey, + expectedKeys map[roachpb.IndexUsageKey]struct{}, + expectedEventCnt int, + timeout time.Duration, +) error { + var timer timeutil.Timer + eventCnt := 0 + + timer.Reset(timeout) + + for eventCnt < expectedEventCnt { + select { + case key := <-notify: + if _, ok := expectedKeys[key]; ok { + eventCnt++ + } + continue + case <-timer.C: + timer.Read = true + return errors.New("failed to wait for index usage stats ingestion") + } + } + + return nil +} diff --git a/pkg/sql/logictest/testdata/logic_test/grant_database b/pkg/sql/logictest/testdata/logic_test/grant_database index 49b56948720f..78192c8d7421 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_database +++ b/pkg/sql/logictest/testdata/logic_test/grant_database @@ -184,3 +184,8 @@ database_name schema_name table_name grantee privilege_type b public t admin ALL b public t root ALL b public t testuser ALL + +# Calling SHOW GRANTS on an invalid user should error out. + +statement error role/user "invaliduser" does not exist +SHOW GRANTS FOR invaliduser diff --git a/pkg/sql/opt/cat/BUILD.bazel b/pkg/sql/opt/cat/BUILD.bazel index 6ec3a34bdf74..cd58913190ae 100644 --- a/pkg/sql/opt/cat/BUILD.bazel +++ b/pkg/sql/opt/cat/BUILD.bazel @@ -21,6 +21,7 @@ go_library( deps = [ "//pkg/geo/geoindex", "//pkg/roachpb:with-mocks", + "//pkg/security", "//pkg/sql/catalog/descpb", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/sql/opt/cat/catalog.go b/pkg/sql/opt/cat/catalog.go index 9b19de632352..158b6a21b0b9 100644 --- a/pkg/sql/opt/cat/catalog.go +++ b/pkg/sql/opt/cat/catalog.go @@ -15,6 +15,7 @@ package cat import ( "context" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -157,4 +158,7 @@ type Catalog interface { // - the fully qualified name of a data source object can change without the // object itself changing (e.g. when a database is renamed). FullyQualifiedName(ctx context.Context, ds DataSource) (DataSourceName, error) + + // RoleExists returns true if the role exists. + RoleExists(ctx context.Context, role security.SQLUsername) (bool, error) } diff --git a/pkg/sql/opt/testutils/testcat/BUILD.bazel b/pkg/sql/opt/testutils/testcat/BUILD.bazel index 65338982cfc4..1fbb516954e5 100644 --- a/pkg/sql/opt/testutils/testcat/BUILD.bazel +++ b/pkg/sql/opt/testutils/testcat/BUILD.bazel @@ -22,6 +22,7 @@ go_library( "//pkg/config/zonepb", "//pkg/geo/geoindex", "//pkg/roachpb:with-mocks", + "//pkg/security", "//pkg/settings/cluster", "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", diff --git a/pkg/sql/opt/testutils/testcat/test_catalog.go b/pkg/sql/opt/testutils/testcat/test_catalog.go index 552b99889e02..b092c570bf65 100644 --- a/pkg/sql/opt/testutils/testcat/test_catalog.go +++ b/pkg/sql/opt/testutils/testcat/test_catalog.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/geo/geoindex" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" @@ -226,6 +227,11 @@ func (tc *Catalog) FullyQualifiedName( return ds.(dataSource).fqName(), nil } +// RoleExists is part of the cat.Catalog interface. +func (tc *Catalog) RoleExists(ctx context.Context, role security.SQLUsername) (bool, error) { + return true, nil +} + func (tc *Catalog) resolveSchema(toResolve *cat.SchemaName) (cat.Schema, cat.SchemaName, error) { if string(toResolve.CatalogName) != testDB { return nil, cat.SchemaName{}, pgerror.Newf(pgcode.InvalidSchemaName, diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 15537e712b7f..48a49029fd49 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -370,6 +371,11 @@ func (oc *optCatalog) fullyQualifiedNameWithTxn( nil } +// RoleExists is part of the cat.Catalog interface. +func (oc *optCatalog) RoleExists(ctx context.Context, role security.SQLUsername) (bool, error) { + return RoleExists(ctx, oc.planner.ExecCfg(), oc.planner.Txn(), role) +} + // dataSourceForDesc returns a data source wrapper for the given descriptor. // The wrapper might come from the cache, or it may be created now. func (oc *optCatalog) dataSourceForDesc( diff --git a/pkg/sql/tests/server_params.go b/pkg/sql/tests/server_params.go index 7c958a767289..9f8bcf9e1175 100644 --- a/pkg/sql/tests/server_params.go +++ b/pkg/sql/tests/server_params.go @@ -27,11 +27,7 @@ func CreateTestServerParams() (base.TestServerArgs, *CommandFilters) { var cmdFilters CommandFilters cmdFilters.AppendFilter(CheckEndTxnTrigger, true) params := base.TestServerArgs{} - params.Knobs = base.TestingKnobs{ - SQLStatsKnobs: &sqlstats.TestingKnobs{ - AOSTClause: "AS OF SYSTEM TIME '-1us'", - }, - } + params.Knobs = CreateTestingKnobs() params.Knobs.Store = &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ TestingEvalFilter: cmdFilters.RunFilters, @@ -43,11 +39,16 @@ func CreateTestServerParams() (base.TestServerArgs, *CommandFilters) { // CreateTestTenantParams creates a set of params suitable for SQL Tenant Tests. func CreateTestTenantParams(tenantID roachpb.TenantID) base.TestTenantArgs { return base.TestTenantArgs{ - TenantID: tenantID, - TestingKnobs: base.TestingKnobs{ - SQLStatsKnobs: &sqlstats.TestingKnobs{ - AOSTClause: "AS OF SYSTEM TIME '-1us'", - }, + TenantID: tenantID, + TestingKnobs: CreateTestingKnobs(), + } +} + +// CreateTestingKnobs creates a testing knob in the unit tests. +func CreateTestingKnobs() base.TestingKnobs { + return base.TestingKnobs{ + SQLStatsKnobs: &sqlstats.TestingKnobs{ + AOSTClause: "AS OF SYSTEM TIME '-1us'", }, } } diff --git a/pkg/testutils/reduce/reduce.go b/pkg/testutils/reduce/reduce.go index e63844f0f7d5..d55a2f285d58 100644 --- a/pkg/testutils/reduce/reduce.go +++ b/pkg/testutils/reduce/reduce.go @@ -56,8 +56,11 @@ const ( type State interface{} // InterestingFn returns true if the string triggers the target test failure. It -// should be context-aware and stop work if the context is canceled. -type InterestingFn func(context.Context, string) bool +// should be context-aware and stop work if the context is canceled. It can +// return a function which will be called if the original test case is not +// interesting. The function should log a hint that will help the user +// understand why the original test case is not interesting. +type InterestingFn func(context.Context, string) (_ bool, logOriginalHint func()) // Mode is an enum specifying how to determine if forward progress was made. type Mode int @@ -89,7 +92,10 @@ func Reduce( } ctx, cancel := context.WithCancel(context.Background()) defer cancel() - if !isInteresting(ctx, originalTestCase) { + if interesting, logHint := isInteresting(ctx, originalTestCase); !interesting { + if logHint != nil { + logHint() + } return "", errors.New("original test case not interesting") } @@ -162,7 +168,7 @@ func Reduce( for i := 0; i < numGoroutines; i++ { g.GoCtx(func(ctx context.Context) error { for vs := range variants { - if isInteresting(ctx, vs.file) { + if interesting, _ := isInteresting(ctx, vs.file); interesting { // Wait for the previous test to finish. select { case <-ctx.Done(): @@ -327,7 +333,7 @@ func attemptChunkReduction( end := rand.Intn(chunkReducer.NumSegments()-start) + start + 1 localReduced := chunkReducer.DeleteSegments(start, end) - if isInteresting(ctx, localReduced) { + if interesting, _ := isInteresting(ctx, localReduced); interesting { reduced = localReduced log(logger, "\tchunk reduction: %d bytes\n", len(reduced)) failedAttempts = 0 diff --git a/pkg/testutils/reduce/reduce_test.go b/pkg/testutils/reduce/reduce_test.go index 3e44f43d742c..05c77f37ec10 100644 --- a/pkg/testutils/reduce/reduce_test.go +++ b/pkg/testutils/reduce/reduce_test.go @@ -52,11 +52,11 @@ var ( ) func isInterestingGo(contains string) reduce.InterestingFn { - return func(ctx context.Context, f string) bool { + return func(ctx context.Context, f string) (bool, func()) { _, err := parser.ParseExpr(f) if err == nil { - return false + return false, nil } - return strings.Contains(err.Error(), contains) + return strings.Contains(err.Error(), contains), nil } } diff --git a/pkg/testutils/reduce/reducesql/reducesql_test.go b/pkg/testutils/reduce/reducesql/reducesql_test.go index 95e201629384..85ffb994780b 100644 --- a/pkg/testutils/reduce/reducesql/reducesql_test.go +++ b/pkg/testutils/reduce/reducesql/reducesql_test.go @@ -38,7 +38,7 @@ func TestReduceSQL(t *testing.T) { } func isInterestingSQL(contains string) reduce.InterestingFn { - return func(ctx context.Context, f string) bool { + return func(ctx context.Context, f string) (bool, func()) { args := base.TestServerArgs{ Insecure: true, } @@ -67,8 +67,8 @@ func isInterestingSQL(contains string) reduce.InterestingFn { } _, err = db.Exec(ctx, f) if err == nil { - return false + return false, nil } - return strings.Contains(err.Error(), contains) + return strings.Contains(err.Error(), contains), nil } }