Skip to content

Commit

Permalink
Merge branch 'master' into fix_bug_8700
Browse files Browse the repository at this point in the history
  • Loading branch information
zz-jason authored Jan 9, 2019
2 parents 7b0e53d + bb2d56a commit bad812f
Show file tree
Hide file tree
Showing 50 changed files with 922 additions and 303 deletions.
2 changes: 1 addition & 1 deletion CONTRIBUTING.md
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ A Contributor refers to the person who contributes to the following projects:

## How to become a TiDB Contributor?

If a PR (Pull Request) submitted to the TiDB / TiKV / TiSpark / PD / DocsDocs-cn projects by you is approved and merged, then you become a TiDB Contributor.
If a PR (Pull Request) submitted to the TiDB/TiKV/TiSpark/PD/Docs/Docs-cn projects by you is approved and merged, then you become a TiDB Contributor.

You are also encouraged to participate in the projects in the following ways:
- Actively answer technical questions asked by community users.
Expand Down
43 changes: 18 additions & 25 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@ PACKAGES := $$($(PACKAGE_LIST))
PACKAGE_DIRECTORIES := $(PACKAGE_LIST) | sed 's|github.com/pingcap/$(PROJECT)/||'
FILES := $$(find $$($(PACKAGE_DIRECTORIES)) -name "*.go")

GOFAIL_ENABLE := $$(find $$PWD/ -type d | grep -vE "(\.git|tools)" | xargs gofail enable)
GOFAIL_DISABLE := $$(find $$PWD/ -type d | grep -vE "(\.git|tools)" | xargs gofail disable)
GOFAIL_ENABLE := $$(find $$PWD/ -type d | grep -vE "(\.git|tools)" | xargs tools/bin/gofail enable)
GOFAIL_DISABLE := $$(find $$PWD/ -type d | grep -vE "(\.git|tools)" | xargs tools/bin/gofail disable)

LDFLAGS += -X "github.com/pingcap/parser/mysql.TiDBReleaseVersion=$(shell git describe --tags --dirty)"
LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBBuildTS=$(shell date -u '+%Y-%m-%d %I:%M:%S')"
Expand Down Expand Up @@ -123,11 +123,7 @@ ifeq ("$(TRAVIS_COVERAGE)", "1")
bash <(curl -s https://codecov.io/bash)
endif

gotest:
@rm -rf $GOPATH/bin/gofail
$(GO) get github.com/pingcap/gofail
@which gofail
@$(GOFAIL_ENABLE)
gotest: gofail-enable
ifeq ("$(TRAVIS_COVERAGE)", "1")
@echo "Running in TRAVIS_COVERAGE mode."
@export log_level=error; \
Expand All @@ -140,23 +136,17 @@ else
endif
@$(GOFAIL_DISABLE)

race:
$(GO) get github.com/pingcap/gofail
@$(GOFAIL_ENABLE)
race: gofail-enable
@export log_level=debug; \
$(GOTEST) -timeout 20m -race $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; }
@$(GOFAIL_DISABLE)

leak:
$(GO) get github.com/pingcap/gofail
@$(GOFAIL_ENABLE)
leak: gofail-enable
@export log_level=debug; \
$(GOTEST) -tags leak $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; }
@$(GOFAIL_DISABLE)

tikv_integration_test:
$(GO) get github.com/pingcap/gofail
@$(GOFAIL_ENABLE)
tikv_integration_test: gofail-enable
$(GOTEST) ./store/tikv/. -with-tikv=true || { $(GOFAIL_DISABLE); exit 1; }
@$(GOFAIL_DISABLE)

Expand Down Expand Up @@ -200,37 +190,40 @@ importer:
checklist:
cat checklist.md

gofail-enable:
gofail-enable: tools/bin/gofail
# Converting gofail failpoints...
@$(GOFAIL_ENABLE)

gofail-disable:
gofail-disable: tools/bin/gofail
# Restoring gofail failpoints...
@$(GOFAIL_DISABLE)

checkdep:
$(GO) list -f '{{ join .Imports "\n" }}' github.com/pingcap/tidb/store/tikv | grep ^github.com/pingcap/parser$$ || exit 0; exit 1

tools/bin/megacheck:
tools/bin/megacheck: tools/check/go.mod
cd tools/check; \
$go build -o ../bin/megacheck honnef.co/go/tools/cmd/megacheck
$(GO) build -o ../bin/megacheck honnef.co/go/tools/cmd/megacheck

tools/bin/revive:
tools/bin/revive: tools/check/go.mod
cd tools/check; \
$(GO) build -o ../bin/revive github.com/mgechev/revive

tools/bin/goword:
tools/bin/goword: tools/check/go.mod
cd tools/check; \
$(GO) build -o ../bin/goword github.com/chzchzchz/goword

