Skip to content

Commit

Permalink
*: make sessionctx.Context depend on contexts in planner and expressi…
Browse files Browse the repository at this point in the history
…on (#51150)

close #51149
  • Loading branch information
lcwangchao authored Feb 19, 2024
1 parent fa340f3 commit 4e202a4
Show file tree
Hide file tree
Showing 22 changed files with 148 additions and 66 deletions.
1 change: 1 addition & 0 deletions br/pkg/lightning/backend/kv/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_library(
"//br/pkg/utils",
"//pkg/errctx",
"//pkg/expression",
"//pkg/infoschema/context",
"//pkg/kv",
"//pkg/meta/autoid",
"//pkg/parser/model",
Expand Down
3 changes: 2 additions & 1 deletion br/pkg/lightning/backend/kv/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/br/pkg/lightning/manual"
"github.com/pingcap/tidb/br/pkg/utils"
"github.com/pingcap/tidb/pkg/errctx"
infoschema "github.com/pingcap/tidb/pkg/infoschema/context"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx"
Expand Down Expand Up @@ -367,7 +368,7 @@ func (se *Session) Value(key fmt.Stringer) any {
func (*Session) StmtAddDirtyTableOP(_ int, _ int64, _ kv.Handle) {}

// GetInfoSchema implements the sessionctx.Context interface.
func (*Session) GetInfoSchema() sessionctx.InfoschemaMetaVersion {
func (*Session) GetInfoSchema() infoschema.InfoSchemaMetaVersion {
return nil
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/expression/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@ go_library(
"//pkg/config",
"//pkg/errctx",
"//pkg/errno",
"//pkg/expression/context",
"//pkg/extension",
"//pkg/kv",
"//pkg/parser",
Expand All @@ -86,7 +87,6 @@ go_library(
"//pkg/parser/terror",
"//pkg/parser/types",
"//pkg/privilege",
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/sessionctx/variable",
"//pkg/types",
Expand Down
43 changes: 5 additions & 38 deletions pkg/expression/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,42 +15,21 @@
package expression

import (
"fmt"
"time"

"github.com/pingcap/tidb/pkg/errctx"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/expression/context"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/intest"
)

// EvalContext is used to evaluate an expression
type EvalContext interface {
// GetSessionVars gets the session variables.
GetSessionVars() *variable.SessionVars
// Value returns the value associated with this context for key.
Value(key fmt.Stringer) any
// IsDDLOwner checks whether this session is DDL owner.
IsDDLOwner() bool
// GetAdvisoryLock acquires an advisory lock (aka GET_LOCK()).
GetAdvisoryLock(string, int64) error
// IsUsedAdvisoryLock checks for existing locks (aka IS_USED_LOCK()).
IsUsedAdvisoryLock(string) uint64
// ReleaseAdvisoryLock releases an advisory lock (aka RELEASE_LOCK()).
ReleaseAdvisoryLock(string) bool
// ReleaseAllAdvisoryLocks releases all advisory locks that this session holds.
ReleaseAllAdvisoryLocks() int
// GetStore returns the store of session.
GetStore() kv.Storage
// GetInfoSchema returns the current infoschema
GetInfoSchema() sessionctx.InfoschemaMetaVersion
// GetDomainInfoSchema returns the latest information schema in domain
GetDomainInfoSchema() sessionctx.InfoschemaMetaVersion
}
type EvalContext = context.EvalContext

// BuildContext is used to build an expression
type BuildContext = context.BuildContext

func sqlMode(ctx EvalContext) mysql.SQLMode {
return ctx.GetSessionVars().SQLMode
Expand Down Expand Up @@ -80,15 +59,3 @@ func warningCount(ctx EvalContext) int {
func truncateWarnings(ctx EvalContext, start int) []stmtctx.SQLWarn {
return ctx.GetSessionVars().StmtCtx.TruncateWarnings(start)
}

// BuildContext is used to build an expression
type BuildContext interface {
EvalContext
// GetSessionVars gets the session variables.
GetSessionVars() *variable.SessionVars
// SetValue saves a value associated with this context for key.
SetValue(key fmt.Stringer, value any)
// BuiltinFunctionUsageInc increase the counting of each builtin function usage
// Notice that this is a thread safe function
BuiltinFunctionUsageInc(scalarFuncSigName string)
}
13 changes: 13 additions & 0 deletions pkg/expression/context/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "context",
srcs = ["context.go"],
importpath = "github.com/pingcap/tidb/pkg/expression/context",
visibility = ["//visibility:public"],
deps = [
"//pkg/infoschema/context",
"//pkg/kv",
"//pkg/sessionctx/variable",
],
)
59 changes: 59 additions & 0 deletions pkg/expression/context/context.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
// Copyright 2024 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package context

import (
"fmt"

infoschema "github.com/pingcap/tidb/pkg/infoschema/context"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
)

// EvalContext is used to evaluate an expression
type EvalContext interface {
// GetSessionVars gets the session variables.
GetSessionVars() *variable.SessionVars
// Value returns the value associated with this context for key.
Value(key fmt.Stringer) any
// IsDDLOwner checks whether this session is DDL owner.
IsDDLOwner() bool
// GetAdvisoryLock acquires an advisory lock (aka GET_LOCK()).
GetAdvisoryLock(string, int64) error
// IsUsedAdvisoryLock checks for existing locks (aka IS_USED_LOCK()).
IsUsedAdvisoryLock(string) uint64
// ReleaseAdvisoryLock releases an advisory lock (aka RELEASE_LOCK()).
ReleaseAdvisoryLock(string) bool
// ReleaseAllAdvisoryLocks releases all advisory locks that this session holds.
ReleaseAllAdvisoryLocks() int
// GetStore returns the store of session.
GetStore() kv.Storage
// GetInfoSchema returns the current infoschema
GetInfoSchema() infoschema.InfoSchemaMetaVersion
// GetDomainInfoSchema returns the latest information schema in domain
GetDomainInfoSchema() infoschema.InfoSchemaMetaVersion
}

// BuildContext is used to build an expression
type BuildContext interface {
EvalContext
// GetSessionVars gets the session variables.
GetSessionVars() *variable.SessionVars
// SetValue saves a value associated with this context for key.
SetValue(key fmt.Stringer, value any)
// BuiltinFunctionUsageInc increase the counting of each builtin function usage
// Notice that this is a thread safe function
BuiltinFunctionUsageInc(scalarFuncSigName string)
}
1 change: 1 addition & 0 deletions pkg/infoschema/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ go_library(
"//pkg/ddl/resourcegroup",
"//pkg/domain/infosync",
"//pkg/errno",
"//pkg/infoschema/context",
"//pkg/infoschema/metrics",
"//pkg/kv",
"//pkg/meta",
Expand Down
8 changes: 8 additions & 0 deletions pkg/infoschema/context/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,8 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "context",
srcs = ["infoschema.go"],
importpath = "github.com/pingcap/tidb/pkg/infoschema/context",
visibility = ["//visibility:public"],
)
23 changes: 23 additions & 0 deletions pkg/infoschema/context/infoschema.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
// Copyright 2024 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package context

// InfoSchemaMetaVersion is a workaround. Due to circular dependency,
// can not return the complete interface. But SchemaMetaVersion is widely used for logging.
// So we give a convenience for that.
// FIXME: remove this interface
type InfoSchemaMetaVersion interface {
SchemaMetaVersion() int64
}
4 changes: 2 additions & 2 deletions pkg/infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,10 @@ import (
"sync"

"github.com/pingcap/tidb/pkg/ddl/placement"
"github.com/pingcap/tidb/pkg/infoschema/context"
"github.com/pingcap/tidb/pkg/meta/autoid"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/intest"
Expand Down Expand Up @@ -388,7 +388,7 @@ func init() {
util.GetSequenceByName = func(is any, schema, sequence model.CIStr) (util.SequenceTable, error) {
return GetSequenceByName(is.(InfoSchema), schema, sequence)
}
mock.MockInfoschema = func(tbList []*model.TableInfo) sessionctx.InfoschemaMetaVersion {
mock.MockInfoschema = func(tbList []*model.TableInfo) context.InfoSchemaMetaVersion {
return MockInfoSchema(tbList)
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/context/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@ go_library(
importpath = "github.com/pingcap/tidb/pkg/planner/context",
visibility = ["//visibility:public"],
deps = [
"//pkg/expression",
"//pkg/expression/context",
"//pkg/kv",
"//pkg/parser/model",
"//pkg/sessionctx/variable",
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/context/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
package context

import (
"github.com/pingcap/tidb/pkg/expression"
exprctx "github.com/pingcap/tidb/pkg/expression/context"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
Expand All @@ -24,7 +24,7 @@ import (

// PlanContext is the context for building plan.
type PlanContext interface {
expression.BuildContext
exprctx.BuildContext
contextutil.ValueStoreContext
// GetSessionVars gets the session variables.
GetSessionVars() *variable.SessionVars
Expand Down
1 change: 1 addition & 0 deletions pkg/session/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ go_library(
"//pkg/extension",
"//pkg/extension/extensionimpl",
"//pkg/infoschema",
"//pkg/infoschema/context",
"//pkg/kv",
"//pkg/meta",
"//pkg/metrics",
Expand Down
5 changes: 3 additions & 2 deletions pkg/session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ import (
"github.com/pingcap/tidb/pkg/extension"
"github.com/pingcap/tidb/pkg/extension/extensionimpl"
"github.com/pingcap/tidb/pkg/infoschema"
infoschemactx "github.com/pingcap/tidb/pkg/infoschema/context"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta"
"github.com/pingcap/tidb/pkg/metrics"
Expand Down Expand Up @@ -4050,7 +4051,7 @@ func (s *session) GetTxnWriteThroughputSLI() *sli.TxnWriteThroughputSLI {
// GetInfoSchema returns snapshotInfoSchema if snapshot schema is set.
// Transaction infoschema is returned if inside an explicit txn.
// Otherwise the latest infoschema is returned.
func (s *session) GetInfoSchema() sessionctx.InfoschemaMetaVersion {
func (s *session) GetInfoSchema() infoschemactx.InfoSchemaMetaVersion {
vars := s.GetSessionVars()
var is infoschema.InfoSchema
if snap, ok := vars.SnapshotInfoschema.(infoschema.InfoSchema); ok {
Expand All @@ -4074,7 +4075,7 @@ func (s *session) GetInfoSchema() sessionctx.InfoschemaMetaVersion {
return temptable.AttachLocalTemporaryTableInfoSchema(s, is)
}

func (s *session) GetDomainInfoSchema() sessionctx.InfoschemaMetaVersion {
func (s *session) GetDomainInfoSchema() infoschemactx.InfoSchemaMetaVersion {
is := domain.GetDomain(s).InfoSchema()
extIs := &infoschema.SessionExtendedInfoSchema{InfoSchema: is}
return temptable.AttachLocalTemporaryTableInfoSchema(s, extIs)
Expand Down
3 changes: 3 additions & 0 deletions pkg/sessionctx/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,10 +6,13 @@ go_library(
importpath = "github.com/pingcap/tidb/pkg/sessionctx",
visibility = ["//visibility:public"],
deps = [
"//pkg/expression/context",
"//pkg/extension",
"//pkg/infoschema/context",
"//pkg/kv",
"//pkg/metrics",
"//pkg/parser/model",
"//pkg/planner/context",
"//pkg/sessionctx/sessionstates",
"//pkg/sessionctx/variable",
"//pkg/statistics/handle/usage/indexusage",
Expand Down
18 changes: 8 additions & 10 deletions pkg/sessionctx/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,13 @@ import (

"github.com/pingcap/errors"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
exprctx "github.com/pingcap/tidb/pkg/expression/context"
"github.com/pingcap/tidb/pkg/extension"
infoschema "github.com/pingcap/tidb/pkg/infoschema/context"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/metrics"
"github.com/pingcap/tidb/pkg/parser/model"
planctx "github.com/pingcap/tidb/pkg/planner/context"
"github.com/pingcap/tidb/pkg/sessionctx/sessionstates"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/statistics/handle/usage/indexusage"
Expand All @@ -37,14 +40,6 @@ import (
"github.com/tikv/client-go/v2/oracle"
)

// InfoschemaMetaVersion is a workaround. Due to circular dependency,
// can not return the complete interface. But SchemaMetaVersion is widely used for logging.
// So we give a convenience for that.
// FIXME: remove this interface
type InfoschemaMetaVersion interface {
SchemaMetaVersion() int64
}

// SessionStatesHandler is an interface for encoding and decoding session states.
type SessionStatesHandler interface {
// EncodeSessionStates encodes session states into a JSON.
Expand All @@ -68,6 +63,9 @@ type PlanCache interface {
type Context interface {
SessionStatesHandler
contextutil.ValueStoreContext
exprctx.EvalContext
exprctx.BuildContext
planctx.PlanContext
// SetDiskFullOpt set the disk full opt when tikv disk full happened.
SetDiskFullOpt(level kvrpcpb.DiskFullOpt)
// RollbackTxn rolls back the current transaction.
Expand All @@ -89,12 +87,12 @@ type Context interface {
// Deprecated: the semantics of session.GetInfoSchema() is ambiguous
// If you want to get the infoschema of the current transaction in SQL layer, use sessiontxn.GetTxnManager(ctx).GetTxnInfoSchema()
// If you want to get the latest infoschema use `GetDomainInfoSchema`
GetInfoSchema() InfoschemaMetaVersion
GetInfoSchema() infoschema.InfoSchemaMetaVersion

// GetDomainInfoSchema returns the latest information schema in domain
// Different with `domain.InfoSchema()`, the information schema returned by this method
// includes the temporary table definitions stored in session
GetDomainInfoSchema() InfoschemaMetaVersion
GetDomainInfoSchema() infoschema.InfoSchemaMetaVersion

GetSessionVars() *variable.SessionVars

Expand Down
1 change: 1 addition & 0 deletions pkg/table/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ go_library(
"//pkg/errctx",
"//pkg/errno",
"//pkg/expression",
"//pkg/infoschema/context",
"//pkg/kv",
"//pkg/meta/autoid",
"//pkg/parser",
Expand Down
3 changes: 2 additions & 1 deletion pkg/table/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (

mysql "github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/expression"
infoschema "github.com/pingcap/tidb/pkg/infoschema/context"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta/autoid"
"github.com/pingcap/tidb/pkg/parser/model"
Expand Down Expand Up @@ -188,7 +189,7 @@ type MutateContext interface {
// StmtGetMutation gets the binlog mutation for current statement.
StmtGetMutation(int64) *binlog.TableMutation
// GetDomainInfoSchema returns the latest information schema in domain
GetDomainInfoSchema() sessionctx.InfoschemaMetaVersion
GetDomainInfoSchema() infoschema.InfoSchemaMetaVersion
}

// AllocatorContext is used to provide context for method `table.Allocators`.
Expand Down
Loading

0 comments on commit 4e202a4

Please sign in to comment.