Skip to content

Commit

Permalink
Merge branch 'master' of github.com:pingcap/tidb into read_ts_in
Browse files Browse the repository at this point in the history
  • Loading branch information
JmPotato committed May 18, 2021
2 parents 4772c3d + 9148ff9 commit bde67d7
Show file tree
Hide file tree
Showing 217 changed files with 4,358 additions and 2,513 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
2 changes: 1 addition & 1 deletion cmd/ddltest/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ func (s *TestDDLSuite) SetUpSuite(c *C) {
s.procs = make([]*server, *serverNum)

// Set server restart retry count.
s.retryCount = 5
s.retryCount = 20

createLogFiles(c, *serverNum)
err = s.startServers()
Expand Down
3 changes: 0 additions & 3 deletions cmd/explaintest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/log"
"github.com/pingcap/parser/ast"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/logutil"
Expand Down Expand Up @@ -663,8 +662,6 @@ func main() {
log.Fatal(fmt.Sprintf("%s failed", sql), zap.Error(err))
}
}
// Wait global variables to reload.
time.Sleep(domain.GlobalVariableCacheExpiry)

if _, err = mdb.Exec("set sql_mode='STRICT_TRANS_TABLES'"); err != nil {
log.Fatal("set sql_mode='STRICT_TRANS_TABLES' failed", zap.Error(err))
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
21 changes: 12 additions & 9 deletions ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,15 +47,18 @@ type testColumnChangeSuite struct {
func (s *testColumnChangeSuite) SetUpSuite(c *C) {
SetWaitTimeWhenErrorOccurred(1 * time.Microsecond)
s.store = testCreateStore(c, "test_column_change")
s.dbInfo = &model.DBInfo{
Name: model.NewCIStr("test_column_change"),
ID: 1,
}
err := kv.RunInNewTxn(context.Background(), s.store, true, func(ctx context.Context, txn kv.Transaction) error {
t := meta.NewMeta(txn)
return errors.Trace(t.CreateDatabase(s.dbInfo))
})
c.Check(err, IsNil)
d := testNewDDLAndStart(
context.Background(),
c,
WithStore(s.store),
WithLease(testLease),
)
defer func() {
err := d.Stop()
c.Assert(err, IsNil)
}()
s.dbInfo = testSchemaInfo(c, d, "test_index_change")
testCreateSchema(c, testNewContext(d), d, s.dbInfo)
}

func (s *testColumnChangeSuite) TearDownSuite(c *C) {
Expand Down
3 changes: 1 addition & 2 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,8 +54,7 @@ func (s *testColumnSuite) SetUpSuite(c *C) {

s.dbInfo = testSchemaInfo(c, d, "test_column")
testCreateSchema(c, testNewContext(d), d, s.dbInfo)
err := d.Stop()
c.Assert(err, IsNil)
c.Assert(d.Stop(), IsNil)
}

func (s *testColumnSuite) TearDownSuite(c *C) {
Expand Down
2 changes: 0 additions & 2 deletions ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1063,7 +1063,6 @@ func (s *testStateChangeSuite) TestParallelAddGeneratedColumnAndAlterModifyColum
_, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0")
c.Assert(err, IsNil)
}()
domain.GetDomain(s.se).GetGlobalVarsCache().Disable()

sql1 := "ALTER TABLE t ADD COLUMN f INT GENERATED ALWAYS AS(a+1);"
sql2 := "ALTER TABLE t MODIFY COLUMN a tinyint;"
Expand All @@ -1083,7 +1082,6 @@ func (s *testStateChangeSuite) TestParallelAlterModifyColumnAndAddPK(c *C) {
_, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0")
c.Assert(err, IsNil)
}()
domain.GetDomain(s.se).GetGlobalVarsCache().Disable()

sql1 := "ALTER TABLE t ADD PRIMARY KEY (b) NONCLUSTERED;"
sql2 := "ALTER TABLE t MODIFY COLUMN b tinyint;"
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
20 changes: 13 additions & 7 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -202,7 +202,7 @@ type ddlCtx struct {
ddlEventCh chan<- *util.Event
lease time.Duration // lease is schema lease.
binlogCli *pumpcli.PumpsClient // binlogCli is used for Binlog.
infoHandle *infoschema.Handle
infoCache *infoschema.InfoCache
statsHandle *handle.Handle
tableLockCkr util.DeadTableLockChecker
etcdCli *clientv3.Client
Expand Down Expand Up @@ -282,6 +282,15 @@ func newDDL(ctx context.Context, options ...Option) *ddl {
deadLockCkr = util.NewDeadTableLockChecker(etcdCli)
}

// TODO: make store and infoCache explicit arguments
// these two should be ensured to exist
if opt.Store == nil {
panic("store should not be nil")
}
if opt.InfoCache == nil {
panic("infoCache should not be nil")
}

ddlCtx := &ddlCtx{
uuid: id,
store: opt.Store,
Expand All @@ -290,7 +299,7 @@ func newDDL(ctx context.Context, options ...Option) *ddl {
ownerManager: manager,
schemaSyncer: syncer,
binlogCli: binloginfo.GetPumpsClient(),
infoHandle: opt.InfoHandle,
infoCache: opt.InfoCache,
tableLockCkr: deadLockCkr,
etcdCli: opt.EtcdCli,
}
Expand Down Expand Up @@ -411,7 +420,7 @@ func (d *ddl) GetLease() time.Duration {
// Please don't use this function, it is used by TestParallelDDLBeforeRunDDLJob to intercept the calling of d.infoHandle.Get(), use d.infoHandle.Get() instead.
// Otherwise, the TestParallelDDLBeforeRunDDLJob will hang up forever.
func (d *ddl) GetInfoSchemaWithInterceptor(ctx sessionctx.Context) infoschema.InfoSchema {
is := d.infoHandle.Get()
is := d.infoCache.GetLatest()

d.mu.RLock()
defer d.mu.RUnlock()
Expand Down Expand Up @@ -649,10 +658,7 @@ func (d *ddl) startCleanDeadTableLock() {
if !d.ownerManager.IsOwner() {
continue
}
if d.infoHandle == nil || !d.infoHandle.IsValid() {
continue
}
deadLockTables, err := d.tableLockCkr.GetDeadLockedTables(d.ctx, d.infoHandle.Get().AllSchemas())
deadLockTables, err := d.tableLockCkr.GetDeadLockedTables(d.ctx, d.infoCache.GetLatest().AllSchemas())
if err != nil {
logutil.BgLogger().Info("[ddl] get dead table lock failed.", zap.Error(err))
continue
Expand Down
Loading

0 comments on commit bde67d7

Please sign in to comment.