tools/bin/gometalinter:
tools/bin/gometalinter: tools/check/go.mod
cd tools/check; \
$(GO) build -o ../bin/gometalinter gopkg.in/alecthomas/gometalinter.v2

tools/bin/gosec:
tools/bin/gosec: tools/check/go.mod
cd tools/check; \
$(GO) build -o ../bin/gosec github.com/securego/gosec/cmd/gosec

tools/bin/errcheck:
tools/bin/errcheck: tools/check/go.mod
cd tools/check; \
$(GO) build -o ../bin/errcheck github.com/kisielk/errcheck

tools/bin/gofail: go.mod
$(GO) build -o $@ github.com/pingcap/gofail
2 changes: 1 addition & 1 deletion README.md
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@

[![Build Status](https://travis-ci.org/pingcap/tidb.svg?branch=master)](https://travis-ci.org/pingcap/tidb)
[![Go Report Card](https://goreportcard.com/badge/github.com/pingcap/tidb)](https://goreportcard.com/report/github.com/pingcap/tidb)
![GitHub release](https://img.shields.io/github/release/pingcap/tidb.svg)
![GitHub release](https://img.shields.io/github/tag/pingcap/tidb.svg?label=release)
[![CircleCI Status](https://circleci.com/gh/pingcap/tidb.svg?style=shield)](https://circleci.com/gh/pingcap/tidb)
[![Coverage Status](https://codecov.io/gh/pingcap/tidb/branch/master/graph/badge.svg)](https://codecov.io/gh/pingcap/tidb)

Expand Down
99 changes: 96 additions & 3 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -473,6 +473,64 @@ func (s *testDBSuite) TestCancelDropIndex(c *C) {
s.mustExec(c, "alter table t drop index idx_c2")
}

// TestCancelRenameIndex tests cancel ddl job which type is rename index.
func (s *testDBSuite) TestCancelRenameIndex(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.mustExec(c, "use test_db")
s.mustExec(c, "create database if not exists test_rename_index")
s.mustExec(c, "drop table if exists t")
s.mustExec(c, "create table t(c1 int, c2 int)")
defer s.mustExec(c, "drop table t;")
for i := 0; i < 100; i++ {
s.mustExec(c, "insert into t values (?, ?)", i, i)
}
s.mustExec(c, "alter table t add index idx_c2(c2)")
var checkErr error
hook := &ddl.TestDDLCallback{}
hook.OnJobRunBeforeExported = func(job *model.Job) {
if job.Type == model.ActionRenameIndex && job.State == model.JobStateNone {
jobIDs := []int64{job.ID}
hookCtx := mock.NewContext()
hookCtx.Store = s.store
err := hookCtx.NewTxn(context.Background())
if err != nil {
checkErr = errors.Trace(err)
return
}
txn, err := hookCtx.Txn(true)
if err != nil {
checkErr = errors.Trace(err)
return
}
errs, err := admin.CancelJobs(txn, jobIDs)
if err != nil {
checkErr = errors.Trace(err)
return
}
if errs[0] != nil {
checkErr = errors.Trace(errs[0])
return
}
checkErr = txn.Commit(context.Background())
}
}
originalHook := s.dom.DDL().GetHook()
s.dom.DDL().(ddl.DDLForTest).SetHook(hook)
rs, err := s.tk.Exec("alter table t rename index idx_c2 to idx_c3")
if rs != nil {
rs.Close()
}
c.Assert(checkErr, IsNil)
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:12]cancelled DDL job")
s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook)
t := s.testGetTable(c, "t")
for _, idx := range t.Indices() {
c.Assert(strings.EqualFold(idx.Meta().Name.L, "idx_c3"), IsFalse)
}
s.mustExec(c, "alter table t rename index idx_c2 to idx_c3")
}

// TestCancelDropTable tests cancel ddl job which type is drop table.
func (s *testDBSuite) TestCancelDropTableAndSchema(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
Expand Down Expand Up @@ -1619,14 +1677,49 @@ func (s *testDBSuite) testRenameTable(c *C, sql string, isAlterTable bool) {

// for failure case
failSQL := fmt.Sprintf(sql, "test_not_exist.t", "test_not_exist.t")
s.testErrorCode(c, failSQL, tmysql.ErrFileNotFound)
if isAlterTable {
s.testErrorCode(c, failSQL, tmysql.ErrNoSuchTable)
} else {
s.testErrorCode(c, failSQL, tmysql.ErrFileNotFound)
}
failSQL = fmt.Sprintf(sql, "test.test_not_exist", "test.test_not_exist")
s.testErrorCode(c, failSQL, tmysql.ErrFileNotFound)
if isAlterTable {
s.testErrorCode(c, failSQL, tmysql.ErrNoSuchTable)
} else {
s.testErrorCode(c, failSQL, tmysql.ErrFileNotFound)
}
failSQL = fmt.Sprintf(sql, "test.t_not_exist", "test_not_exist.t")
s.testErrorCode(c, failSQL, tmysql.ErrFileNotFound)
if isAlterTable {
s.testErrorCode(c, failSQL, tmysql.ErrNoSuchTable)
} else {
s.testErrorCode(c, failSQL, tmysql.ErrFileNotFound)
}
failSQL = fmt.Sprintf(sql, "test1.t2", "test_not_exist.t")
s.testErrorCode(c, failSQL, tmysql.ErrErrorOnRename)

s.tk.MustExec("use test1")
s.tk.MustExec("create table if not exists t_exist (c1 int, c2 int)")
failSQL = fmt.Sprintf(sql, "test1.t2", "test1.t_exist")
s.testErrorCode(c, failSQL, tmysql.ErrTableExists)
failSQL = fmt.Sprintf(sql, "test.t_not_exist", "test1.t_exist")
if isAlterTable {
s.testErrorCode(c, failSQL, tmysql.ErrNoSuchTable)
} else {
s.testErrorCode(c, failSQL, tmysql.ErrTableExists)
}
failSQL = fmt.Sprintf(sql, "test_not_exist.t", "test1.t_exist")
if isAlterTable {
s.testErrorCode(c, failSQL, tmysql.ErrNoSuchTable)
} else {
s.testErrorCode(c, failSQL, tmysql.ErrTableExists)
}
failSQL = fmt.Sprintf(sql, "test_not_exist.t", "test1.t_not_exist")
if isAlterTable {
s.testErrorCode(c, failSQL, tmysql.ErrNoSuchTable)
} else {
s.testErrorCode(c, failSQL, tmysql.ErrFileNotFound)
}

// for the same table name
s.tk.MustExec("use test1")
s.tk.MustExec("create table if not exists t (c1 int, c2 int)")
Expand Down
12 changes: 12 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -2283,10 +2283,22 @@ func (d *ddl) RenameTable(ctx sessionctx.Context, oldIdent, newIdent ast.Ident,
is := d.GetInformationSchema(ctx)
oldSchema, ok := is.SchemaByName(oldIdent.Schema)
if !ok {
if isAlterTable {
return infoschema.ErrTableNotExists.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name)
}
if is.TableExists(newIdent.Schema, newIdent.Name) {
return infoschema.ErrTableExists.GenWithStackByArgs(newIdent)
}
return errFileNotFound.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name)
}
oldTbl, err := is.TableByName(oldIdent.Schema, oldIdent.Name)
if err != nil {
if isAlterTable {
return infoschema.ErrTableNotExists.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name)
}
if is.TableExists(newIdent.Schema, newIdent.Name) {
return infoschema.ErrTableExists.GenWithStackByArgs(newIdent)
}
return errFileNotFound.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name)
}
if isAlterTable && newIdent.Schema.L == oldIdent.Schema.L && newIdent.Name.L == oldIdent.Name.L {
Expand Down
35 changes: 20 additions & 15 deletions ddl/ddl_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -375,9 +375,9 @@ func buildCancelJobTests(firstID int64) []testCancelJob {
// Test create database, watch out, database id will alloc a globalID.
{act: model.ActionCreateSchema, jobIDs: []int64{firstID + 12}, cancelRetErrs: noErrs, cancelState: model.StateNone, ddlRetErr: err},

{act: model.ActionDropColumn, jobIDs: []int64{firstID + 13}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 13)}, cancelState: model.StateDeleteOnly, ddlRetErr: err},
{act: model.ActionDropColumn, jobIDs: []int64{firstID + 14}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 14)}, cancelState: model.StateWriteOnly, ddlRetErr: err},
{act: model.ActionDropColumn, jobIDs: []int64{firstID + 15}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 15)}, cancelState: model.StateWriteReorganization, ddlRetErr: err},
{act: model.ActionDropColumn, jobIDs: []int64{firstID + 13}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 13)}, cancelState: model.StateDeleteOnly, ddlRetErr: err},
{act: model.ActionDropColumn, jobIDs: []int64{firstID + 14}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 14)}, cancelState: model.StateWriteOnly, ddlRetErr: err},
{act: model.ActionDropColumn, jobIDs: []int64{firstID + 15}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 15)}, cancelState: model.StateWriteReorganization, ddlRetErr: err},
}

