Skip to content

Commit

Permalink
session: move session interface into new package (#48792)
Browse files Browse the repository at this point in the history
ref #44940
  • Loading branch information
hawkingrei committed Nov 23, 2023
1 parent 7e8de2f commit b8515ae
Show file tree
Hide file tree
Showing 53 changed files with 405 additions and 307 deletions.
1 change: 1 addition & 0 deletions br/pkg/gluetidb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_library(
"//pkg/parser/model",
"//pkg/parser/mysql",
"//pkg/session",
"//pkg/session/types",
"//pkg/sessionctx",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_log//:log",
Expand Down
9 changes: 5 additions & 4 deletions br/pkg/gluetidb/glue.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx"
pd "github.com/tikv/pd/client"
"go.uber.org/zap"
Expand Down Expand Up @@ -58,7 +59,7 @@ type Glue struct {
}

type tidbSession struct {
se session.Session
se sessiontypes.Session
}

// GetDomain implements glue.Glue.
Expand Down Expand Up @@ -358,7 +359,7 @@ func (gs *tidbSession) showCreatePlacementPolicy(policy *model.PolicyInfo) strin

// mockSession is used for test.
type mockSession struct {
se session.Session
se sessiontypes.Session
globalVars map[string]string
}

Expand Down Expand Up @@ -434,11 +435,11 @@ func (s *mockSession) GetGlobalVariable(name string) (string, error) {

// MockGlue only used for test
type MockGlue struct {
se session.Session
se sessiontypes.Session
GlobalVars map[string]string
}

func (m *MockGlue) SetSession(se session.Session) {
func (m *MockGlue) SetSession(se sessiontypes.Session) {
m.se = se
}

Expand Down
1 change: 1 addition & 0 deletions cmd/benchdb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ go_library(
deps = [
"//pkg/parser/terror",
"//pkg/session",
"//pkg/session/types",
"//pkg/store",
"//pkg/store/driver",
"//pkg/util/logutil",
Expand Down
3 changes: 2 additions & 1 deletion cmd/benchdb/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/log"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/store"
"github.com/pingcap/tidb/pkg/store/driver"
"github.com/pingcap/tidb/pkg/util/logutil"
Expand Down Expand Up @@ -88,7 +89,7 @@ func main() {

type benchDB struct {
store tikv.Storage
session session.Session
session sessiontypes.Session
}

func newBenchDB() *benchDB {
Expand Down
1 change: 1 addition & 0 deletions cmd/ddltest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ go_test(
"//pkg/parser/model",
"//pkg/parser/terror",
"//pkg/session",
"//pkg/session/types",
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/sessiontxn",
Expand Down
3 changes: 2 additions & 1 deletion cmd/ddltest/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/sessiontxn"
Expand Down Expand Up @@ -76,7 +77,7 @@ type server struct {
type ddlSuite struct {
store kv.Storage
dom *domain.Domain
s session.Session
s sessiontypes.Session
ctx sessionctx.Context

m sync.Mutex
Expand Down
1 change: 1 addition & 0 deletions pkg/ddl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -276,6 +276,7 @@ go_test(
"//pkg/planner/core",
"//pkg/server",
"//pkg/session",
"//pkg/session/types",
"//pkg/sessionctx",
"//pkg/sessionctx/stmtctx",
"//pkg/sessionctx/variable",
Expand Down
3 changes: 2 additions & 1 deletion pkg/ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessiontxn"
"github.com/pingcap/tidb/pkg/testkit"
Expand Down Expand Up @@ -289,7 +290,7 @@ func TestTwoStates(t *testing.T) {
}

type stateCase struct {
session session.Session
session sessiontypes.Session
rawStmt ast.StmtNode
stmt sqlexec.Statement
expectedExecErr string
Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/ddl_api_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -134,7 +134,7 @@ func TestIsJobRollbackable(t *testing.T) {
}
}

func enQueueDDLJobs(t *testing.T, sess session.Session, txn kv.Transaction, jobType model.ActionType, start, end int) {
func enQueueDDLJobs(t *testing.T, sess sessiontypes.Session, txn kv.Transaction, jobType model.ActionType, start, end int) {
for i := start; i < end; i++ {
job := &model.Job{
ID: int64(i),
Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/stat_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessiontxn"
"github.com/pingcap/tidb/pkg/testkit"
Expand Down Expand Up @@ -148,7 +148,7 @@ func TestGetDDLInfo(t *testing.T) {
tk.MustExec("rollback")
}

func addDDLJobs(sess session.Session, txn kv.Transaction, job *model.Job) error {
func addDDLJobs(sess sessiontypes.Session, txn kv.Transaction, job *model.Job) error {
b, err := job.Encode(true)
if err != nil {
return err
Expand Down
4 changes: 2 additions & 2 deletions pkg/ddl/table_modify_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import (
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/parser/terror"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessiontxn"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util"
Expand Down Expand Up @@ -127,7 +127,7 @@ func TestConcurrentLockTables(t *testing.T) {
tk2.MustExec("unlock tables")
}

func testParallelExecSQL(t *testing.T, store kv.Storage, dom *domain.Domain, sql1, sql2 string, se1, se2 session.Session, f func(t *testing.T, err1, err2 error)) {
func testParallelExecSQL(t *testing.T, store kv.Storage, dom *domain.Domain, sql1, sql2 string, se1, se2 sessiontypes.Session, f func(t *testing.T, err1, err2 error)) {
callback := &callback.TestDDLCallback{}
times := 0
callback.OnJobRunBeforeExported = func(job *model.Job) {
Expand Down
1 change: 1 addition & 0 deletions pkg/ddl/testutil/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ go_library(
"//pkg/kv",
"//pkg/parser/model",
"//pkg/session",
"//pkg/session/types",
"//pkg/sessiontxn",
"//pkg/table",
"//pkg/table/tables",
Expand Down
3 changes: 2 additions & 1 deletion pkg/ddl/testutil/testutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessiontxn"
"github.com/pingcap/tidb/pkg/table"
"github.com/pingcap/tidb/pkg/table/tables"
Expand Down Expand Up @@ -68,7 +69,7 @@ func ExecMultiSQLInGoroutine(s kv.Storage, dbName string, multiSQL []string, don
}

// ExtractAllTableHandles extracts all handles of a given table.
func ExtractAllTableHandles(se session.Session, dbName, tbName string) ([]int64, error) {
func ExtractAllTableHandles(se sessiontypes.Session, dbName, tbName string) ([]int64, error) {
dom := domain.GetDomain(se)
tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(dbName), model.NewCIStr(tbName))
if err != nil {
Expand Down
1 change: 1 addition & 0 deletions pkg/executor/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -401,6 +401,7 @@ go_test(
"//pkg/planner/util",
"//pkg/server",
"//pkg/session",
"//pkg/session/types",
"//pkg/sessionctx",
"//pkg/sessionctx/binloginfo",
"//pkg/sessionctx/stmtctx",
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/inspection_result_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import (
"github.com/pingcap/sysutil"
"github.com/pingcap/tidb/pkg/infoschema"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/types"
Expand Down Expand Up @@ -179,7 +179,7 @@ func TestInspectionResult(t *testing.T) {
}
}

func parseTime(t *testing.T, se session.Session, str string) types.Time {
func parseTime(t *testing.T, se sessiontypes.Session, str string) types.Time {
time, err := types.ParseTime(se.GetSessionVars().StmtCtx.TypeCtx(), str, mysql.TypeDatetime, types.MaxFsp)
require.NoError(t, err)
return time
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/test/issuetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ go_test(
"//pkg/parser/auth",
"//pkg/parser/charset",
"//pkg/parser/mysql",
"//pkg/session",
"//pkg/session/types",
"//pkg/testkit",
"//pkg/util",
"//pkg/util/dbterror/exeerrors",
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/test/issuetest/executor_issue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/auth"
"github.com/pingcap/tidb/pkg/parser/charset"
"github.com/pingcap/tidb/pkg/parser/mysql"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/dbterror/exeerrors"
Expand Down Expand Up @@ -576,7 +576,7 @@ func TestIssue42662(t *testing.T) {
sm := &testkit.MockSessionManager{
PS: []*util.ProcessInfo{tk.Session().ShowProcess()},
}
sm.Conn = make(map[uint64]session.Session)
sm.Conn = make(map[uint64]sessiontypes.Session)
sm.Conn[tk.Session().GetSessionVars().ConnectionID] = tk.Session()
dom.ServerMemoryLimitHandle().SetSessionManager(sm)
go dom.ServerMemoryLimitHandle().Run()
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/update_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import (
"time"

"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -50,7 +50,7 @@ func testUpdatePKLazyCheck(t *testing.T, tk *testkit.TestKit, clusteredIndex var
tk.MustExec("commit")
}

func getPresumeExistsCount(t *testing.T, se session.Session) int {
func getPresumeExistsCount(t *testing.T, se sessiontypes.Session) int {
txn, err := se.Txn(false)
require.NoError(t, err)
buf := txn.GetMemBuffer()
Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -250,6 +250,7 @@ go_test(
"//pkg/planner/property",
"//pkg/planner/util",
"//pkg/session",
"//pkg/session/types",
"//pkg/sessionctx",
"//pkg/sessionctx/variable",
"//pkg/sessiontxn",
Expand Down
3 changes: 2 additions & 1 deletion pkg/planner/core/memtable_predicate_extractor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,14 +32,15 @@ import (
"github.com/pingcap/tidb/pkg/planner"
plannercore "github.com/pingcap/tidb/pkg/planner/core"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/hint"
"github.com/pingcap/tidb/pkg/util/set"
"github.com/stretchr/testify/require"
)

func getLogicalMemTable(t *testing.T, dom *domain.Domain, se session.Session, parser *parser.Parser, sql string) *plannercore.LogicalMemTable {
func getLogicalMemTable(t *testing.T, dom *domain.Domain, se sessiontypes.Session, parser *parser.Parser, sql string) *plannercore.LogicalMemTable {
stmt, err := parser.ParseOneStmt(sql, "", "")
require.NoError(t, err)

Expand Down
1 change: 1 addition & 0 deletions pkg/planner/core/tests/prepare/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ go_test(
"//pkg/parser/auth",
"//pkg/planner/core",
"//pkg/session",
"//pkg/session/types",
"//pkg/sessionctx/variable",
"//pkg/testkit",
"//pkg/testkit/testsetup",
Expand Down
5 changes: 3 additions & 2 deletions pkg/planner/core/tests/prepare/prepare_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/pingcap/tidb/pkg/parser/auth"
"github.com/pingcap/tidb/pkg/planner/core"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/util/hint"
Expand Down Expand Up @@ -635,15 +636,15 @@ func TestPrepareCacheForPartition(t *testing.T) {
}
}

func newSession(t *testing.T, store kv.Storage, dbName string) session.Session {
func newSession(t *testing.T, store kv.Storage, dbName string) sessiontypes.Session {
se, err := session.CreateSession4Test(store)
require.NoError(t, err)
mustExec(t, se, "create database if not exists "+dbName)
mustExec(t, se, "use "+dbName)
return se
}

func mustExec(t *testing.T, se session.Session, sql string) {
func mustExec(t *testing.T, se sessiontypes.Session, sql string) {
_, err := se.Execute(context.Background(), sql)
require.NoError(t, err)
}
Expand Down
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@ go_library(
"//pkg/server/metrics",
"//pkg/session",
"//pkg/session/txninfo",
"//pkg/session/types",
"//pkg/sessionctx",
"//pkg/sessionctx/sessionstates",
"//pkg/sessionctx/stmtctx",
Expand Down
3 changes: 2 additions & 1 deletion pkg/server/driver_tidb.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/pingcap/tidb/pkg/server/internal/column"
"github.com/pingcap/tidb/pkg/server/internal/resultset"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/sessionstates"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
Expand All @@ -55,7 +56,7 @@ func NewTiDBDriver(store kv.Storage) *TiDBDriver {

// TiDBContext implements QueryCtx.
type TiDBContext struct {
session.Session
sessiontypes.Session
stmts map[int]*TiDBStatement
}

Expand Down
1 change: 1 addition & 0 deletions pkg/server/handler/tikvhandler/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ go_library(
"//pkg/server/handler",
"//pkg/session",
"//pkg/session/txninfo",
"//pkg/session/types",
"//pkg/sessionctx",
"//pkg/sessionctx/binloginfo",
"//pkg/sessionctx/variable",
Expand Down
3 changes: 2 additions & 1 deletion pkg/server/handler/tikvhandler/tikv_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ import (
"github.com/pingcap/tidb/pkg/server/handler"
"github.com/pingcap/tidb/pkg/session"
"github.com/pingcap/tidb/pkg/session/txninfo"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/sessionctx/binloginfo"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
Expand Down Expand Up @@ -825,7 +826,7 @@ type SchemaTableStorage struct {
}

func getSchemaTablesStorageInfo(h *SchemaStorageHandler, schema *model.CIStr, table *model.CIStr) (messages []*SchemaTableStorage, err error) {
var s session.Session
var s sessiontypes.Session
if s, err = session.CreateSession(h.Store); err != nil {
return
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/server/rpc_server.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/pingcap/tidb/pkg/privilege"
"github.com/pingcap/tidb/pkg/privilege/privileges"
"github.com/pingcap/tidb/pkg/session"
sessiontypes "github.com/pingcap/tidb/pkg/session/types"
"github.com/pingcap/tidb/pkg/sessionctx/variable"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/logutil"
Expand Down Expand Up @@ -214,7 +215,7 @@ func (s *rpcServer) handleCopRequest(ctx context.Context, req *coprocessor.Reque
return h.HandleRequest(ctx, req)
}

func (s *rpcServer) createSession() (session.Session, error) {
func (s *rpcServer) createSession() (sessiontypes.Session, error) {
se, err := session.CreateSessionWithDomain(s.dom.Store(), s.dom)
if err != nil {
return nil, err
Expand Down
Loading

0 comments on commit b8515ae

Please sign in to comment.