Skip to content

Commit

Permalink
Merge branch 'master' into resolve_indices
Browse files Browse the repository at this point in the history
  • Loading branch information
zz-jason authored Jan 13, 2019
2 parents 965bb26 + 1d00f75 commit bc7f14c
Show file tree
Hide file tree
Showing 81 changed files with 2,325 additions and 617 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
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
154 changes: 154 additions & 0 deletions cmd/pluginpkg/pluginpkg.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,154 @@
// Copyright 2018 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package main

import (
"context"
"encoding/json"
"flag"
"fmt"
"log"
"os"
"os/exec"
"path"
"path/filepath"
"text/template"
"time"

"github.com/BurntSushi/toml"
)

var (
pkgDir string
outDir string
)

const codeTemplate = `
package main
import (
"github.com/pingcap/tidb/plugin"
"github.com/pingcap/tidb/sessionctx/variable"
)
func PluginManifest() *plugin.Manifest {
return plugin.ExportManifest(&plugin.{{.kind}}Manifest{
Manifest: plugin.Manifest{
Kind: plugin.{{.kind}},
Name: "{{.name}}",
Description: "{{.description}}",
Version: {{.version}},
RequireVersion: map[string]uint16{},
License: "{{.license}}",
BuildTime: "{{.buildTime}}",
SysVars: map[string]*variable.SysVar{
{{range .sysVars}}
"{{.name}}": {
Scope: variable.Scope{{.scope}},
Name: "{{.name}}",
Value: "{{.value}}",
},
{{end}}
},
Validate: {{.validate}},
OnInit: {{.onInit}},
OnShutdown: {{.onShutdown}},
},
{{range .export}}
{{.extPoint}}: {{.impl}},
{{end}}
})
}
`

func init() {
flag.StringVar(&pkgDir, "pkg-dir", "", "plugin package folder path")
flag.StringVar(&outDir, "out-dir", "", "plugin packaged folder path")
flag.Usage = usage
}

func usage() {
log.Printf("Usage: %s --pkg-dir [plugin source pkg folder] --outDir-dir [outDir-dir]\n", path.Base(os.Args[0]))
flag.PrintDefaults()
os.Exit(1)
}

func main() {
flag.Parse()
if pkgDir == "" || outDir == "" {
flag.Usage()
}
var manifest map[string]interface{}
_, err := toml.DecodeFile(filepath.Join(pkgDir, "manifest.toml"), &manifest)
if err != nil {
log.Printf("read pkg %s's manifest failure, %+v\n", pkgDir, err)
os.Exit(1)
}
manifest["buildTime"] = time.Now().String()

pluginName := manifest["name"].(string)
if pluginName != filepath.Base(pkgDir) {
log.Printf("plugin package must be same with plugin name in manifest file\n")
os.Exit(1)
}

version := manifest["version"].(string)
tmpl, err := template.New("gen-plugin").Parse(codeTemplate)
if err != nil {
log.Printf("generate code failure during parse template, %+v\n", err)
os.Exit(1)
}

genFileName := filepath.Join(pkgDir, filepath.Base(pkgDir)+".gen.go")
genFile, err := os.OpenFile(genFileName, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0755)
if err != nil {
log.Printf("generate code failure during prepare output file, %+v\n", err)
os.Exit(1)
}
defer func() {
err1 := os.Remove(genFileName)
if err1 != nil {
log.Printf("remove tmp file %s failure, please clean up manually at %v", genFileName, err1)
}
}()

err = tmpl.Execute(genFile, manifest)
if err != nil {
log.Printf("generate code failure during generating code, %+v\n", err)
os.Exit(1)
}

outputFile := filepath.Join(outDir, pluginName+"-"+version+".so")
pluginPath := `-pluginpath=` + pluginName + "-" + version
ctx := context.Background()
buildCmd := exec.CommandContext(ctx, "go", "build",
"-ldflags", pluginPath,
"-buildmode=plugin",
"-o", outputFile, pkgDir)
buildCmd.Stderr = os.Stderr
buildCmd.Stdout = os.Stdout
buildCmd.Env = append(os.Environ(), "GO111MODULE=on")
err = buildCmd.Run()
if err != nil {
log.Printf("compile plugin source code failure, %+v\n", err)
os.Exit(1)
}
fmt.Printf(`Package "%s" as plugin "%s" success.`+"\nManifest:\n", pkgDir, outputFile)
encoder := json.NewEncoder(os.Stdout)
encoder.SetIndent(" ", "\t")
err = encoder.Encode(manifest)
if err != nil {
log.Printf("print manifest detail failure, err: %v", err)
}
}
58 changes: 58 additions & 0 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
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
7 changes: 5 additions & 2 deletions ddl/failtest/fail_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/ddl/testutil"
ddlutil "github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/session"
Expand Down Expand Up @@ -342,11 +343,13 @@ func (s *testFailDBSuite) TestAddIndexWorkerNum(c *C) {
// Split table to multi region.
s.cluster.SplitTable(s.mvccStore, tbl.Meta().ID, splitCount)

err = ddlutil.LoadDDLReorgVars(tk.Se)
c.Assert(err, IsNil)
originDDLAddIndexWorkerCnt := variable.GetDDLReorgWorkerCounter()
lastSetWorkerCnt := originDDLAddIndexWorkerCnt
atomic.StoreInt32(&ddl.TestCheckWorkerNumber, lastSetWorkerCnt)
ddl.TestCheckWorkerNumber = lastSetWorkerCnt
defer variable.SetDDLReorgWorkerCounter(originDDLAddIndexWorkerCnt)
defer tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_worker_cnt=%d", originDDLAddIndexWorkerCnt))

gofail.Enable("github.com/pingcap/tidb/ddl/checkIndexWorkerNum", `return(true)`)
defer gofail.Disable("github.com/pingcap/tidb/ddl/checkIndexWorkerNum")
Expand All @@ -364,7 +367,7 @@ LOOP:
c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err)))
case <-ddl.TestCheckWorkerNumCh:
lastSetWorkerCnt = int32(rand.Intn(8) + 8)
tk.MustExec(fmt.Sprintf("set @@tidb_ddl_reorg_worker_cnt=%d", lastSetWorkerCnt))
tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_worker_cnt=%d", lastSetWorkerCnt))
atomic.StoreInt32(&ddl.TestCheckWorkerNumber, lastSetWorkerCnt)
checkNum++
}
Expand Down
Loading

0 comments on commit bc7f14c

Please sign in to comment.