Skip to content

Commit

Permalink
Merge branch 'master' into fix-data-race
Browse files Browse the repository at this point in the history
  • Loading branch information
ciscoxll authored Nov 27, 2018
2 parents c46e366 + 487bb50 commit 77e25cc
Show file tree
Hide file tree
Showing 145 changed files with 3,156 additions and 597 deletions.
2 changes: 2 additions & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -12,5 +12,7 @@ y.output
profile.coverprofile
explain_test
cmd/explaintest/explain-test.out
cmd/explaintest/explaintest_tidb-server
_tools/
*.fail.go
vendor
11 changes: 9 additions & 2 deletions Dockerfile
Original file line number Diff line number Diff line change
@@ -1,10 +1,14 @@
# Builder image
FROM golang:1.10.1-alpine as builder
FROM golang:1.11.2-alpine as builder

RUN apk add --no-cache \
wget \
make \
git

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 . /go/src/github.com/pingcap/tidb

WORKDIR /go/src/github.com/pingcap/tidb/
Expand All @@ -15,10 +19,13 @@ RUN make
FROM scratch

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



WORKDIR /

EXPOSE 4000

ENTRYPOINT ["/tidb-server"]
ENTRYPOINT ["/usr/local/bin/dumb-init", "/tidb-server"]

2 changes: 1 addition & 1 deletion README.md
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ TiDB (The pronunciation is: /'taɪdiːbi:/ tai-D-B, etymology: titanium) is an o

TiDB is designed to work in the cloud -- public, private, or hybrid -- making deployment, provisioning, and maintenance drop-dead simple.

- __No more ETL__
- __Minimize ETL__

ETL (Extract, Transform and Load) is no longer necessary with TiDB's hybrid OLTP/OLAP architecture, enabling you to create new values for your users, easier and faster.

Expand Down
12 changes: 8 additions & 4 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -150,6 +150,7 @@ type Status struct {
// Performance is the performance section of the config.
type Performance struct {
MaxProcs uint `toml:"max-procs" json:"max-procs"`
MaxMemory uint64 `toml:"max-memory" json:"max-memory"`
TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"`
CrossJoin bool `toml:"cross-join" json:"cross-join"`
StatsLease string `toml:"stats-lease" json:"stats-lease"`
Expand Down Expand Up @@ -184,8 +185,9 @@ type TxnLocalLatches struct {

// PreparedPlanCache is the PreparedPlanCache section of the config.
type PreparedPlanCache struct {
Enabled bool `toml:"enabled" json:"enabled"`
Capacity uint `toml:"capacity" json:"capacity"`
Enabled bool `toml:"enabled" json:"enabled"`
Capacity uint `toml:"capacity" json:"capacity"`
MemoryGuardRatio float64 `toml:"memory-guard-ratio" json:"memory-guard-ratio"`
}

// OpenTracing is the opentracing section of the config.
Expand Down Expand Up @@ -287,6 +289,7 @@ var defaultConf = Config{
MetricsInterval: 15,
},
Performance: Performance{
MaxMemory: 0,
TCPKeepAlive: true,
CrossJoin: true,
StatsLease: "3s",
Expand All @@ -306,8 +309,9 @@ var defaultConf = Config{
HeaderTimeout: 5,
},
PreparedPlanCache: PreparedPlanCache{
Enabled: false,
Capacity: 100,
Enabled: false,
Capacity: 100,
MemoryGuardRatio: 0.1,
},
OpenTracing: OpenTracing{
Enable: false,
Expand Down
3 changes: 3 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,8 @@ metrics-interval = 15
[performance]
# Max CPUs to use, 0 use number of CPUs in the machine.
max-procs = 0
# Max memory size to use, 0 use the total usable memory in the machine.
max-memory = 0
# StmtCountLimit limits the max count of statement inside a transaction.
stmt-count-limit = 5000

Expand Down Expand Up @@ -162,6 +164,7 @@ header-timeout = 5
[prepared-plan-cache]
enabled = false
capacity = 100
memory-guard-ratio = 0.1

[opentracing]
# Enable opentracing.
Expand Down
6 changes: 3 additions & 3 deletions ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ func (s *testColumnChangeSuite) TestColumnChange(c *C) {
row := types.MakeDatums(1, 2)
h, err := originTable.AddRecord(ctx, row, false)
c.Assert(err, IsNil)
err = ctx.Txn().Commit(context.Background())
err = ctx.Txn(true).Commit(context.Background())
c.Assert(err, IsNil)

var mu sync.Mutex
Expand Down Expand Up @@ -127,7 +127,7 @@ func (s *testColumnChangeSuite) TestColumnChange(c *C) {
}
mu.Unlock()
}
err = hookCtx.Txn().Commit(context.Background())
err = hookCtx.Txn(true).Commit(context.Background())
if err != nil {
checkErr = errors.Trace(err)
}
Expand Down Expand Up @@ -179,7 +179,7 @@ func (s *testColumnChangeSuite) testAddColumnNoDefault(c *C, ctx sessionctx.Cont
checkErr = errors.Trace(err)
}
}
err = hookCtx.Txn().Commit(context.TODO())
err = hookCtx.Txn(true).Commit(context.TODO())
if err != nil {
checkErr = errors.Trace(err)
}
Expand Down
12 changes: 6 additions & 6 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -271,9 +271,9 @@ func (s *testColumnSuite) checkColumnKVExist(ctx sessionctx.Context, t table.Tab
if err != nil {
return errors.Trace(err)
}
defer ctx.Txn().Commit(context.Background())
defer ctx.Txn(true).Commit(context.Background())
key := t.RecordKey(handle)
data, err := ctx.Txn().Get(key)
data, err := ctx.Txn(true).Get(key)
if !isExist {
if terror.ErrorEqual(err, kv.ErrNotExist) {
return nil
Expand Down Expand Up @@ -760,7 +760,7 @@ func (s *testColumnSuite) TestAddColumn(c *C) {
handle, err := t.AddRecord(ctx, oldRow, false)
c.Assert(err, IsNil)

err = ctx.Txn().Commit(context.Background())
err = ctx.Txn(true).Commit(context.Background())
c.Assert(err, IsNil)

newColName := "c4"
Expand Down Expand Up @@ -823,7 +823,7 @@ func (s *testColumnSuite) TestAddColumn(c *C) {
job = testDropTable(c, ctx, d, s.dbInfo, tblInfo)
testCheckJobDone(c, d, job, false)

err = ctx.Txn().Commit(context.Background())
err = ctx.Txn(true).Commit(context.Background())
c.Assert(err, IsNil)

d.Stop()
Expand All @@ -847,7 +847,7 @@ func (s *testColumnSuite) TestDropColumn(c *C) {
_, err = t.AddRecord(ctx, append(row, types.NewDatum(defaultColValue)), false)
c.Assert(err, IsNil)

err = ctx.Txn().Commit(context.Background())
err = ctx.Txn(true).Commit(context.Background())
c.Assert(err, IsNil)

checkOK := false
Expand Down Expand Up @@ -896,7 +896,7 @@ func (s *testColumnSuite) TestDropColumn(c *C) {
job = testDropTable(c, ctx, d, s.dbInfo, tblInfo)
testCheckJobDone(c, d, job, false)

err = ctx.Txn().Commit(context.Background())
err = ctx.Txn(true).Commit(context.Background())
c.Assert(err, IsNil)

d.Stop()
Expand Down
111 changes: 98 additions & 13 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -301,17 +301,21 @@ func (s *testDBSuite) TestRenameIndex(c *C) {
s.testErrorCode(c, "alter table t rename key k3 to K2", mysql.ErrDupKeyName)
}

func (s *testDBSuite) testGetTable(c *C, name string) table.Table {
func (s *testDBSuite) testGetTableByName(c *C, db, table string) table.Table {
ctx := s.s.(sessionctx.Context)
dom := domain.GetDomain(ctx)
// Make sure the table schema is the new schema.
err := dom.Reload()
c.Assert(err, IsNil)
tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(s.schemaName), model.NewCIStr(name))
tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(table))
c.Assert(err, IsNil)
return tbl
}

func (s *testDBSuite) testGetTable(c *C, name string) table.Table {
return s.testGetTableByName(c, s.schemaName, name)
}

func backgroundExec(s kv.Storage, sql string, done chan error) {
se, err := session.CreateSession4Test(s)
if err != nil {
Expand Down Expand Up @@ -683,12 +687,12 @@ LOOP:
// Make sure there is index with name c3_index.
c.Assert(nidx, NotNil)
c.Assert(nidx.Meta().ID, Greater, int64(0))
ctx.Txn().Rollback()
ctx.Txn(true).Rollback()

c.Assert(ctx.NewTxn(), IsNil)
defer ctx.Txn().Rollback()
defer ctx.Txn(true).Rollback()

it, err := nidx.SeekFirst(ctx.Txn())
it, err := nidx.SeekFirst(ctx.Txn(true))
c.Assert(err, IsNil)
defer it.Close()

Expand Down Expand Up @@ -784,9 +788,9 @@ func checkDelRangeDone(c *C, ctx sessionctx.Context, idx table.Index) {
handles := make(map[int64]struct{})

c.Assert(ctx.NewTxn(), IsNil)
defer ctx.Txn().Rollback()
defer ctx.Txn(true).Rollback()

it, err := idx.SeekFirst(ctx.Txn())
it, err := idx.SeekFirst(ctx.Txn(true))
c.Assert(err, IsNil)
defer it.Close()

Expand Down Expand Up @@ -1026,7 +1030,7 @@ LOOP:
i := 0
j := 0
ctx.NewTxn()
defer ctx.Txn().Rollback()
defer ctx.Txn(true).Rollback()
err = t.IterRecords(ctx, t.FirstKey(), t.Cols(),
func(h int64, data []types.Datum, cols []*table.Column) (bool, error) {
i++
Expand Down Expand Up @@ -1537,6 +1541,23 @@ func (s *testDBSuite) TestCreateTable(c *C) {

_, err = s.tk.Exec("CREATE TABLE `t` (`a` int) DEFAULT CHARSET=abcdefg")
c.Assert(err, NotNil)

// test for enum column
failSQL := "create table t_enum (a enum('e','e'));"
s.testErrorCode(c, failSQL, tmysql.ErrDuplicatedValueInType)
failSQL = "create table t_enum (a enum('e','E'));"
s.testErrorCode(c, failSQL, tmysql.ErrDuplicatedValueInType)
failSQL = "create table t_enum (a enum('abc','Abc'));"
s.testErrorCode(c, failSQL, tmysql.ErrDuplicatedValueInType)
// test for set column
failSQL = "create table t_enum (a set('e','e'));"
s.testErrorCode(c, failSQL, tmysql.ErrDuplicatedValueInType)
failSQL = "create table t_enum (a set('e','E'));"
s.testErrorCode(c, failSQL, tmysql.ErrDuplicatedValueInType)
failSQL = "create table t_enum (a set('abc','Abc'));"
s.testErrorCode(c, failSQL, tmysql.ErrDuplicatedValueInType)
_, err = s.tk.Exec("create table t_enum (a enum('B','b'));")
c.Assert(err.Error(), Equals, "[types:1291]Column 'a' has duplicated value 'B' in ENUM")
}

func (s *testDBSuite) TestTableForeignKey(c *C) {
Expand Down Expand Up @@ -1792,6 +1813,7 @@ func (s *testDBSuite) TestCreateTableWithHashPartition(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use test;")
s.tk.MustExec("drop table if exists employees;")
s.tk.MustExec("set @@session.tidb_enable_table_partition = 1")
s.tk.MustExec(`
create table employees (
id int not null,
Expand Down Expand Up @@ -3443,7 +3465,7 @@ func backgroundExecOnJobUpdatedExported(c *C, s *testDBSuite, hook *ddl.TestDDLC
return
}
jobIDs := []int64{job.ID}
errs, err := admin.CancelJobs(hookCtx.Txn(), jobIDs)
errs, err := admin.CancelJobs(hookCtx.Txn(true), jobIDs)
if err != nil {
checkErr = errors.Trace(err)
return
Expand All @@ -3453,7 +3475,7 @@ func backgroundExecOnJobUpdatedExported(c *C, s *testDBSuite, hook *ddl.TestDDLC
checkErr = errors.Trace(errs[0])
return
}
err = hookCtx.Txn().Commit(context.Background())
err = hookCtx.Txn(true).Commit(context.Background())
if err != nil {
checkErr = errors.Trace(err)
}
Expand Down Expand Up @@ -3522,7 +3544,7 @@ func (s *testDBSuite) TestModifyColumnRollBack(c *C) {
}

jobIDs := []int64{job.ID}
errs, err := admin.CancelJobs(hookCtx.Txn(), jobIDs)
errs, err := admin.CancelJobs(hookCtx.Txn(true), jobIDs)
if err != nil {
checkErr = errors.Trace(err)
return
Expand All @@ -3533,7 +3555,7 @@ func (s *testDBSuite) TestModifyColumnRollBack(c *C) {
return
}

err = hookCtx.Txn().Commit(context.Background())
err = hookCtx.Txn(true).Commit(context.Background())
if err != nil {
checkErr = errors.Trace(err)
}
Expand Down Expand Up @@ -3588,7 +3610,7 @@ func (s *testDBSuite) TestPartitionAddIndex(c *C) {

tk.MustExec("alter table partition_add_idx add index idx1 (hired)")
tk.MustExec("alter table partition_add_idx add index idx2 (id, hired)")
ctx := s.tk.Se.(sessionctx.Context)
ctx := tk.Se.(sessionctx.Context)
is := domain.GetDomain(ctx).InfoSchema()
t, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("partition_add_idx"))
c.Assert(err, IsNil)
Expand All @@ -3607,3 +3629,66 @@ func (s *testDBSuite) TestPartitionAddIndex(c *C) {
tk.MustExec("admin check table partition_add_idx")
tk.MustExec("drop table partition_add_idx")
}

func (s *testDBSuite) TestDropSchemaWithPartitionTable(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("drop database if exists test_db_with_partition")
s.tk.MustExec("create database test_db_with_partition")
s.tk.MustExec("use test_db_with_partition")
s.tk.MustExec(`create table t_part (a int key)
partition by range(a) (
partition p0 values less than (10),
partition p1 values less than (20)
);`)
s.tk.MustExec("insert into t_part values (1),(2),(11),(12);")
ctx := s.s.(sessionctx.Context)
tbl := s.testGetTableByName(c, "test_db_with_partition", "t_part")

// check records num before drop database.
recordsNum := getPartitionTableRecordsNum(c, ctx, tbl.(table.PartitionedTable))
c.Assert(recordsNum, Equals, 4)

s.tk.MustExec("drop database if exists test_db_with_partition")

// check job args.
rs, err := s.tk.Exec("admin show ddl jobs")
c.Assert(err, IsNil)
rows, err := session.GetRows4Test(context.Background(), s.tk.Se, rs)
c.Assert(err, IsNil)
row := rows[0]
c.Assert(row.GetString(3), Equals, "drop schema")
jobID := row.GetInt64(0)
kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error {
t := meta.NewMeta(txn)
historyJob, err := t.GetHistoryDDLJob(jobID)
c.Assert(err, IsNil)
var tableIDs []int64
err = historyJob.DecodeArgs(&tableIDs)
c.Assert(err, IsNil)
// There is 2 partitions.
c.Assert(len(tableIDs), Equals, 3)
return nil
})

// check records num after drop database.
recordsNum = getPartitionTableRecordsNum(c, ctx, tbl.(table.PartitionedTable))
c.Assert(recordsNum, Equals, 0)
}

func getPartitionTableRecordsNum(c *C, ctx sessionctx.Context, tbl table.PartitionedTable) int {
num := 0
info := tbl.Meta().GetPartitionInfo()
for _, def := range info.Definitions {
pid := def.ID
partition := tbl.(table.PartitionedTable).GetPartition(pid)
startKey := partition.RecordKey(math.MinInt64)
c.Assert(ctx.NewTxn(), IsNil)
err := partition.IterRecords(ctx, startKey, partition.Cols(),
func(h int64, data []types.Datum, cols []*table.Column) (bool, error) {
num++
return true, nil
})
c.Assert(err, IsNil)
}
return num
}
6 changes: 6 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -431,6 +431,12 @@ func (d *ddl) genGlobalID() (int64, error) {
var globalID int64
err := kv.RunInNewTxn(d.store, true, func(txn kv.Transaction) error {
var err error

// gofail: var mockGenGlobalIDFail bool
// if mockGenGlobalIDFail {
// return errors.New("gofail genGlobalID error")
// }

globalID, err = meta.NewMeta(txn).GenGlobalID()
return errors.Trace(err)
})
Expand Down
Loading

0 comments on commit 77e25cc

Please sign in to comment.