return tests
Expand Down Expand Up @@ -427,15 +427,15 @@ func (s *testDDLSuite) TestCancelJob(c *C) {
dbInfo := testSchemaInfo(c, d, "test_cancel_job")
testCreateSchema(c, testNewContext(d), d, dbInfo)

// create table t (c1 int, c2 int);
tblInfo := testTableInfo(c, d, "t", 2)
// create table t (c1 int, c2 int, c3 int, c4 int, c5 int);
tblInfo := testTableInfo(c, d, "t", 5)
ctx := testNewContext(d)
err := ctx.NewTxn(context.Background())
c.Assert(err, IsNil)
job := testCreateTable(c, ctx, d, dbInfo, tblInfo)
// insert t values (1, 2);
// insert t values (1, 2, 3, 4, 5);
originTable := testGetTable(c, d, dbInfo.ID, tblInfo.ID)
row := types.MakeDatums(1, 2)
row := types.MakeDatums(1, 2, 3, 4, 5)
_, err = originTable.AddRecord(ctx, row)
c.Assert(err, IsNil)
txn, err := ctx.Txn(true)
Expand Down Expand Up @@ -559,21 +559,26 @@ func (s *testDDLSuite) TestCancelJob(c *C) {

// for drop column.
test = &tests[10]
dropColName := "c2"
dropColumnArgs := []interface{}{model.NewCIStr(dropColName)}
doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionDropColumn, dropColumnArgs, &cancelState)
c.Check(errors.ErrorStack(checkErr), Equals, "")
dropColName := "c3"
s.checkCancelDropColumn(c, d, dbInfo.ID, tblInfo.ID, dropColName, false)
testDropColumn(c, ctx, d, dbInfo, tblInfo, dropColName, false)
c.Check(errors.ErrorStack(checkErr), Equals, "")
s.checkCancelDropColumn(c, d, dbInfo.ID, tblInfo.ID, dropColName, true)

test = &tests[11]
doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionDropColumn, dropColumnArgs, &cancelState)
c.Check(errors.ErrorStack(checkErr), Equals, "")

