Skip to content

Commit

Permalink
Merge branch 'master' into dev_partitionTest
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 committed May 14, 2021
2 parents 3dd9819 + 80a557e commit 9822aad
Show file tree
Hide file tree
Showing 150 changed files with 2,120 additions and 1,354 deletions.
1 change: 1 addition & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ check-static: tools/bin/golangci-lint
--enable=unused \
--enable=structcheck \
--enable=deadcode \
--enable=gosimple \
$$($(PACKAGE_DIRECTORIES))

check-slow:tools/bin/gometalinter tools/bin/gosec
Expand Down
6 changes: 3 additions & 3 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,9 +60,9 @@ const (
DefHost = "0.0.0.0"
// DefStatusHost is the default status host of TiDB
DefStatusHost = "0.0.0.0"
// Def TableColumnCountLimit is limit of the number of columns in a table
// DefTableColumnCountLimit is limit of the number of columns in a table
DefTableColumnCountLimit = 1017
// Def TableColumnCountLimit is maximum limitation of the number of columns in a table
// DefMaxOfTableColumnCountLimit is maximum limitation of the number of columns in a table
DefMaxOfTableColumnCountLimit = 4096
)

Expand All @@ -73,7 +73,7 @@ var (
"tikv": true,
"unistore": true,
}
// checkTableBeforeDrop enable to execute `admin check table` before `drop table`.
// CheckTableBeforeDrop enable to execute `admin check table` before `drop table`.
CheckTableBeforeDrop = false
// checkBeforeDropLDFlag is a go build flag.
checkBeforeDropLDFlag = "None"
Expand Down
2 changes: 1 addition & 1 deletion ddl/backfilling.go
Original file line number Diff line number Diff line change
Expand Up @@ -677,7 +677,7 @@ func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version

ver := kv.Version{Ver: version}
snap := store.GetSnapshot(ver)
snap.SetOption(tikvstore.Priority, priority)
snap.SetOption(kv.Priority, priority)

it, err := snap.Iter(firstKey, upperBound)
if err != nil {
Expand Down
3 changes: 1 addition & 2 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ import (
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -1346,7 +1345,7 @@ func (w *updateColumnWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t
errInTxn = kv.RunInNewTxn(context.Background(), w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error {
taskCtx.addedCount = 0
taskCtx.scanCount = 0
txn.SetOption(tikvstore.Priority, w.priority)
txn.SetOption(kv.Priority, w.priority)

rowRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange)
if err != nil {
Expand Down
42 changes: 36 additions & 6 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -969,9 +969,9 @@ func (s *testIntegrationSuite5) TestModifyColumnOption(c *C) {

func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) {
tk := testkit.NewTestKit(c, s.store)

tk.MustExec("create database if not exists test")
tk.MustExec("use test")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int as (a + 1), c int as (b + 1))")
tk.MustExec("insert into t (a) values (1)")
Expand All @@ -980,42 +980,72 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) {
res := tk.MustQuery("select * from t use index(idx) where c > 1")
tk.MustQuery("select * from t ignore index(idx) where c > 1").Check(res.Rows())
tk.MustExec("admin check table t")
}

func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn1(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test")
tk.MustExec("use test")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int as (a + 1), c int as (b + 1), d int as (c + 1))")
tk.MustExec("insert into t (a) values (1)")
tk.MustExec("create index idx on t (d)")
tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 3 4"))
res = tk.MustQuery("select * from t use index(idx) where d > 2")
res := tk.MustQuery("select * from t use index(idx) where d > 2")
tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows())
tk.MustExec("admin check table t")
}

func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn2(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test")
tk.MustExec("use test")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a bigint, b decimal as (a+1), c varchar(20) as (b*2), d float as (a*23+b-1+length(c)))")
tk.MustExec("insert into t (a) values (1)")
tk.MustExec("create index idx on t (d)")
tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 4 25"))
res = tk.MustQuery("select * from t use index(idx) where d > 2")
res := tk.MustQuery("select * from t use index(idx) where d > 2")
tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows())
tk.MustExec("admin check table t")
}

