Skip to content

Commit

Permalink
sql: rename NewFakeSessionData to NewInternalSessionData
Browse files Browse the repository at this point in the history
Release note: None
  • Loading branch information
rafiss committed Apr 13, 2023
1 parent 2791b25 commit 03b12de
Show file tree
Hide file tree
Showing 22 changed files with 69 additions and 69 deletions.
2 changes: 1 addition & 1 deletion pkg/bench/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1474,7 +1474,7 @@ func BenchmarkFuncExprTypeCheck(b *testing.B) {

ctx := context.Background()
execCfg := s.ExecutorConfig().(sql.ExecutorConfig)
sd := sql.NewFakeSessionData(ctx, execCfg.Settings, "type-check-benchmark")
sd := sql.NewInternalSessionData(ctx, execCfg.Settings, "type-check-benchmark")
sd.Database = "defaultdb"
p, cleanup := sql.NewInternalPlanner(
"type-check-benchmark",
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/scheduled_changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -707,7 +707,7 @@ func TestCheckScheduleAlreadyExists(t *testing.T) {

ctx := context.Background()

sd := sql.NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := sql.NewInternalSessionData(ctx, execCfg.Settings, "test")
sd.Database = "d"
p, cleanup := sql.NewInternalPlanner("test",
execCfg.DB.NewTxn(ctx, "test-planner"),
Expand Down Expand Up @@ -742,7 +742,7 @@ func TestFullyQualifyTables(t *testing.T) {
require.NoError(t, err)
createChangeFeedStmt := stmt.AST.(*tree.CreateChangefeed)

sd := sql.NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := sql.NewInternalSessionData(ctx, execCfg.Settings, "test")
sd.Database = "ocean"
p, cleanupPlanHook := sql.NewInternalPlanner("test",
execCfg.DB.NewTxn(ctx, "test-planner"),
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3237,7 +3237,7 @@ func TestAdminPrivilegeChecker(t *testing.T) {
username.RootUserName(),
&sql.MemoryMetrics{},
&execCfg,
sql.NewFakeSessionData(ctx, execCfg.Settings, opName),
sql.NewInternalSessionData(ctx, execCfg.Settings, opName),
)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1065,7 +1065,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
username.RootUserName(),
&sql.MemoryMetrics{},
sqlServer.execCfg,
sql.NewFakeSessionData(ctx, sqlServer.execCfg.Settings, opName),
sql.NewInternalSessionData(ctx, sqlServer.execCfg.Settings, opName),
)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/server/server_internal_executor_factory_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ func TestInternalExecutorClearsMonitorMemory(t *testing.T) {

mon := s.(*TestServer).sqlServer.internalDBMemMonitor
ief := s.ExecutorConfig().(sql.ExecutorConfig).InternalDB
sessionData := sql.NewFakeSessionData(ctx, s.ClusterSettings(), "TestInternalExecutorClearsMonitorMemory")
sessionData := sql.NewInternalSessionData(ctx, s.ClusterSettings(), "TestInternalExecutorClearsMonitorMemory")
ie := ief.NewInternalExecutor(sessionData)
rows, err := ie.QueryIteratorEx(ctx, "test", nil, sessiondata.NodeUserSessionDataOverride, `SELECT 1`)
require.NoError(t, err)
Expand Down
4 changes: 2 additions & 2 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1191,7 +1191,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
sql.ValidateForwardIndexes,
sql.ValidateInvertedIndexes,
sql.ValidateConstraint,
sql.NewFakeSessionData,
sql.NewInternalSessionData,
)

jobsInternalDB := sql.NewInternalDBWithSessionDataOverrides(internalDB, func(sd *sessiondata.SessionData) {
Expand Down Expand Up @@ -1662,7 +1662,7 @@ func (s *SQLServer) preStart(
user,
&sql.MemoryMetrics{},
s.execCfg,
sql.NewFakeSessionData(ctx, s.execCfg.Settings, opName),
sql.NewInternalSessionData(ctx, s.execCfg.Settings, opName),
)
},
},
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -367,7 +367,7 @@ func newTenantServer(
username.RootUserName(),
&sql.MemoryMetrics{},
sqlServer.execCfg,
sql.NewFakeSessionData(ctx, sqlServer.execCfg.Settings, opName),
sql.NewInternalSessionData(ctx, sqlServer.execCfg.Settings, opName),
)
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -795,7 +795,7 @@ func (sc *SchemaChanger) validateConstraints(
semaCtx.TypeResolver = &resolver
semaCtx.NameResolver = NewSkippingCacheSchemaResolver(
txn.Descriptors(),
sessiondata.NewStack(NewFakeSessionData(ctx, sc.settings, "validate constraint")),
sessiondata.NewStack(NewInternalSessionData(ctx, sc.settings, "validate constraint")),
txn.KV(),
nil, /* authAccessor */
)
Expand Down Expand Up @@ -1055,7 +1055,7 @@ func (sc *SchemaChanger) distIndexBackfill(
if err != nil {
return err
}
sd := NewFakeSessionData(ctx, sc.execCfg.Settings, "dist-index-backfill")
sd := NewInternalSessionData(ctx, sc.execCfg.Settings, "dist-index-backfill")
evalCtx = createSchemaChangeEvalCtx(ctx, sc.execCfg, sd, txn.KV().ReadTimestamp(), txn.Descriptors())
planCtx = sc.distSQLPlanner.NewPlanningCtx(ctx, &evalCtx, nil, /* planner */
txn.KV(), DistributionTypeSystemTenantOnly)
Expand Down Expand Up @@ -1351,7 +1351,7 @@ func (sc *SchemaChanger) distColumnBackfill(
return nil
}
cbw := MetadataCallbackWriter{rowResultWriter: &errOnlyResultWriter{}, fn: metaFn}
sd := NewFakeSessionData(ctx, sc.execCfg.Settings, "dist-column-backfill")
sd := NewInternalSessionData(ctx, sc.execCfg.Settings, "dist-column-backfill")
evalCtx := createSchemaChangeEvalCtx(ctx, sc.execCfg, sd, txn.KV().ReadTimestamp(), txn.Descriptors())
recv := MakeDistSQLReceiver(
ctx,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/descs/collection_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -856,7 +856,7 @@ parent schema name id kind version dropped public
`, formatCatalog(allDescs.OrderedDescriptors()))
return nil
}
sd := sql.NewFakeSessionData(&s0.ClusterSettings().SV, "TestGetAllDescriptorsInDatabase")
sd := sql.NewInternalSessionData(&s0.ClusterSettings().SV, "TestGetAllDescriptorsInDatabase")
sd.Database = "db"
require.NoError(t, tm.DescsTxn(ctx, run, isql.WithSessionData(sd)))
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/distsql_plan_changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ CREATE TABLE foo (

ctx := context.Background()
execCfg := s.ExecutorConfig().(ExecutorConfig)
sd := NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := NewInternalSessionData(ctx, execCfg.Settings, "test")
sd.Database = "defaultdb"

p, cleanup := NewInternalPlanner("test", kv.NewTxn(ctx, kvDB, s.NodeID()),
Expand Down Expand Up @@ -464,7 +464,7 @@ func TestChangefeedStreamsResults(t *testing.T) {

ctx := context.Background()
execCfg := s.ExecutorConfig().(ExecutorConfig)
sd := NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := NewInternalSessionData(ctx, execCfg.Settings, "test")
sd.Database = "defaultdb"
p, cleanup := NewInternalPlanner("test", kv.NewTxn(ctx, kvDB, s.NodeID()),
username.RootUserName(), &MemoryMetrics{}, &execCfg, sd,
Expand Down Expand Up @@ -511,7 +511,7 @@ FAMILY extra (extra)

ctx := context.Background()
execCfg := s.ExecutorConfig().(ExecutorConfig)
sd := NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := NewInternalSessionData(ctx, execCfg.Settings, "test")
sd.Database = "defaultdb"
p, cleanup := NewInternalPlanner("test", kv.NewTxn(ctx, kvDB, s.NodeID()),
username.RootUserName(), &MemoryMetrics{}, &execCfg, sd,
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/distsql_running_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ func TestDistSQLRunningInAbortedTxn(t *testing.T) {

// Plan a statement.
execCfg := s.ExecutorConfig().(ExecutorConfig)
sd := NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := NewInternalSessionData(ctx, execCfg.Settings, "test")
internalPlanner, cleanup := NewInternalPlanner(
"test",
kv.NewTxn(ctx, db, s.NodeID()),
Expand Down Expand Up @@ -252,7 +252,7 @@ func TestDistSQLRunningParallelFKChecksAfterAbort(t *testing.T) {

createPlannerAndRunQuery := func(ctx context.Context, txn *kv.Txn, query string) error {
execCfg := s.ExecutorConfig().(ExecutorConfig)
sd := NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := NewInternalSessionData(ctx, execCfg.Settings, "test")
// Plan the statement.
internalPlanner, cleanup := NewInternalPlanner(
"test",
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/explain_tree_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ func TestPlanToTreeAndPlanToString(t *testing.T) {
t.Fatal(err)
}

sd := NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := NewInternalSessionData(ctx, execCfg.Settings, "test")
internalPlanner, cleanup := NewInternalPlanner(
"test",
kv.NewTxn(ctx, db, s.NodeID()),
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/flowinfra/flow_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ func BenchmarkFlowSetup(b *testing.B) {
if vectorize {
vectorizeMode = sessiondatapb.VectorizeOn
}
sd := sql.NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := sql.NewInternalSessionData(ctx, execCfg.Settings, "test")
sd.VectorizeMode = vectorizeMode
b.ResetTimer()
for i := 0; i < b.N; i++ {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/index_backfiller.go
Original file line number Diff line number Diff line change
Expand Up @@ -174,7 +174,7 @@ func (ib *IndexBackfillPlanner) plan(
if err := DescsTxn(ctx, ib.execCfg, func(
ctx context.Context, txn isql.Txn, descriptors *descs.Collection,
) error {
sd := NewFakeSessionData(ctx, ib.execCfg.Settings, "plan-index-backfill")
sd := NewInternalSessionData(ctx, ib.execCfg.Settings, "plan-index-backfill")
evalCtx = createSchemaChangeEvalCtx(ctx, ib.execCfg, sd, nowTimestamp, descriptors)
planCtx = ib.execCfg.DistSQLPlanner.NewPlanningCtx(ctx, &evalCtx,
nil /* planner */, txn.KV(), DistributionTypeSystemTenantOnly)
Expand Down
42 changes: 41 additions & 1 deletion pkg/sql/internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,46 @@ import (
"github.com/cockroachdb/logtags"
)

// NewInternalSessionData returns a session data for use in internal queries
// that are not run on behalf of a user session, such as those run during the
// steps of background jobs and schema changes. Each session variable is
// initialized using the correct default value.
func NewInternalSessionData(
ctx context.Context, settings *cluster.Settings, opName string,
) *sessiondata.SessionData {
appName := catconstants.InternalAppNamePrefix
if opName != "" {
appName = catconstants.InternalAppNamePrefix + "-" + opName
}

sd := &sessiondata.SessionData{}
sds := sessiondata.NewStack(sd)
defaults := SessionDefaults(map[string]string{
"application_name": appName,
})
sdMutIterator := makeSessionDataMutatorIterator(sds, defaults, settings)

sdMutIterator.applyOnEachMutator(func(m sessionDataMutator) {
for varName, v := range varGen {
if v.Set != nil {
hasDefault, defVal := getSessionVarDefaultString(varName, v, m.sessionDataMutatorBase)
if hasDefault {
if err := v.Set(ctx, m, defVal); err != nil {
panic(err)
}
}
}
}
})

sd.UserProto = username.NodeUserName().EncodeProto()
sd.Internal = true
sd.SearchPath = sessiondata.DefaultSearchPathForUser(username.NodeUserName())
sd.SequenceState = sessiondata.NewSequenceState()
sd.Location = time.UTC
return sd
}

var _ isql.Executor = &InternalExecutor{}

// InternalExecutor can be used internally by code modules to execute SQL
Expand Down Expand Up @@ -1384,7 +1424,7 @@ func (ief *InternalDB) newInternalExecutorWithTxn(
// than the actual user, a security boundary should be added to the error
// handling of internal executor.
if sd == nil {
sd = NewFakeSessionData(ctx, settings, "" /* opName */)
sd = NewInternalSessionData(ctx, settings, "" /* opName */)
sd.UserProto = username.RootUserName().EncodeProto()
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/job_exec_context.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ func MakeJobExecContext(
user,
memMetrics,
execCfg,
NewFakeSessionData(ctx, execCfg.Settings, opName),
NewInternalSessionData(ctx, execCfg.Settings, opName),
)
p := plannerInterface.(*planner)
return &plannerJobExecContext{p: p}, close
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/mvcc_backfiller.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ func (im *IndexBackfillerMergePlanner) plan(
if err := DescsTxn(ctx, im.execCfg, func(
ctx context.Context, txn isql.Txn, descriptors *descs.Collection,
) error {
sd := NewFakeSessionData(ctx, im.execCfg.Settings, "plan-index-backfill-merge")
sd := NewInternalSessionData(ctx, im.execCfg.Settings, "plan-index-backfill-merge")
evalCtx = createSchemaChangeEvalCtx(ctx, im.execCfg, sd, txn.KV().ReadTimestamp(), descriptors)
planCtx = im.execCfg.DistSQLPlanner.NewPlanningCtx(ctx, &evalCtx, nil /* planner */, txn.KV(),
DistributionTypeSystemTenantOnly)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/plan_opt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -656,7 +656,7 @@ func TestPlanGistControl(t *testing.T) {
s, _, db := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
execCfg := s.ExecutorConfig().(ExecutorConfig)
sd := NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := NewInternalSessionData(ctx, execCfg.Settings, "test")
internalPlanner, cleanup := NewInternalPlanner(
"test",
kv.NewTxn(ctx, db, s.NodeID()),
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/reference_provider.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ func NewReferenceProviderFactoryForTest(
execCfg *ExecutorConfig,
curDB string,
) (scbuild.ReferenceProviderFactory, func()) {
sd := NewFakeSessionData(ctx, execCfg.Settings, opName)
sd := NewInternalSessionData(ctx, execCfg.Settings, opName)
sd.Database = "defaultdb"
ip, cleanup := newInternalPlanner(opName, txn, user, &MemoryMetrics{}, execCfg, sd)
return &referenceProviderFactory{p: ip}, cleanup
Expand Down
42 changes: 1 addition & 41 deletions pkg/sql/schema_changer.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
Expand Down Expand Up @@ -288,7 +287,7 @@ func (sc *SchemaChanger) backfillQueryIntoTable(
username.RootUserName(),
&MemoryMetrics{},
sc.execCfg,
NewFakeSessionData(ctx, sc.execCfg.Settings, "backfillQueryIntoTable"),
NewInternalSessionData(ctx, sc.execCfg.Settings, "backfillQueryIntoTable"),
)

defer cleanup()
Expand Down Expand Up @@ -2522,45 +2521,6 @@ func createSchemaChangeEvalCtx(
return evalCtx
}

// NewFakeSessionData returns "fake" session data for use in internal queries
// that are not run on behalf of a user session, such as those run during the
// steps of background jobs and schema changes.
func NewFakeSessionData(
ctx context.Context, settings *cluster.Settings, opName string,
) *sessiondata.SessionData {
appName := catconstants.InternalAppNamePrefix
if opName != "" {
appName = catconstants.InternalAppNamePrefix + "-" + opName
}

sd := &sessiondata.SessionData{}
sds := sessiondata.NewStack(sd)
defaults := SessionDefaults(map[string]string{
"application_name": appName,
})
sdMutIterator := makeSessionDataMutatorIterator(sds, defaults, settings)

sdMutIterator.applyOnEachMutator(func(m sessionDataMutator) {
for varName, v := range varGen {
if v.Set != nil {
hasDefault, defVal := getSessionVarDefaultString(varName, v, m.sessionDataMutatorBase)
if hasDefault {
if err := v.Set(ctx, m, defVal); err != nil {
panic(err)
}
}
}
}
})

sd.UserProto = username.NodeUserName().EncodeProto()
sd.Internal = true
sd.SearchPath = sessiondata.DefaultSearchPathForUser(username.NodeUserName())
sd.SequenceState = sessiondata.NewSequenceState()
sd.Location = time.UTC
return sd
}

type schemaChangeResumer struct {
job *jobs.Job
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/schemachanger/scdeps/sctestutils/sctestutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ func WithBuilderDependenciesFromTestServer(
) {
ctx := context.Background()
execCfg := s.ExecutorConfig().(sql.ExecutorConfig)
sd := sql.NewFakeSessionData(ctx, execCfg.Settings, "test")
sd := sql.NewInternalSessionData(ctx, execCfg.Settings, "test")
sd.Database = "defaultdb"
ip, cleanup := sql.NewInternalPlanner(
"test",
Expand Down
2 changes: 1 addition & 1 deletion pkg/upgrade/upgradejob/upgrade_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,7 @@ func (r resumer) Resume(ctx context.Context, execCtxI interface{}) error {
txn *kv.Txn, descriptors *descs.Collection, currDb string,
) (resolver.SchemaResolver, func(), error) {
opName := "internal-planner-for-upgrades"
sd := sql.NewFakeSessionData(ctx, execCtx.ExecCfg().Settings, opName)
sd := sql.NewInternalSessionData(ctx, execCtx.ExecCfg().Settings, opName)
sd.Database = currDb
internalPlanner, cleanup := sql.NewInternalPlanner(
opName,
Expand Down

0 comments on commit 03b12de

Please sign in to comment.