dropColName = "c4"
s.checkCancelDropColumn(c, d, dbInfo.ID, tblInfo.ID, dropColName, false)
testDropColumn(c, ctx, d, dbInfo, tblInfo, dropColName, false)
c.Check(errors.ErrorStack(checkErr), Equals, "")
s.checkCancelDropColumn(c, d, dbInfo.ID, tblInfo.ID, dropColName, true)

test = &tests[12]
doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionDropColumn, dropColumnArgs, &cancelState)
c.Check(errors.ErrorStack(checkErr), Equals, "")
dropColName = "c5"
s.checkCancelDropColumn(c, d, dbInfo.ID, tblInfo.ID, dropColName, false)
testDropColumn(c, ctx, d, dbInfo, tblInfo, dropColName, false)
c.Check(errors.ErrorStack(checkErr), Equals, "")
s.checkCancelDropColumn(c, d, dbInfo.ID, tblInfo.ID, dropColName, true)
}

func (s *testDDLSuite) TestIgnorableSpec(c *C) {
Expand Down
42 changes: 26 additions & 16 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -200,26 +200,11 @@ func validateRenameIndex(from, to model.CIStr, tbl *model.TableInfo) (ignore boo
}

func onRenameIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) {
var from, to model.CIStr
if err := job.DecodeArgs(&from, &to); err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
tblInfo, err := getTableInfo(t, job, job.SchemaID)
tblInfo, from, to, err := checkRenameIndex(t, job)
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}

// Double check. See function `RenameIndex` in ddl_api.go
duplicate, err := validateRenameIndex(from, to, tblInfo)
if duplicate {
return ver, nil
}
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
idx := schemautil.FindIndexByName(from.L, tblInfo.Indices)
idx.Name = to
if ver, err = updateVersionAndTableInfo(t, job, tblInfo, true); err != nil {
Expand Down Expand Up @@ -436,6 +421,31 @@ func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Inde
return tblInfo, indexInfo, nil
}

func checkRenameIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, model.CIStr, model.CIStr, error) {
var from, to model.CIStr
schemaID := job.SchemaID
tblInfo, err := getTableInfo(t, job, schemaID)
if err != nil {
return nil, from, to, errors.Trace(err)
}

if err := job.DecodeArgs(&from, &to); err != nil {
job.State = model.JobStateCancelled
return nil, from, to, errors.Trace(err)
}

// Double check. See function `RenameIndex` in ddl_api.go
duplicate, err := validateRenameIndex(from, to, tblInfo)
if duplicate {
return nil, from, to, nil
}
if err != nil {
job.State = model.JobStateCancelled
return nil, from, to, errors.Trace(err)
}
return tblInfo, from, to, errors.Trace(err)
}

const (
// DefaultTaskHandleCnt is default batch size of adding indices.
DefaultTaskHandleCnt = 128
Expand Down
Loading

0 comments on commit bad812f

Please sign in to comment.