func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn3(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test")
tk.MustExec("use test")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a varchar(10), b float as (length(a)+123), c varchar(20) as (right(a, 2)), d float as (b+b-7+1-3+3*ASCII(c)))")
tk.MustExec("insert into t (a) values ('adorable')")
tk.MustExec("create index idx on t (d)")
tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("adorable 131 le 577")) // 131+131-7+1-3+3*108
res = tk.MustQuery("select * from t use index(idx) where d > 2")
res := tk.MustQuery("select * from t use index(idx) where d > 2")
tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows())
tk.MustExec("admin check table t")
}

func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn4(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test")
tk.MustExec("use test")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a bigint, b decimal as (a), c int(10) as (a+b), d float as (a+b+c), e decimal as (a+b+c+d))")
tk.MustExec("insert into t (a) values (1)")
tk.MustExec("create index idx on t (d)")
tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 1 2 4 8"))
res = tk.MustQuery("select * from t use index(idx) where d > 2")
res := tk.MustQuery("select * from t use index(idx) where d > 2")
tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows())
tk.MustExec("admin check table t")
}

func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn5(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test")
tk.MustExec("use test")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a bigint, b bigint as (a+1) virtual, c bigint as (b+1) virtual)")
Expand All @@ -1025,7 +1055,7 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) {
tk.MustExec("alter table t add column(d bigint as (c+1) virtual)")
tk.MustExec("alter table t add index idx_d(d)")
tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 3 4"))
res = tk.MustQuery("select * from t use index(idx_d) where d > 2")
res := tk.MustQuery("select * from t use index(idx_d) where d > 2")
tk.MustQuery("select * from t ignore index(idx_d) where d > 2").Check(res.Rows())
tk.MustExec("admin check table t")
}
Expand Down
8 changes: 3 additions & 5 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2280,10 +2280,8 @@ func (s *testDBSuite6) TestDropColumn(c *C) {
testddlutil.ExecMultiSQLInGoroutine(c, s.store, "drop_col_db", []string{"insert into t2 set c1 = 1, c2 = 1, c3 = 1, c4 = 1"}, dmlDone)
}
for i := 0; i < num; i++ {
select {
case err := <-ddlDone:
c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err)))
}
err := <-ddlDone
c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err)))
}

