Skip to content

Commit

Permalink
server: refactor http handler (#45627)
Browse files Browse the repository at this point in the history
ref #44940
  • Loading branch information
hawkingrei committed Aug 1, 2023
1 parent fc906e7 commit 1bf82bb
Show file tree
Hide file tree
Showing 43 changed files with 9,452 additions and 8,567 deletions.
11 changes: 10 additions & 1 deletion build/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,7 @@
"/rules_go_work-*": "ignore generated code",
"GOROOT/": "ignore code",
"/parser/": "ignore code",
"server/internal/testserverclient/server_client.go": "ignore code",
".*_/testmain\\.go$": "ignore code"
}
},
Expand Down Expand Up @@ -288,6 +289,7 @@
"tools/check/xprog.go:": "ignore tools/check code",
"cmd/explaintest/main.go": "ignore cmd/explaintest code",
"GOROOT/": "ignore code",
"server/internal/testserverclient/server_client.go": "ignore server_client code",
".*_generated\\.go$": "ignore generated code"
}
},
Expand Down Expand Up @@ -513,6 +515,7 @@
"server/conn_stmt.go": "server/conn_stmt.go",
"server/conn_test.go": "server/conn_test.go",
"server/internal": "server/internal",
"server/handle": "server/handle",
"distsql/": "ignore distsql code",
"disttask": "disttask code",
"dumpling/export": "dumpling/export code",
Expand Down Expand Up @@ -683,11 +686,17 @@
"external/": "no need to vet third party code",
".*_generated\\.go$": "ignore generated code",
"server/tidb_test.go": "ignore test code",
"server/tests/tidb_test.go": "ignore test code",
"server/tests/tidb_serial_test.go": "ignore test code",
"server/tidb_serial_test.go": "ignore test code",
"server/statistics_handler_test.go": "ignore test code",
"server/handler/optimizor/optimize_trace_test.go": "ignore test code",
"server/handler/optimizor/plan_replayer_test.go": "ignore test code",
"server/handler/optimizor/statistics_handler_test.go": "ignore test code",
"server/server_test.go": "ignore test code",
"server/optimize_trace_test.go": "ignore test code",
"server/plan_replayer_test.go": "ignore test code"
"server/plan_replayer_test.go": "ignore test code",
"server/internal/testserverclient/server_client.go": "ignore test code"
}
},
"S1000": {
Expand Down
72 changes: 5 additions & 67 deletions server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -12,20 +12,16 @@ go_library(
"http_handler.go",
"http_status.go",
"mock_conn.go",
"optimize_trace.go",
"plan_replayer.go",
"rpc_server.go",
"server.go",
"stat.go",
"statistics_handler.go",
"tokenlimiter.go",
],
importpath = "github.com/pingcap/tidb/server",
visibility = ["//visibility:public"],
deps = [
"//autoid_service",
"//config",
"//ddl",
"//domain",
"//domain/infosync",
"//errno",
Expand All @@ -35,13 +31,11 @@ go_library(
"//extension",
"//infoschema",
"//kv",
"//meta",
"//metrics",
"//parser",
"//parser/ast",
"//parser/auth",
"//parser/charset",
"//parser/model",
"//parser/mysql",
"//parser/terror",
"//planner/core",
Expand All @@ -51,6 +45,10 @@ go_library(
"//privilege/privileges",
"//privilege/privileges/ldap",
"//server/err",
"//server/handler",
"//server/handler/extactorhandler",
"//server/handler/optimizor",
"//server/handler/tikvhandler",
"//server/handler/ttlhandler",
"//server/internal",
"//server/internal/column",
Expand All @@ -63,37 +61,28 @@ go_library(
"//session",
"//session/txninfo",
"//sessionctx",
"//sessionctx/binloginfo",
"//sessionctx/sessionstates",
"//sessionctx/stmtctx",
"//sessionctx/variable",
"//sessiontxn",
"//statistics/handle",
"//store",
"//store/driver/error",
"//store/gcworker",
"//store/helper",
"//table",
"//table/tables",
"//tablecodec",
"//types",
"//util",
"//util/arena",
"//util/chunk",
"//util/codec",
"//util/cpuprofile",
"//util/dbterror/exeerrors",
"//util/deadlockhistory",
"//util/execdetails",
"//util/fastrand",
"//util/gcutil",
"//util/hack",
"//util/intest",
"//util/logutil",
"//util/memory",
"//util/pdapi",
"//util/printer",
"//util/replayer",
"//util/sqlexec",
"//util/sys/linux",
"//util/timeutil",
Expand All @@ -104,27 +93,21 @@ go_library(
"//util/tracing",
"//util/versioninfo",
"@com_github_blacktear23_go_proxyprotocol//:go-proxyprotocol",
"@com_github_burntsushi_toml//:toml",
"@com_github_gorilla_mux//:mux",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_pingcap_fn//:fn",
"@com_github_pingcap_kvproto//pkg/autoid",
"@com_github_pingcap_kvproto//pkg/coprocessor",
"@com_github_pingcap_kvproto//pkg/diagnosticspb",
"@com_github_pingcap_kvproto//pkg/kvrpcpb",
"@com_github_pingcap_kvproto//pkg/metapb",
"@com_github_pingcap_kvproto//pkg/mpp",
"@com_github_pingcap_kvproto//pkg/tikvpb",
"@com_github_pingcap_log//:log",
"@com_github_pingcap_sysutil//:sysutil",
"@com_github_prometheus_client_golang//prometheus",
"@com_github_prometheus_client_golang//prometheus/promhttp",
"@com_github_soheilhy_cmux//:cmux",
"@com_github_stretchr_testify//require",
"@com_github_tiancaiamao_appdash//traceapp",
"@com_github_tikv_client_go_v2//oracle",
"@com_github_tikv_client_go_v2//tikv",
"@com_github_tikv_client_go_v2//util",
"@com_sourcegraph_sourcegraph_appdash_data//:appdash-data",
"@org_golang_google_grpc//:grpc",
Expand All @@ -143,106 +126,61 @@ go_test(
"conn_stmt_test.go",
"conn_test.go",
"driver_tidb_test.go",
"extract_test.go",
"http_handler_serial_test.go",
"http_handler_test.go",
"main_test.go",
"mock_conn_test.go",
"optimize_trace_test.go",
"plan_replayer_test.go",
"server_test.go",
"stat_test.go",
"statistics_handler_test.go",
"tidb_library_test.go",
"tidb_serial_test.go",
"tidb_test.go",
],
data = glob(["testdata/**"]),
embed = [":server"],
flaky = True,
shard_count = 50,
shard_count = 46,
deps = [
"//config",
"//ddl",
"//ddl/util",
"//domain",
"//domain/infosync",
"//errno",
"//extension",
"//infoschema",
"//keyspace",
"//kv",
"//meta",
"//metrics",
"//parser",
"//parser/ast",
"//parser/auth",
"//parser/charset",
"//parser/model",
"//parser/mysql",
"//planner/core",
"//server/internal",
"//server/internal/column",
"//server/internal/handshake",
"//server/internal/parse",
"//server/internal/resultset",
"//server/internal/testutil",
"//server/internal/util",
"//session",
"//sessionctx",
"//sessionctx/binloginfo",
"//sessionctx/stmtctx",
"//sessionctx/variable",
"//sessiontxn",
"//statistics/handle",
"//store/helper",
"//store/mockstore",
"//store/mockstore/unistore",
"//tablecodec",
"//testkit",
"//testkit/external",
"//testkit/testdata",
"//testkit/testenv",
"//testkit/testmain",
"//testkit/testsetup",
"//types",
"//util",
"//util/arena",
"//util/chunk",
"//util/codec",
"//util/cpuprofile",
"//util/dbterror/exeerrors",
"//util/deadlockhistory",
"//util/plancodec",
"//util/replayer",
"//util/resourcegrouptag",
"//util/rowcodec",
"//util/stmtsummary/v2:stmtsummary",
"//util/syncutil",
"//util/topsql",
"//util/topsql/collector",
"//util/topsql/collector/mock",
"//util/topsql/state",
"//util/topsql/stmtstats",
"//util/versioninfo",
"@com_github_burntsushi_toml//:toml",
"@com_github_docker_go_units//:go-units",
"@com_github_go_sql_driver_mysql//:mysql",
"@com_github_gorilla_mux//:mux",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_pingcap_kvproto//pkg/kvrpcpb",
"@com_github_pingcap_kvproto//pkg/metapb",
"@com_github_pingcap_log//:log",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//error",
"@com_github_tikv_client_go_v2//oracle",
"@com_github_tikv_client_go_v2//testutils",
"@com_github_tikv_client_go_v2//tikv",
"@com_github_tikv_client_go_v2//tikvrpc",
"@io_opencensus_go//stats/view",
"@org_golang_x_exp//slices",
"@org_uber_go_goleak//:goleak",
"@org_uber_go_zap//:zap",
],
)
13 changes: 7 additions & 6 deletions server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ import (
"github.com/pingcap/tidb/privilege/conn"
"github.com/pingcap/tidb/privilege/privileges/ldap"
servererr "github.com/pingcap/tidb/server/err"
"github.com/pingcap/tidb/server/handler/tikvhandler"
"github.com/pingcap/tidb/server/internal"
"github.com/pingcap/tidb/server/internal/column"
"github.com/pingcap/tidb/server/internal/dump"
Expand Down Expand Up @@ -182,7 +183,7 @@ func (cc *clientConn) getCtx() *TiDBContext {
return cc.ctx.TiDBContext
}

func (cc *clientConn) setCtx(ctx *TiDBContext) {
func (cc *clientConn) SetCtx(ctx *TiDBContext) {
cc.ctx.Lock()
cc.ctx.TiDBContext = ctx
cc.ctx.Unlock()
Expand Down Expand Up @@ -417,7 +418,7 @@ func (cc *clientConn) writeInitialHandshake(ctx context.Context) error {
if err = cc.ctx.Close(); err != nil {
return err
}
cc.setCtx(nil)
cc.SetCtx(nil)

data = append(data, 0)
if err = cc.writePacket(data); err != nil {
Expand Down Expand Up @@ -704,7 +705,7 @@ func (cc *clientConn) openSession() error {
if err != nil {
return err
}
cc.setCtx(ctx)
cc.SetCtx(ctx)

err = cc.server.checkConnectionCount()
if err != nil {
Expand Down Expand Up @@ -1293,7 +1294,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error {
data = data[:len(data)-1]
dataStr = string(hack.String(data))
}
return cc.handleStmtPrepare(ctx, dataStr)
return cc.HandleStmtPrepare(ctx, dataStr)
case mysql.ComStmtExecute:
return cc.handleStmtExecute(ctx, data)
case mysql.ComStmtSendLongData:
Expand All @@ -1318,7 +1319,7 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error {
func (cc *clientConn) writeStats(ctx context.Context) error {
var err error
var uptime int64
info := serverInfo{}
info := tikvhandler.ServerInfo{}
info.ServerInfo, err = infosync.GetServerInfo()
if err != nil {
logutil.BgLogger().Error("Failed to get ServerInfo for uptime status", zap.Error(err))
Expand Down Expand Up @@ -2436,7 +2437,7 @@ func (cc *clientConn) handleResetConnection(ctx context.Context) error {
if err != nil {
return err
}
cc.setCtx(tidbCtx)
cc.SetCtx(tidbCtx)
if !cc.ctx.AuthWithoutVerification(user) {
return errors.New("Could not reset connection")
}
Expand Down
2 changes: 1 addition & 1 deletion server/conn_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ import (
"go.uber.org/zap"
)

func (cc *clientConn) handleStmtPrepare(ctx context.Context, sql string) error {
func (cc *clientConn) HandleStmtPrepare(ctx context.Context, sql string) error {
stmt, columns, params, err := cc.ctx.Prepare(sql)
if err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion server/conn_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -398,7 +398,7 @@ func getExpectOutput(t *testing.T, originalConn *mockConn, writeFn func(conn *cl
pkt: internal.NewPacketIOForTest(bufio.NewWriter(buf)),
}
conn.pkt.SetSequence(originalConn.pkt.Sequence())
conn.setCtx(originalConn.getCtx())
conn.SetCtx(originalConn.getCtx())
writeFn(conn)
require.NoError(t, conn.flush(context.Background()))

Expand Down

0 comments on commit 1bf82bb

Please sign in to comment.