Skip to content

Commit

Permalink
Merge branch 'master' of https://github.com/pingcap/tidb into put_ser…
Browse files Browse the repository at this point in the history
…ver_info_to_pd
  • Loading branch information
crazycs520 committed Aug 15, 2018
2 parents 93c76ce + 5404e2e commit f8b7b98
Show file tree
Hide file tree
Showing 13 changed files with 377 additions and 9 deletions.
15 changes: 14 additions & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -34,12 +34,18 @@ LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBGitHash=$(shell git rev-
LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBGitBranch=$(shell git rev-parse --abbrev-ref HEAD)"
LDFLAGS += -X "github.com/pingcap/tidb/util/printer.GoVersion=$(shell go version)"

TEST_LDFLAGS = -X "github.com/pingcap/tidb/config.checkBeforeDropLDFlag=1"

CHECK_LDFLAGS += $(LDFLAGS) ${TEST_LDFLAGS}

TARGET = ""

.PHONY: all build update parser clean todo test gotest interpreter server dev benchkv benchraw check parserlib checklist

default: server buildsucc

server-admin-check: server_check buildsucc

buildsucc:
@echo Build TiDB Server successfully!

Expand Down Expand Up @@ -141,7 +147,7 @@ ifeq ("$(TRAVIS_COVERAGE)", "1")
else
@echo "Running in native mode."
@export log_level=error; \
$(GOTEST) -cover $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; }
$(GOTEST) -ldflags '$(TEST_LDFLAGS)' -cover $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; }
endif
@$(GOFAIL_DISABLE)

Expand Down Expand Up @@ -178,6 +184,13 @@ else
$(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS)' -o '$(TARGET)' tidb-server/main.go
endif

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

benchkv:
$(GOBUILD) -ldflags '$(LDFLAGS)' -o bin/benchkv cmd/benchkv/main.go

Expand Down
10 changes: 10 additions & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,10 @@ var (
"mocktikv": true,
"tikv": true,
}
// checkTableBeforeDrop enable to execute `admin check table` before `drop table`.
CheckTableBeforeDrop = false
// checkBeforeDropLDFlag is a go build flag.
checkBeforeDropLDFlag = "None"
)

// Config contains configuration options.
Expand Down Expand Up @@ -374,6 +378,12 @@ func (t *OpenTracing) ToTracingConfig() *tracing.Configuration {
return ret
}

func init() {
if checkBeforeDropLDFlag == "1" {
CheckTableBeforeDrop = true
}
}

