Skip to content

Commit

Permalink
Merge branch 'push-null' of https://github.com/xuyifangreeneyes/tidb
Browse files Browse the repository at this point in the history
…into push-null
  • Loading branch information
xuyifangreeneyes committed Oct 24, 2022
2 parents 626a39c + 47b8ad5 commit 42778ff
Show file tree
Hide file tree
Showing 40 changed files with 10,633 additions and 9,642 deletions.
10 changes: 10 additions & 0 deletions .dockerignore
Original file line number Diff line number Diff line change
@@ -0,0 +1,10 @@
.git/
bazel-bin/
bazel-out/
bazel-tidb/
bazel-testlogs/
bin/
tidb-server/tidb-server
*.test.bin
cmd/
Dockerfile
1 change: 1 addition & 0 deletions .github/licenserc.yml
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ header:
- "docs/"
- "br/"
- ".gitignore"
- ".dockerignore"
- ".gitattributes"
- ".cilinter.yaml"
- ".golangci.yml"
Expand Down
46 changes: 10 additions & 36 deletions Dockerfile
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# Copyright 2019 PingCAP, Inc.
# Copyright 2022 PingCAP, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
Expand All @@ -13,45 +13,19 @@
# limitations under the License.

# Builder image
FROM golang:1.19.1-alpine as builder
FROM alpine:edge as builder

RUN apk add --no-cache \
wget \
make \
git \
gcc \
binutils-gold \
musl-dev
ADD . https://raw.githubusercontent.com/njhallett/apk-fastest-mirror/c4ca44caef3385d830fea34df2dbc2ba4a17e021/apk-fastest-mirror.sh ./proxy
RUN sh ./proxy/apk-fastest-mirror.sh -t 50 && apk add --no-cache git build-base go

RUN wget -O /usr/local/bin/dumb-init https://github.com/Yelp/dumb-init/releases/download/v1.2.2/dumb-init_1.2.2_amd64 \
&& chmod +x /usr/local/bin/dumb-init
COPY . /tidb
ARG GOPROXY
RUN export GOPROXY=${GOPROXY} && cd /tidb && make server

RUN mkdir -p /go/src/github.com/pingcap/tidb
WORKDIR /go/src/github.com/pingcap/tidb
FROM alpine:latest

# Cache dependencies
COPY go.mod .
COPY go.sum .
COPY parser/go.mod parser/go.mod
COPY parser/go.sum parser/go.sum

RUN GO111MODULE=on go mod download

# Build real binaries
COPY . .
RUN make

# Executable image
FROM alpine

RUN apk add --no-cache \
curl

COPY --from=builder /go/src/github.com/pingcap/tidb/bin/tidb-server /tidb-server
COPY --from=builder /usr/local/bin/dumb-init /usr/local/bin/dumb-init
COPY --from=builder /tidb/bin/tidb-server /tidb-server

WORKDIR /

EXPOSE 4000

ENTRYPOINT ["/usr/local/bin/dumb-init", "/tidb-server"]
ENTRYPOINT ["/tidb-server"]
22 changes: 14 additions & 8 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -140,30 +140,30 @@ race: failpoint-enable

server:
ifeq ($(TARGET), "")
CGO_ENABLED=1 $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server tidb-server/main.go
CGO_ENABLED=1 $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server ./tidb-server
else
CGO_ENABLED=1 $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' tidb-server/main.go
CGO_ENABLED=1 $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' ./tidb-server
endif

server_debug:
ifeq ($(TARGET), "")
CGO_ENABLED=1 $(GOBUILD) -gcflags="all=-N -l" $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server-debug tidb-server/main.go
CGO_ENABLED=1 $(GOBUILD) -gcflags="all=-N -l" $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server-debug ./tidb-server
else
CGO_ENABLED=1 $(GOBUILD) -gcflags="all=-N -l" $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' tidb-server/main.go
CGO_ENABLED=1 $(GOBUILD) -gcflags="all=-N -l" $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' ./tidb-server
endif

