From badf836c1845f9dc7fe1beda79cfd74bec4a7f96 Mon Sep 17 00:00:00 2001 From: Yevgeniy Miretskiy Date: Thu, 17 Sep 2020 17:58:02 -0400 Subject: [PATCH 1/2] bulkio: Propagate errors when executing schedule. Use correct error object when checking for retryable errors. In addition, add a `FOR UPDATE` clause when picking up schedules to execute to reduce contention. Release Notes: None Release Justification: Bug fix; Incorrect handling of transaction errors resulted in scheduled jobs showing incorrect and confusing status message. --- pkg/jobs/job_scheduler.go | 5 ++-- pkg/jobs/job_scheduler_test.go | 49 +++++++++++++++++++++++++++++++ pkg/jobs/schedule_control_test.go | 13 ++++---- pkg/jobs/testutils_test.go | 37 ++++++++++------------- 4 files changed, 74 insertions(+), 30 deletions(-) diff --git a/pkg/jobs/job_scheduler.go b/pkg/jobs/job_scheduler.go index 312d06b63dd2..067fb1393b55 100644 --- a/pkg/jobs/job_scheduler.go +++ b/pkg/jobs/job_scheduler.go @@ -88,7 +88,8 @@ SELECT FROM %s S WHERE next_run < %s ORDER BY random() -%s`, env.SystemJobsTableName(), CreatedByScheduledJobs, +%s +FOR UPDATE`, env.SystemJobsTableName(), CreatedByScheduledJobs, StatusSucceeded, StatusCanceled, StatusFailed, env.ScheduledJobsTableName(), env.NowExpr(), limitClause) } @@ -297,7 +298,7 @@ func (s *jobScheduler) executeSchedules( if processErr := withSavePoint(ctx, txn, func() error { return s.processSchedule(ctx, schedule, numRunning, stats, txn) }); processErr != nil { - if errors.HasType(err, (*savePointError)(nil)) { + if errors.HasType(processErr, (*savePointError)(nil)) { return errors.Wrapf(err, "savepoint error for schedule %d", schedule.ScheduleID()) } diff --git a/pkg/jobs/job_scheduler_test.go b/pkg/jobs/job_scheduler_test.go index 9c71502917a5..0dbb28fb94c3 100644 --- a/pkg/jobs/job_scheduler_test.go +++ b/pkg/jobs/job_scheduler_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" @@ -592,3 +593,51 @@ func TestJobSchedulerDaemonUsesSystemTables(t *testing.T) { return nil }) } + +func TestTransientTxnErrors(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + h, cleanup := newTestHelper(t) + defer cleanup() + ctx := context.Background() + + h.sqlDB.Exec(t, "CREATE TABLE defaultdb.foo(a int primary key, b timestamp not null)") + + // Setup 10 schedules updating defaultdb.foo timestamp. + for i := 0; i < 10; i++ { + schedule := NewScheduledJob(h.env) + schedule.SetScheduleLabel(fmt.Sprintf("test schedule: %d", i)) + schedule.SetOwner("test") + require.NoError(t, schedule.SetSchedule("*/1 * * * *")) + any, err := types.MarshalAny(&jobspb.SqlStatementExecutionArg{ + Statement: fmt.Sprintf("UPSERT INTO defaultdb.foo (a, b) VALUES (%d, now())", i), + }) + require.NoError(t, err) + schedule.SetExecutionDetails(InlineExecutorName, jobspb.ExecutionArguments{Args: any}) + require.NoError(t, schedule.Create( + ctx, h.cfg.InternalExecutor, nil)) + } + + // Setup numConcurrent workers, each executing maxExec executeSchedule calls. + const maxExec = 100 + const numConcurrent = 3 + require.NoError(t, + ctxgroup.GroupWorkers(context.Background(), numConcurrent, func(ctx context.Context, _ int) error { + ticker := time.NewTicker(time.Millisecond) + numExecs := 0 + for range ticker.C { + h.env.AdvanceTime(time.Minute) + // Transaction retry errors should never bubble up. + require.NoError(t, + h.cfg.DB.Txn(context.Background(), func(ctx context.Context, txn *kv.Txn) error { + return h.execSchedules(ctx, allSchedules, txn) + })) + numExecs++ + if numExecs == maxExec { + return nil + } + } + return nil + })) +} diff --git a/pkg/jobs/schedule_control_test.go b/pkg/jobs/schedule_control_test.go index 04fd125f86e6..d525bc53cc61 100644 --- a/pkg/jobs/schedule_control_test.go +++ b/pkg/jobs/schedule_control_test.go @@ -15,6 +15,7 @@ import ( "fmt" "strings" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/jobs/jobstest" @@ -31,9 +32,6 @@ func TestScheduleControl(t *testing.T) { th, cleanup := newTestHelper(t) defer cleanup() - // Inject our test environment into schedule control execution via testing knobs. - th.cfg.TestingKnobs.(*TestingKnobs).JobSchedulerEnv = th.env - t.Run("non-existent", func(t *testing.T) { for _, command := range []string{ "PAUSE SCHEDULE 123", @@ -123,8 +121,7 @@ func TestScheduleControl(t *testing.T) { func TestJobsControlForSchedules(t *testing.T) { defer leaktest.AfterTest(t)() - th, cleanup := newTestHelperForTables(t, jobstest.UseSystemTables, - true /* accelerateIntervals */) + th, cleanup := newTestHelperForTables(t, jobstest.UseSystemTables) defer cleanup() registry := th.server.JobRegistry().(*Registry) @@ -227,10 +224,12 @@ func TestJobsControlForSchedules(t *testing.T) { func TestFilterJobsControlForSchedules(t *testing.T) { defer leaktest.AfterTest(t)() defer ResetConstructors()() - th, cleanup := newTestHelperForTables(t, jobstest.UseSystemTables, - false /* accelerateIntervals */) + th, cleanup := newTestHelperForTables(t, jobstest.UseSystemTables) defer cleanup() + // Prevent registry from changing job state while running this test. + defer TestingSetAdoptAndCancelIntervals(24*time.Hour, 24*time.Hour)() + registry := th.server.JobRegistry().(*Registry) blockResume := make(chan struct{}) defer close(blockResume) diff --git a/pkg/jobs/testutils_test.go b/pkg/jobs/testutils_test.go index ed41d8499f06..e973aef34615 100644 --- a/pkg/jobs/testutils_test.go +++ b/pkg/jobs/testutils_test.go @@ -14,7 +14,6 @@ import ( "context" "fmt" "testing" - "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -33,11 +32,13 @@ import ( "github.com/stretchr/testify/require" ) +type execSchedulesFn func(ctx context.Context, maxSchedules int64, txn *kv.Txn) error type testHelper struct { - env *jobstest.JobSchedulerTestEnv - server serverutils.TestServerInterface - cfg *scheduledjobs.JobExecutionConfig - sqlDB *sqlutils.SQLRunner + env *jobstest.JobSchedulerTestEnv + server serverutils.TestServerInterface + execSchedules execSchedulesFn + cfg *scheduledjobs.JobExecutionConfig + sqlDB *sqlutils.SQLRunner } // newTestHelper creates and initializes appropriate state for a test, @@ -51,20 +52,20 @@ type testHelper struct { // The testHelper will accelerate the adoption and cancellation loops inside of // the registry. func newTestHelper(t *testing.T) (*testHelper, func()) { - return newTestHelperForTables(t, jobstest.UseTestTables, - true /* accelerateIntervals */) + return newTestHelperForTables(t, jobstest.UseTestTables) } func newTestHelperForTables( - t *testing.T, envTableType jobstest.EnvTablesType, accelerateIntervals bool, + t *testing.T, envTableType jobstest.EnvTablesType, ) (*testHelper, func()) { - var cleanupIntervals func() - if accelerateIntervals { - cleanupIntervals = TestingSetAdoptAndCancelIntervals(10*time.Millisecond, 10*time.Millisecond) - } + var execSchedules execSchedulesFn + // Setup test scheduled jobs table. + env := jobstest.NewJobSchedulerTestEnv(envTableType, timeutil.Now()) knobs := &TestingKnobs{ - TakeOverJobsScheduling: func(_ func(ctx context.Context, maxSchedules int64, txn *kv.Txn) error) { + JobSchedulerEnv: env, + TakeOverJobsScheduling: func(daemon func(ctx context.Context, maxSchedules int64, txn *kv.Txn) error) { + execSchedules = daemon }, } s, db, kvDB := serverutils.StartServer(t, base.TestServerArgs{ @@ -73,9 +74,6 @@ func newTestHelperForTables( sqlDB := sqlutils.MakeSQLRunner(db) - // Setup test scheduled jobs table. - env := jobstest.NewJobSchedulerTestEnv(envTableType, timeutil.Now()) - if envTableType == jobstest.UseTestTables { sqlDB.Exec(t, jobstest.GetScheduledJobsTableSchema(env)) sqlDB.Exec(t, jobstest.GetJobsTableSchema(env)) @@ -91,12 +89,9 @@ func newTestHelperForTables( DB: kvDB, TestingKnobs: knobs, }, - sqlDB: sqlDB, + sqlDB: sqlDB, + execSchedules: execSchedules, }, func() { - if cleanupIntervals != nil { - cleanupIntervals() - } - if envTableType == jobstest.UseTestTables { sqlDB.Exec(t, "DROP TABLE "+env.SystemJobsTableName()) sqlDB.Exec(t, "DROP TABLE "+env.ScheduledJobsTableName()) From bd03b1709f513ba06ef6f9aaf4bcc7be607ba506 Mon Sep 17 00:00:00 2001 From: arulajmani Date: Thu, 17 Sep 2020 17:42:31 -0400 Subject: [PATCH 2/2] sql: update search_path semantics to match pg now that we support UDS Postgres semantics dictate that the default search_path should be `$user, public`, where `$user` expands to the current user's username. This wasn't a problem until now as we lacked support for user defined schemas, which meant a schema by the name of `$user` wouldn't ever exist. This however had changed now and this patch brings us in line with the PG semantics. Fixes #53560 Release note (sql change): The default search path for all sessions is now `$user, public` (as opposed to just `public`). This affects our name resolution semantics -- now, if a table is present in both the public schema and the schema named the current user's username, an unqualified object name will be searched/placed in the user's schema. This doesn't impact the search semantics of tables in pg_catalog/information_schema/temp_schema -- these continued to be searched before checking the $user schema and the public schema. --- pkg/sql/catalog/catconstants/constants.go | 9 +-- pkg/sql/conn_executor.go | 3 +- pkg/sql/distsql/server.go | 8 +- pkg/sql/logictest/testdata/logic_test/discard | 2 +- .../logictest/testdata/logic_test/pg_catalog | 4 +- pkg/sql/logictest/testdata/logic_test/reset | 4 +- pkg/sql/logictest/testdata/logic_test/schema | 74 +++++++++++++++++++ .../logictest/testdata/logic_test/show_source | 2 +- pkg/sql/planner.go | 3 +- pkg/sql/schema_changer.go | 3 +- pkg/sql/sessiondata/search_path.go | 46 +++++++++++- pkg/sql/temporary_schema.go | 3 +- pkg/sql/vars.go | 3 +- 13 files changed, 135 insertions(+), 29 deletions(-) diff --git a/pkg/sql/catalog/catconstants/constants.go b/pkg/sql/catalog/catconstants/constants.go index 0b04fa57f7c7..2125afca2ace 100644 --- a/pkg/sql/catalog/catconstants/constants.go +++ b/pkg/sql/catalog/catconstants/constants.go @@ -10,14 +10,7 @@ package catconstants -import ( - "math" - - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" -) - -// DefaultSearchPath is the search path used by virgin sessions. -var DefaultSearchPath = sessiondata.MakeSearchPath([]string{"public"}) +import "math" // ReportableAppNamePrefix indicates that the application name can be // reported in telemetry without scrubbing. (Note this only applies to diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index dd9e0bba4f89..e9f5f591e2de 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -29,7 +29,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/database" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -545,7 +544,7 @@ func (s *Server) populateMinimalSessionData(sd *sessiondata.SessionData) { } } if len(sd.SearchPath.GetPathArray()) == 0 { - sd.SearchPath = catconstants.DefaultSearchPath + sd.SearchPath = sessiondata.DefaultSearchPathForUser(sd.User) } } diff --git a/pkg/sql/distsql/server.go b/pkg/sql/distsql/server.go index 540a9777c328..da2a91fc109e 100644 --- a/pkg/sql/distsql/server.go +++ b/pkg/sql/distsql/server.go @@ -284,8 +284,12 @@ func (ds *ServerImpl) setupFlow( ApplicationName: req.EvalContext.ApplicationName, Database: req.EvalContext.Database, User: req.EvalContext.User, - SearchPath: sessiondata.MakeSearchPath(req.EvalContext.SearchPath).WithTemporarySchemaName(req.EvalContext.TemporarySchemaName), - SequenceState: sessiondata.NewSequenceState(), + SearchPath: sessiondata.MakeSearchPath( + req.EvalContext.SearchPath, + ).WithTemporarySchemaName( + req.EvalContext.TemporarySchemaName, + ).WithUserSchemaName(req.EvalContext.User), + SequenceState: sessiondata.NewSequenceState(), DataConversion: sessiondata.DataConversionConfig{ Location: location, BytesEncodeFormat: be, diff --git a/pkg/sql/logictest/testdata/logic_test/discard b/pkg/sql/logictest/testdata/logic_test/discard index afa6ad74b253..a980b87e68ea 100644 --- a/pkg/sql/logictest/testdata/logic_test/discard +++ b/pkg/sql/logictest/testdata/logic_test/discard @@ -12,7 +12,7 @@ DISCARD ALL query T SHOW SEARCH_PATH ---- -public +$user,public query T SET timezone = 'Europe/Amsterdam'; SHOW TIMEZONE diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 70a8dc7eb1ec..74654dba84b1 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1854,7 +1854,7 @@ reorder_joins_limit 8 NULL NUL require_explicit_primary_keys off NULL NULL NULL string results_buffer_size 16384 NULL NULL NULL string row_security off NULL NULL NULL string -search_path public NULL NULL NULL string +search_path $user,public NULL NULL NULL string serial_normalization rowid NULL NULL NULL string server_encoding UTF8 NULL NULL NULL string server_version 9.5.0 NULL NULL NULL string @@ -1924,7 +1924,7 @@ reorder_joins_limit 8 NULL user require_explicit_primary_keys off NULL user NULL off off results_buffer_size 16384 NULL user NULL 16384 16384 row_security off NULL user NULL off off -search_path public NULL user NULL public public +search_path $user,public NULL user NULL $user,public $user,public serial_normalization rowid NULL user NULL rowid rowid server_encoding UTF8 NULL user NULL UTF8 UTF8 server_version 9.5.0 NULL user NULL 9.5.0 9.5.0 diff --git a/pkg/sql/logictest/testdata/logic_test/reset b/pkg/sql/logictest/testdata/logic_test/reset index ac0441a4ff5e..263427309a11 100644 --- a/pkg/sql/logictest/testdata/logic_test/reset +++ b/pkg/sql/logictest/testdata/logic_test/reset @@ -15,7 +15,7 @@ RESET SEARCH_PATH query T SHOW SEARCH_PATH ---- -public +$user,public statement error parameter "server_version" cannot be changed RESET SERVER_VERSION @@ -39,7 +39,7 @@ RESET search_path query T SHOW search_path ---- -public +$user,public statement ok RESET client_encoding; RESET NAMES diff --git a/pkg/sql/logictest/testdata/logic_test/schema b/pkg/sql/logictest/testdata/logic_test/schema index 48a5ad848040..3a6bbfe136be 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -459,3 +459,77 @@ CREATE TABLE new_db.public.bar() statement ok CREATE TABLE new_db.testuser.bar() + +# cleanup the testuser schema created as part of the CREATE SCHEMA AUTHORIZATION +# command above +statement ok +DROP SCHEMA testuser CASCADE + +# If a schema with a username exists, then that should be the first entry in +# the search path. +subtest user_schema_search_path + +# Test setup +user root + +statement ok +CREATE SCHEMA testuser + +statement ok +GRANT ALL ON SCHEMA testuser TO testuser + +statement ok +CREATE TABLE public.public_table(a INT) + +statement ok +GRANT SELECT ON public.public_table TO testuser + +user testuser + +statement ok +CREATE TABLE test_table(a INT); + +statement error pq: relation "public.test_table" does not exist +SELECT * FROM public.test_table + +statement ok +SELECT * FROM testuser.test_table + +# Only root has privs to create inside public +user root + +statement ok +CREATE TABLE public.test_table(a INT, b INT) + +statement ok +GRANT SELECT ON public.test_table TO testuser + +user testuser + +query I colnames +SELECT * FROM test_table +---- +a + +query II colnames +SELECT * FROM public.test_table +---- +a b + +query I colnames +SELECT * FROM public_table +---- +a + +# The search path is configured to be user specific. +user root + +query II colnames +SELECT * FROM test_table +---- +a b + +query I colnames +SELECT * FROM testuser.test_table +---- +a diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index ef38e1edaba4..404aade86a9c 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -67,7 +67,7 @@ reorder_joins_limit 8 require_explicit_primary_keys off results_buffer_size 16384 row_security off -search_path public +search_path $user,public serial_normalization rowid server_encoding UTF8 server_version 9.5.0 diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index f727c0d9c138..81a2738c661e 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" @@ -263,7 +262,7 @@ func newInternalPlanner( ctx := logtags.AddTag(context.Background(), opName, "") sd := &sessiondata.SessionData{ - SearchPath: catconstants.DefaultSearchPath, + SearchPath: sessiondata.DefaultSearchPathForUser(user), User: user, Database: "system", SequenceState: sessiondata.NewSequenceState(), diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 60c35911ed8e..293aee27e1e5 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -30,7 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" @@ -1972,7 +1971,7 @@ func createSchemaChangeEvalCtx( func newFakeSessionData() *sessiondata.SessionData { sd := &sessiondata.SessionData{ - SearchPath: catconstants.DefaultSearchPath, + SearchPath: sessiondata.DefaultSearchPathForUser(security.NodeUser), // The database is not supposed to be needed in schema changes, as there // shouldn't be unqualified identifiers in backfills, and the pure functions // that need it should have already been evaluated. diff --git a/pkg/sql/sessiondata/search_path.go b/pkg/sql/sessiondata/search_path.go index 38fe294bc956..e4de19f1961d 100644 --- a/pkg/sql/sessiondata/search_path.go +++ b/pkg/sql/sessiondata/search_path.go @@ -23,6 +23,9 @@ const PgCatalogName = "pg_catalog" // PublicSchemaName is the name of the pg_catalog system schema. const PublicSchemaName = "public" +// UserSchemaName is the alias for schema names for users. +const UserSchemaName = "$user" + // InformationSchemaName is the name of the information_schema system schema. const InformationSchemaName = "information_schema" @@ -40,6 +43,11 @@ const PgTempSchemaName = "pg_temp" // when installing an extension, but must be stored as a separate schema in CRDB. const PgExtensionSchemaName = "pg_extension" +// DefaultSearchPath is the search path used by virgin sessions. +var DefaultSearchPath = MakeSearchPath( + []string{UserSchemaName, PublicSchemaName}, +) + // SearchPath represents a list of namespaces to search builtins in. // The names must be normalized (as per Name.Normalize) already. type SearchPath struct { @@ -48,11 +56,18 @@ type SearchPath struct { containsPgExtension bool containsPgTempSchema bool tempSchemaName string + userSchemaName string } // EmptySearchPath is a SearchPath with no schema names in it. var EmptySearchPath = SearchPath{} +// DefaultSearchPathForUser returns the default search path with the user +// specific schema name set so that it can be expanded during resolution. +func DefaultSearchPathForUser(username string) SearchPath { + return DefaultSearchPath.WithUserSchemaName(username) +} + // MakeSearchPath returns a new immutable SearchPath struct. The paths slice // must not be modified after hand-off to MakeSearchPath. func MakeSearchPath(paths []string) SearchPath { @@ -87,14 +102,28 @@ func (s SearchPath) WithTemporarySchemaName(tempSchemaName string) SearchPath { containsPgCatalog: s.containsPgCatalog, containsPgTempSchema: s.containsPgTempSchema, containsPgExtension: s.containsPgExtension, + userSchemaName: s.userSchemaName, tempSchemaName: tempSchemaName, } } +// WithUserSchemaName returns a new immutable SearchPath struct with the +// userSchemaName populated and the same values for all other fields as before. +func (s SearchPath) WithUserSchemaName(userSchemaName string) SearchPath { + return SearchPath{ + paths: s.paths, + containsPgCatalog: s.containsPgCatalog, + containsPgTempSchema: s.containsPgTempSchema, + containsPgExtension: s.containsPgExtension, + userSchemaName: userSchemaName, + tempSchemaName: s.tempSchemaName, + } +} + // UpdatePaths returns a new immutable SearchPath struct with the paths supplied -// and the same tempSchemaName as before. +// and the same tempSchemaName and userSchemaName as before. func (s SearchPath) UpdatePaths(paths []string) SearchPath { - return MakeSearchPath(paths).WithTemporarySchemaName(s.tempSchemaName) + return MakeSearchPath(paths).WithTemporarySchemaName(s.tempSchemaName).WithUserSchemaName(s.userSchemaName) } // MaybeResolveTemporarySchema returns the session specific temporary schema @@ -135,6 +164,7 @@ func (s SearchPath) Iter() SearchPathIter { implicitPgExtension: !s.containsPgExtension, implicitPgTempSchema: implicitPgTempSchema, tempSchemaName: s.tempSchemaName, + userSchemaName: s.userSchemaName, } return sp } @@ -147,6 +177,7 @@ func (s SearchPath) IterWithoutImplicitPGSchemas() SearchPathIter { implicitPgCatalog: false, implicitPgTempSchema: false, tempSchemaName: s.tempSchemaName, + userSchemaName: s.userSchemaName, } return sp } @@ -202,7 +233,7 @@ func (s SearchPath) Equals(other *SearchPath) bool { } func (s SearchPath) String() string { - return strings.Join(s.paths, ", ") + return strings.Join(s.paths, ",") } // SearchPathIter enables iteration over the search paths without triggering an @@ -216,6 +247,7 @@ type SearchPathIter struct { implicitPgExtension bool implicitPgTempSchema bool tempSchemaName string + userSchemaName string i int } @@ -245,6 +277,14 @@ func (iter *SearchPathIter) Next() (path string, ok bool) { } return iter.tempSchemaName, true } + if iter.paths[iter.i-1] == UserSchemaName { + // In case the user schema name is unset, we simply iterate to the next + // entry. + if iter.userSchemaName == "" { + return iter.Next() + } + return iter.userSchemaName, true + } // pg_extension should be read before delving into the schema. if iter.paths[iter.i-1] == PublicSchemaName && iter.implicitPgExtension { iter.implicitPgExtension = false diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go index 2a3ba7742baa..02cc7bf6425f 100644 --- a/pkg/sql/temporary_schema.go +++ b/pkg/sql/temporary_schema.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -229,7 +228,7 @@ func cleanupSchemaObjects( } a := catalogkv.UncachedPhysicalAccessor{} - searchPath := catconstants.DefaultSearchPath.WithTemporarySchemaName(schemaName) + searchPath := sessiondata.DefaultSearchPathForUser(security.RootUser).WithTemporarySchemaName(schemaName) override := sessiondata.InternalExecutorOverride{ SearchPath: &searchPath, User: security.RootUser, diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 8484c78545c2..0d4b5bb3db20 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/delegate" "github.com/cockroachdb/cockroach/pkg/sql/lex" "github.com/cockroachdb/cockroach/pkg/sql/paramparse" @@ -827,7 +826,7 @@ var varGen = map[string]sessionVar{ return evalCtx.SessionData.SearchPath.String() }, GlobalDefault: func(sv *settings.Values) string { - return catconstants.DefaultSearchPath.String() + return sessiondata.DefaultSearchPath.String() }, },