// The following constants represents the valid action configurations for OOMAction.
// NOTE: Althrough the values is case insensitiv, we should use lower-case
// strings because the configuration value will be transformed to lower-case
Expand Down
5 changes: 5 additions & 0 deletions executor/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -471,6 +471,11 @@ func (s *testSuite) TestAdminCheckTable(c *C) {
INDEX indexIDname (ID(8),name(8)));`)
tk.MustExec(`INSERT INTO t VALUES ('keyword','urlprefix','text/ /text');`)
tk.MustExec(`admin check table t;`)

tk.MustExec("use mysql")
tk.MustExec(`admin check table test.t;`)
_, err := tk.Exec("admin check table t")
c.Assert(err, NotNil)
}

func (s *testSuite) TestAdminCheckPrimaryIndex(c *C) {
Expand Down
13 changes: 13 additions & 0 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,17 +14,21 @@
package executor

import (
"fmt"
"strings"

"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/sqlexec"
log "github.com/sirupsen/logrus"
"golang.org/x/net/context"
)

Expand Down Expand Up @@ -194,6 +198,15 @@ func (e *DDLExec) executeDropTable(s *ast.DropTableStmt) error {
return errors.Trace(err)
}

if config.CheckTableBeforeDrop {
log.Warnf("admin check table `%s`.`%s` before drop.", fullti.Schema.O, fullti.Name.O)
sql := fmt.Sprintf("admin check table `%s`.`%s`", fullti.Schema.O, fullti.Name.O)
_, _, err = e.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(e.ctx, sql)
if err != nil {
return errors.Trace(err)
}
}

err = domain.GetDomain(e.ctx).DDL().DropTable(e.ctx, fullti)
if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableNotExists.Equal(err) {
notExistTables = append(notExistTables, fullti.String())
Expand Down
2 changes: 1 addition & 1 deletion executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -369,8 +369,8 @@ func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error {
return nil
}
defer func() { e.done = true }()
dbName := model.NewCIStr(e.ctx.GetSessionVars().CurrentDB)
for _, t := range e.tables {
dbName := t.DBInfo.Name
tb, err := e.is.TableByName(dbName, t.Name)
if err != nil {
return errors.Trace(err)
Expand Down
12 changes: 10 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -233,9 +233,17 @@ func (s *testSuite) TestAdmin(c *C) {
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
r, err = tk.Exec("admin check table admin_test")
c.Assert(err, NotNil)
r, err_admin := tk.Exec("admin check table admin_test")
c.Assert(err_admin, NotNil)

if config.CheckTableBeforeDrop {
r, err = tk.Exec("drop table admin_test")
c.Assert(err.Error(), Equals, err_admin.Error())

// Drop inconsistency index.
tk.MustExec("alter table admin_test drop index c1")
tk.MustExec("admin check table admin_test")
}
// checksum table test
tk.MustExec("create table checksum_with_index (id int, count int, PRIMARY KEY(id), KEY(count))")
tk.MustExec("create table checksum_without_index (id int, count int, PRIMARY KEY(id))")
Expand Down
2 changes: 2 additions & 0 deletions store/mockstore/mocktikv/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -437,10 +437,12 @@ type MVCCStore interface {
// RawKV is a key-value storage. MVCCStore can be implemented upon it with timestamp encoded into key.
type RawKV interface {
RawGet(key []byte) []byte
RawBatchGet(keys [][]byte) [][]byte
RawScan(startKey, endKey []byte, limit int) []Pair
RawPut(key, value []byte)
RawBatchPut(keys, values [][]byte)
RawDelete(key []byte)
RawBatchDelete(keys [][]byte)
RawDeleteRange(startKey, endKey []byte)
}

Expand Down
26 changes: 26 additions & 0 deletions store/mockstore/mocktikv/mvcc_leveldb.go
Original file line number Diff line number Diff line change
Expand Up @@ -919,6 +919,20 @@ func (mvcc *MVCCLevelDB) RawGet(key []byte) []byte {
return ret
}

// RawBatchGet implements the RawKV interface.
func (mvcc *MVCCLevelDB) RawBatchGet(keys [][]byte) [][]byte {
mvcc.mu.Lock()
defer mvcc.mu.Unlock()

var values [][]byte
for _, key := range keys {
value, err := mvcc.db.Get(key, nil)
terror.Log(err)
values = append(values, value)
}
return values
}

// RawDelete implements the RawKV interface.
func (mvcc *MVCCLevelDB) RawDelete(key []byte) {
mvcc.mu.Lock()
Expand All @@ -927,6 +941,18 @@ func (mvcc *MVCCLevelDB) RawDelete(key []byte) {
terror.Log(mvcc.db.Delete(key, nil))
}

// RawBatchDelete implements the RawKV interface.
func (mvcc *MVCCLevelDB) RawBatchDelete(keys [][]byte) {
mvcc.mu.Lock()
defer mvcc.mu.Unlock()

batch := &leveldb.Batch{}
for _, key := range keys {
batch.Delete(key)
}
terror.Log(mvcc.db.Write(batch, nil))
}

// RawScan implements the RawKV interface.
func (mvcc *MVCCLevelDB) RawScan(startKey, endKey []byte, limit int) []Pair {
mvcc.mu.Lock()
Expand Down
47 changes: 47 additions & 0 deletions store/mockstore/mocktikv/rpc.go
Original file line number Diff line number Diff line change
Expand Up @@ -377,6 +377,29 @@ func (h *rpcHandler) handleKvRawGet(req *kvrpcpb.RawGetRequest) *kvrpcpb.RawGetR
}
}

func (h *rpcHandler) handleKvRawBatchGet(req *kvrpcpb.RawBatchGetRequest) *kvrpcpb.RawBatchGetResponse {
rawKV, ok := h.mvccStore.(RawKV)
if !ok {
// TODO should we add error ?
return &kvrpcpb.RawBatchGetResponse{
RegionError: &errorpb.Error{
Message: "not implemented",
},
}
}
values := rawKV.RawBatchGet(req.Keys)
kvPairs := make([]*kvrpcpb.KvPair, len(values))
for i, key := range req.Keys {
kvPairs[i] = &kvrpcpb.KvPair{
Key: key,
Value: values[i],
}
}
return &kvrpcpb.RawBatchGetResponse{
Pairs: kvPairs,
}
}

func (h *rpcHandler) handleKvRawPut(req *kvrpcpb.RawPutRequest) *kvrpcpb.RawPutResponse {
rawKV, ok := h.mvccStore.(RawKV)
if !ok {
Expand Down Expand Up @@ -416,6 +439,17 @@ func (h *rpcHandler) handleKvRawDelete(req *kvrpcpb.RawDeleteRequest) *kvrpcpb.R
return &kvrpcpb.RawDeleteResponse{}
}

func (h *rpcHandler) handleKvRawBatchDelete(req *kvrpcpb.RawBatchDeleteRequest) *kvrpcpb.RawBatchDeleteResponse {
rawKV, ok := h.mvccStore.(RawKV)
if !ok {
return &kvrpcpb.RawBatchDeleteResponse{
Error: "not implemented",
}
}
rawKV.RawBatchDelete(req.Keys)
return &kvrpcpb.RawBatchDeleteResponse{}
}

func (h *rpcHandler) handleKvRawDeleteRange(req *kvrpcpb.RawDeleteRangeRequest) *kvrpcpb.RawDeleteRangeResponse {
rawKV, ok := h.mvccStore.(RawKV)
if !ok {
Expand Down Expand Up @@ -625,6 +659,13 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R
return resp, nil
}
resp.RawGet = handler.handleKvRawGet(r)
case tikvrpc.CmdRawBatchGet:
r := req.RawBatchGet
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
resp.RawBatchGet = &kvrpcpb.RawBatchGetResponse{RegionError: err}
return resp, nil
}
resp.RawBatchGet = handler.handleKvRawBatchGet(r)
case tikvrpc.CmdRawPut:
r := req.RawPut
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
Expand All @@ -646,6 +687,12 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R
return resp, nil
}
resp.RawDelete = handler.handleKvRawDelete(r)
case tikvrpc.CmdRawBatchDelete:
r := req.RawBatchDelete
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
resp.RawBatchDelete = &kvrpcpb.RawBatchDeleteResponse{RegionError: err}
}
resp.RawBatchDelete = handler.handleKvRawBatchDelete(r)
case tikvrpc.CmdRawDeleteRange:
r := req.RawDeleteRange
if err := handler.checkRequest(reqCtx, r.Size()); err != nil {
Expand Down
Loading

0 comments on commit f8b7b98

Please sign in to comment.