server_check:
ifeq ($(TARGET), "")
$(GOBUILD) $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' -o bin/tidb-server tidb-server/main.go
$(GOBUILD) $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' -o bin/tidb-server ./tidb-server
else
$(GOBUILD) $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' -o '$(TARGET)' tidb-server/main.go
$(GOBUILD) $(RACE_FLAG) -ldflags '$(CHECK_LDFLAGS)' -o '$(TARGET)' ./tidb-server
endif

linux:
ifeq ($(TARGET), "")
GOOS=linux $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server-linux tidb-server/main.go
GOOS=linux $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server-linux ./tidb-server
else
GOOS=linux $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' tidb-server/main.go
GOOS=linux $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o '$(TARGET)' ./tidb-server
endif

server_coverage:
Expand Down Expand Up @@ -461,3 +461,9 @@ bazel_addindextest: failpoint-enable bazel_ci_prepare

bazel_lint: bazel_prepare
bazel build //... --//build:with_nogo_flag=true

docker:
docker build -t "$(DOCKERPREFIX)tidb:latest" --build-arg 'GOPROXY=$(shell go env GOPROXY),' -f Dockerfile .

docker-test:
docker buildx build --platform linux/amd64,linux/arm64 --push -t "$(DOCKERPREFIX)tidb:latest" --build-arg 'GOPROXY=$(shell go env GOPROXY),' -f Dockerfile .
3 changes: 3 additions & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -534,6 +534,9 @@ type Security struct {
ClusterSSLCert string `toml:"cluster-ssl-cert" json:"cluster-ssl-cert"`
ClusterSSLKey string `toml:"cluster-ssl-key" json:"cluster-ssl-key"`
ClusterVerifyCN []string `toml:"cluster-verify-cn" json:"cluster-verify-cn"`
// Used for auth plugin `tidb_session_token`.
SessionTokenSigningCert string `toml:"session-token-signing-cert" json:"session-token-signing-cert"`
SessionTokenSigningKey string `toml:"session-token-signing-key" json:"session-token-signing-key"`
// If set to "plaintext", the spilled files will not be encrypted.
SpilledFileEncryptionMethod string `toml:"spilled-file-encryption-method" json:"spilled-file-encryption-method"`
// EnableSEM prevents SUPER users from having full access.
Expand Down
2 changes: 1 addition & 1 deletion docs/design/2022-07-20-session-manager.md
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ When the Session Manager migrates a session, it needs to authenticate with the n

It's unsafe to save user passwords in the Session Manager, so we use a token-based authentication:

1. The administrator places a self-signed certificate on each TiDB server. The certificate and key paths are defined by global variables `tidb_auth_signing_cert` and `tidb_auth_signing_key`. The certificates on all the servers are the same so that a message encrypted by one server can be decrypted by another.
1. The administrator places a self-signed certificate on each TiDB server. The certificate and key paths are defined by configurations `security.session-token-signing-cert` and `security.session-token-signing-key`. The certificates on all the servers are the same so that a message encrypted by one server can be decrypted by another.
2. When the Session Manager is going to migrate a session from one TiDB instance to another, it queries the session token. The session token is composed by the username, token expiration time, and a signature. The signature is signed with the private key of the certificate.
3. The Session Manager then authenticates with the new TiDB server with a new auth-plugin. The session token acts as the password. The new server checks the username, token expiration time, and the signature. The signature should be verified by the public key.

Expand Down
5 changes: 4 additions & 1 deletion domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -1892,7 +1892,10 @@ func (do *Domain) NotifyUpdateSysVarCache() {
}

// LoadSigningCertLoop loads the signing cert periodically to make sure it's fresh new.
func (do *Domain) LoadSigningCertLoop() {
func (do *Domain) LoadSigningCertLoop(signingCert, signingKey string) {
sessionstates.SetCertPath(signingCert)
sessionstates.SetKeyPath(signingKey)

do.wg.Add(1)
go func() {
defer func() {
Expand Down
18 changes: 9 additions & 9 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import (
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/privilege"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/sessiontxn"
"github.com/pingcap/tidb/sessiontxn/staleread"
Expand Down Expand Up @@ -173,8 +172,14 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {
err = e.executeRecoverTable(x)
case *ast.FlashBackTableStmt:
err = e.executeFlashbackTable(x)
case *ast.FlashBackClusterStmt:
err = e.executeFlashBackCluster(ctx, x)
case *ast.FlashBackToTimestampStmt:
if len(x.Tables) != 0 {
err = dbterror.ErrGeneralUnsupportedDDL.GenWithStack("Unsupported FLASHBACK table TO TIMESTAMP")
} else if x.DBName.O != "" {
err = dbterror.ErrGeneralUnsupportedDDL.GenWithStack("Unsupported FLASHBACK database TO TIMESTAMP")
} else {
err = e.executeFlashBackCluster(x)
}
case *ast.RenameTableStmt:
err = e.executeRenameTable(x)
case *ast.TruncateTableStmt:
Expand Down Expand Up @@ -525,12 +530,7 @@ func (e *DDLExec) getRecoverTableByTableName(tableName *ast.TableName) (*model.J
return jobInfo, tableInfo, nil
}

func (e *DDLExec) executeFlashBackCluster(ctx context.Context, s *ast.FlashBackClusterStmt) error {
checker := privilege.GetPrivilegeManager(e.ctx)
if !checker.RequestVerification(e.ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.SuperPriv) {
return core.ErrSpecificAccessDenied.GenWithStackByArgs("SUPER")
}

func (e *DDLExec) executeFlashBackCluster(s *ast.FlashBackToTimestampStmt) error {
tiFlashInfo, err := getTiFlashStores(e.ctx)
if err != nil {
return err
Expand Down
20 changes: 15 additions & 5 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -1694,6 +1694,7 @@ func keyColumnUsageInTable(schema *model.DBInfo, table *model.TableInfo) [][]typ
}

func (e *memtableRetriever) setDataForTiKVRegionStatus(ctx sessionctx.Context) (err error) {
var extractorTableIDs []int64
tikvStore, ok := ctx.GetStore().(helper.Storage)
if !ok {
return errors.New("Information about TiKV region status can be gotten only when the storage is TiKV")
Expand All @@ -1708,9 +1709,13 @@ func (e *memtableRetriever) setDataForTiKVRegionStatus(ctx sessionctx.Context) (
if e.extractor != nil {
extractor, ok := e.extractor.(*plannercore.TiKVRegionStatusExtractor)
if ok && len(extractor.GetTablesID()) > 0 {
for _, tableID := range extractor.GetTablesID() {
extractorTableIDs = extractor.GetTablesID()
for _, tableID := range extractorTableIDs {
regionsInfo, err := e.getRegionsInfoForTable(tikvHelper, is, tableID)
if err != nil {
if errors.ErrorEqual(err, infoschema.ErrTableExists) {
continue
}
return err
}
allRegionsInfo = allRegionsInfo.Merge(regionsInfo)
Expand All @@ -1726,12 +1731,17 @@ func (e *memtableRetriever) setDataForTiKVRegionStatus(ctx sessionctx.Context) (
}
tableInfos := tikvHelper.GetRegionsTableInfo(allRegionsInfo, is.AllSchemas())
for i := range allRegionsInfo.Regions {
tableList := tableInfos[allRegionsInfo.Regions[i].ID]
if len(tableList) == 0 {
regionTableList := tableInfos[allRegionsInfo.Regions[i].ID]
if len(regionTableList) == 0 {
e.setNewTiKVRegionStatusCol(&allRegionsInfo.Regions[i], nil)
}
for j := range tableList {
e.setNewTiKVRegionStatusCol(&allRegionsInfo.Regions[i], &tableList[j])
for j := range regionTableList {
if len(extractorTableIDs) == 0 {
e.setNewTiKVRegionStatusCol(&allRegionsInfo.Regions[i], &regionTableList[j])
}
if slices.Contains(extractorTableIDs, regionTableList[j].Table.ID) {
e.setNewTiKVRegionStatusCol(&allRegionsInfo.Regions[i], &regionTableList[j])
}
}
}
return nil
Expand Down
3 changes: 1 addition & 2 deletions executor/recover_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ import (
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util/dbterror"
Expand Down Expand Up @@ -329,7 +328,7 @@ func TestRecoverClusterMeetError(t *testing.T) {
tk.MustExec("CREATE USER 'testflashback'@'localhost';")
newTk := testkit.NewTestKit(t, store)
require.NoError(t, newTk.Session().Auth(&auth.UserIdentity{Username: "testflashback", Hostname: "localhost"}, nil, nil))
newTk.MustGetErrCode(fmt.Sprintf("flashback cluster to timestamp '%s'", time.Now().Add(0-30*time.Second)), int(core.ErrSpecificAccessDenied.Code()))
newTk.MustGetErrCode(fmt.Sprintf("flashback cluster to timestamp '%s'", time.Now().Add(0-30*time.Second)), errno.ErrPrivilegeCheckFail)
tk.MustExec("drop user 'testflashback'@'localhost';")

// Flashback failed because of ddl history.
Expand Down
10 changes: 9 additions & 1 deletion executor/simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -1554,6 +1554,14 @@ func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error
}

func (e *SimpleExec) executeKillStmt(ctx context.Context, s *ast.KillStmt) error {
if x, ok := s.Expr.(*ast.FuncCallExpr); ok {
if x.FnName.L == ast.ConnectionID {
sm := e.ctx.GetSessionManager()
sm.Kill(e.ctx.GetSessionVars().ConnectionID, s.Query)
return nil
}
return errors.New("Invalid operation. Please use 'KILL TIDB [CONNECTION | QUERY] [connectionID | CONNECTION_ID()]' instead")
}
if !config.GetGlobalConfig().EnableGlobalKill {
conf := config.GetGlobalConfig()
if s.TiDBExtension || conf.CompatibleKillQuery {
Expand All @@ -1563,7 +1571,7 @@ func (e *SimpleExec) executeKillStmt(ctx context.Context, s *ast.KillStmt) error
}
sm.Kill(s.ConnectionID, s.Query)
} else {
err := errors.New("Invalid operation. Please use 'KILL TIDB [CONNECTION | QUERY] connectionID' instead")
err := errors.New("Invalid operation. Please use 'KILL TIDB [CONNECTION | QUERY] [connectionID | CONNECTION_ID()]' instead")
e.ctx.GetSessionVars().StmtCtx.AppendWarning(err)
}
return nil
Expand Down
3 changes: 2 additions & 1 deletion executor/simple_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func TestKillStmt(t *testing.T) {
tk.MustExec("use test")
tk.MustExec(fmt.Sprintf("kill %d", connID))
result := tk.MustQuery("show warnings")
result.Check(testkit.Rows("Warning 1105 Invalid operation. Please use 'KILL TIDB [CONNECTION | QUERY] connectionID' instead"))
result.Check(testkit.Rows("Warning 1105 Invalid operation. Please use 'KILL TIDB [CONNECTION | QUERY] [connectionID | CONNECTION_ID()]' instead"))

newCfg2 := *originCfg
newCfg2.EnableGlobalKill = true
Expand All @@ -79,6 +79,7 @@ func TestKillStmt(t *testing.T) {
result = tk.MustQuery("show warnings")
result.Check(testkit.Rows())

tk.MustExecToErr("kill rand()", "Invalid operation. Please use 'KILL TIDB [CONNECTION | QUERY] [connectionID | CONNECTION_ID()]' instead")
// remote kill is tested in `tests/globalkilltest`
}

Expand Down
3 changes: 3 additions & 0 deletions expression/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ go_library(
"explain.go",
"expr_to_pb.go",
"expression.go",
"extension.go",
"function_traits.go",
"helper.go",
"partition_pruner.go",
Expand All @@ -66,6 +67,7 @@ go_library(
deps = [
"//config",
"//errno",
"//extension",
"//kv",
"//parser",
"//parser/ast",
Expand Down Expand Up @@ -97,6 +99,7 @@ go_library(
"//util/parser",
"//util/plancodec",
"//util/printer",
"//util/sem",
"//util/set",
"//util/size",
"//util/sqlexec",
Expand Down
7 changes: 7 additions & 0 deletions expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -944,6 +944,13 @@ func GetBuiltinList() []string {
}
res = append(res, funcName)
}

extensionFuncs.Range(func(key, _ any) bool {
funcName := key.(string)
res = append(res, funcName)
return true
})

slices.Sort(res)
return res
}
Expand Down
Loading

0 comments on commit 42778ff

Please sign in to comment.