// Test for drop partition table column.
Expand Down Expand Up @@ -6575,7 +6573,7 @@ func (s *testSerialDBSuite) TestModifyColumnTypeWhenInterception(c *C) {

count := defaultBatchSize * 4
// Add some rows.
dml := fmt.Sprintf("insert into t values")
dml := "insert into t values"
for i := 1; i <= count; i++ {
dml += fmt.Sprintf("(%d, %f)", i, 11.22)
if i != count {
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -4040,7 +4040,7 @@ func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNe
autoid.MaxAutoRandomBits, newRandBits, specNewColumn.Name.Name.O)
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
}
break // Increasing auto_random shard bits is allowed.
// increasing auto_random shard bits is allowed.
case oldRandBits > newRandBits:
if newRandBits == 0 {
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg)
Expand Down
7 changes: 3 additions & 4 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/store/tikv"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
Expand Down Expand Up @@ -1117,7 +1116,7 @@ func (w *addIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (taskC
errInTxn = kv.RunInNewTxn(context.Background(), w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error {
taskCtx.addedCount = 0
taskCtx.scanCount = 0
txn.SetOption(tikvstore.Priority, w.priority)
txn.SetOption(kv.Priority, w.priority)

idxRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange)
if err != nil {
Expand Down Expand Up @@ -1329,7 +1328,7 @@ func (w *cleanUpIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t
errInTxn = kv.RunInNewTxn(context.Background(), w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error {
taskCtx.addedCount = 0
taskCtx.scanCount = 0
txn.SetOption(tikvstore.Priority, w.priority)
txn.SetOption(kv.Priority, w.priority)

idxRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange)
if err != nil {
Expand All @@ -1344,7 +1343,7 @@ func (w *cleanUpIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t
// we fetch records row by row, so records will belong to
// index[0], index[1] ... index[n-1], index[0], index[1] ...
// respectively. So indexes[i%n] is the index of idxRecords[i].
err := w.indexes[i%n].Delete(w.sessCtx.GetSessionVars().StmtCtx, txn.GetUnionStore(), idxRecord.vals, idxRecord.handle)
err := w.indexes[i%n].Delete(w.sessCtx.GetSessionVars().StmtCtx, txn, idxRecord.vals, idxRecord.handle)
if err != nil {
return errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion ddl/index_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,7 @@ func checkIndexExists(ctx sessionctx.Context, tbl table.Table, indexValue interf
if err != nil {
return errors.Trace(err)
}
doesExist, _, err := idx.Exist(ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), types.MakeDatums(indexValue), kv.IntHandle(handle))
doesExist, _, err := idx.Exist(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(indexValue), kv.IntHandle(handle))
if err != nil {
return errors.Trace(err)
}
Expand Down
10 changes: 5 additions & 5 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -503,15 +503,15 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) {

// for failure table cases
tk.MustExec("use ctwl_db")
failSQL := fmt.Sprintf("create table t1 like test_not_exist.t")
failSQL := "create table t1 like test_not_exist.t"
tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable)
failSQL = fmt.Sprintf("create table t1 like test.t_not_exist")
failSQL = "create table t1 like test.t_not_exist"
tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable)
failSQL = fmt.Sprintf("create table t1 (like test_not_exist.t)")
failSQL = "create table t1 (like test_not_exist.t)"
tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable)
failSQL = fmt.Sprintf("create table test_not_exis.t1 like ctwl_db.t")
failSQL = "create table test_not_exis.t1 like ctwl_db.t"
tk.MustGetErrCode(failSQL, mysql.ErrBadDB)
failSQL = fmt.Sprintf("create table t1 like ctwl_db.t")
failSQL = "create table t1 like ctwl_db.t"
tk.MustGetErrCode(failSQL, mysql.ErrTableExists)

// test failure for wrong object cases
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -501,6 +501,11 @@ error = '''
Deadlock found when trying to get lock; try restarting transaction
'''

["executor:1221"]
error = '''
Incorrect usage of %s and %s
'''

["executor:1242"]
error = '''
Subquery returns more than 1 row
Expand Down
3 changes: 1 addition & 2 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
tikverr "github.com/pingcap/tidb/store/tikv/error"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/store/tikv/util"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
Expand Down Expand Up @@ -629,7 +628,7 @@ func UpdateForUpdateTS(seCtx sessionctx.Context, newForUpdateTS uint64) error {
newForUpdateTS = version.Ver
}
seCtx.GetSessionVars().TxnCtx.SetForUpdateTS(newForUpdateTS)
txn.SetOption(tikvstore.SnapshotTS, seCtx.GetSessionVars().TxnCtx.GetForUpdateTS())
txn.SetOption(kv.SnapshotTS, seCtx.GetSessionVars().TxnCtx.GetForUpdateTS())
return nil
}

Expand Down
2 changes: 1 addition & 1 deletion executor/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -575,7 +575,7 @@ func (e *CleanupIndexExec) deleteDanglingIdx(txn kv.Transaction, values map[stri
return errors.Trace(errors.Errorf("batch keys are inconsistent with handles"))
}
for _, handleIdxVals := range handleIdxValsGroup.([][]types.Datum) {
if err := e.index.Delete(e.ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), handleIdxVals, handle); err != nil {
if err := e.index.Delete(e.ctx.GetSessionVars().StmtCtx, txn, handleIdxVals, handle); err != nil {
return err
}
e.removeCnt++
Expand Down
Loading

0 comments on commit 9822aad

Please sign in to comment.