From aa3e64d08bf19c7230cc3b53c3a57f68a50ae3c3 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Thu, 13 May 2021 19:31:39 +0800 Subject: [PATCH 01/16] store/tikv: use latest PD TS plus one as min commit ts (#24579) --- store/tikv/2pc.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 9e4c28ad6fd34..19f3e4faf40e3 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -996,17 +996,18 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { // If we want to use async commit or 1PC and also want linearizability across // all nodes, we have to make sure the commit TS of this transaction is greater // than the snapshot TS of all existent readers. So we get a new timestamp - // from PD as our MinCommitTS. + // from PD and plus one as our MinCommitTS. if commitTSMayBeCalculated && c.needLinearizability() { failpoint.Inject("getMinCommitTSFromTSO", nil) - minCommitTS, err := c.store.oracle.GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + latestTS, err := c.store.oracle.GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) // If we fail to get a timestamp from PD, we just propagate the failure // instead of falling back to the normal 2PC because a normal 2PC will // also be likely to fail due to the same timestamp issue. if err != nil { return errors.Trace(err) } - c.minCommitTS = minCommitTS + // Plus 1 to avoid producing the same commit TS with previously committed transactions + c.minCommitTS = latestTS + 1 } // Calculate maxCommitTS if necessary if commitTSMayBeCalculated { From 1df03a6808b3d69add58a1c55d61146bca13b8f2 Mon Sep 17 00:00:00 2001 From: sylzd Date: Thu, 13 May 2021 19:51:38 +0800 Subject: [PATCH 02/16] planner: fix incorrect TableDual plan built from nulleq (#24596) --- util/ranger/points.go | 3 ++- util/ranger/ranger_test.go | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/util/ranger/points.go b/util/ranger/points.go index d98b548dcbb7a..9c33ccef7feb3 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -246,7 +246,8 @@ func (r *builder) buildFormBinOp(expr *expression.ScalarFunction) []*point { if col.RetType.EvalType() == types.ETString && (value.Kind() == types.KindString || value.Kind() == types.KindBinaryLiteral) { value.SetString(value.GetString(), col.RetType.Collate) } - if col.GetType().Tp == mysql.TypeYear { + // If nulleq with null value, values.ToInt64 will return err + if col.GetType().Tp == mysql.TypeYear && !value.IsNull() { // If the original value is adjusted, we need to change the condition. // For example, col < 2156. Since the max year is 2155, 2156 is changed to 2155. // col < 2155 is wrong. It should be col <= 2155. diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 194ba1e779dc3..575d7f196fbf9 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -1504,12 +1504,13 @@ func (s *testRangerSuite) TestIndexRangeForYear(c *C) { // test index range testKit.MustExec("DROP TABLE IF EXISTS t") testKit.MustExec("CREATE TABLE t (a year(4), key(a))") - testKit.MustExec("INSERT INTO t VALUES (1), (70), (99), (0), ('0')") + testKit.MustExec("INSERT INTO t VALUES (1), (70), (99), (0), ('0'), (NULL)") testKit.MustQuery("SELECT * FROM t WHERE a < 15698").Check(testkit.Rows("0", "1970", "1999", "2000", "2001")) testKit.MustQuery("SELECT * FROM t WHERE a <= 0").Check(testkit.Rows("0")) testKit.MustQuery("SELECT * FROM t WHERE a <= 1").Check(testkit.Rows("0", "1970", "1999", "2000", "2001")) testKit.MustQuery("SELECT * FROM t WHERE a < 2000").Check(testkit.Rows("0", "1970", "1999")) testKit.MustQuery("SELECT * FROM t WHERE a > -1").Check(testkit.Rows("0", "1970", "1999", "2000", "2001")) + testKit.MustQuery("SELECT * FROM t WHERE a <=> NULL").Check(testkit.Rows("")) tests := []struct { indexPos int From aecff1c42e3a05b911311f695db66fce228db369 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 13 May 2021 20:03:38 +0800 Subject: [PATCH 03/16] ranger: fix the case which could have duplicate ranges (#24590) --- util/ranger/ranger.go | 6 +++--- util/ranger/ranger_test.go | 10 +++++++++- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 4f1efef7a7d90..d69c3dbc02392 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -474,7 +474,7 @@ func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bo for _, ran := range ranges { lowTail := len(ran.LowVal) - 1 for i := 0; i < lowTail; i++ { - CutDatumByPrefixLen(&ran.LowVal[i], lengths[i], tp[i]) + hasCut = CutDatumByPrefixLen(&ran.LowVal[i], lengths[i], tp[i]) || hasCut } lowCut := CutDatumByPrefixLen(&ran.LowVal[lowTail], lengths[lowTail], tp[lowTail]) // If the length of the last column of LowVal is equal to the prefix length, LowExclude should be set false. @@ -485,13 +485,13 @@ func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bo } highTail := len(ran.HighVal) - 1 for i := 0; i < highTail; i++ { - CutDatumByPrefixLen(&ran.HighVal[i], lengths[i], tp[i]) + hasCut = CutDatumByPrefixLen(&ran.HighVal[i], lengths[i], tp[i]) || hasCut } highCut := CutDatumByPrefixLen(&ran.HighVal[highTail], lengths[highTail], tp[highTail]) if highCut { ran.HighExclude = false } - hasCut = lowCut || highCut + hasCut = hasCut || lowCut || highCut } return hasCut } diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 575d7f196fbf9..ed4722566033a 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -351,7 +351,8 @@ create table t( index idx_cb(c, a), index idx_d(d(2)), index idx_e(e(2)), - index idx_f(f) + index idx_f(f), + index idx_de(d(2), e) )`) tests := []struct { @@ -620,6 +621,13 @@ create table t( filterConds: "[like(test.t.f, @%, 92)]", resultStr: "[[NULL,+inf]]", }, + { + indexPos: 5, + exprStr: "d in ('aab', 'aac') and e = 'a'", + accessConds: "[in(test.t.d, aab, aac) eq(test.t.e, a)]", + filterConds: "[in(test.t.d, aab, aac)]", + resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", + }, } collate.SetNewCollationEnabledForTest(true) From 5d40ea459a4bf3fc862d193c246dda96da976fc4 Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Thu, 13 May 2021 20:43:38 +0800 Subject: [PATCH 04/16] executor, store: Pass the SQL digest down to pessimistic lock request (#24380) --- executor/executor.go | 3 + store/mockstore/unistore/tikv/deadlock.go | 32 ++++- store/mockstore/unistore/tikv/detector.go | 48 ++++++- .../mockstore/unistore/tikv/detector_test.go | 37 +++-- store/mockstore/unistore/tikv/errors.go | 2 + store/mockstore/unistore/tikv/mvcc.go | 12 +- store/mockstore/unistore/tikv/server.go | 12 +- store/tikv/kv/kv.go | 1 + store/tikv/pessimistic.go | 2 +- store/tikv/region_request_test.go | 8 +- store/tikv/tests/lock_test.go | 132 ++++++++++++++++++ store/tikv/txn.go | 21 +-- util/resourcegrouptag/resource_group_tag.go | 85 +++++++++++ .../resource_group_tag_test.go | 111 +++++++++++++++ 14 files changed, 465 insertions(+), 41 deletions(-) create mode 100644 util/resourcegrouptag/resource_group_tag.go create mode 100644 util/resourcegrouptag/resource_group_tag_test.go diff --git a/executor/executor.go b/executor/executor.go index e5d5d44efefe3..1666f6955bba9 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -62,6 +62,7 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/resourcegrouptag" "go.uber.org/zap" ) @@ -971,6 +972,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { } func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.LockCtx { + _, sqlDigest := seVars.StmtCtx.SQLDigest() return &tikvstore.LockCtx{ Killed: &seVars.Killed, ForUpdateTS: seVars.TxnCtx.GetForUpdateTS(), @@ -980,6 +982,7 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.Loc LockKeysDuration: &seVars.StmtCtx.LockKeysDuration, LockKeysCount: &seVars.StmtCtx.LockKeysCount, LockExpired: &seVars.TxnCtx.LockExpire, + ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(sqlDigest), } } diff --git a/store/mockstore/unistore/tikv/deadlock.go b/store/mockstore/unistore/tikv/deadlock.go index 6641a500e2cc1..de2eaf8fa61d9 100644 --- a/store/mockstore/unistore/tikv/deadlock.go +++ b/store/mockstore/unistore/tikv/deadlock.go @@ -44,7 +44,10 @@ type DetectorServer struct { func (ds *DetectorServer) Detect(req *deadlockPb.DeadlockRequest) *deadlockPb.DeadlockResponse { switch req.Tp { case deadlockPb.DeadlockRequestType_Detect: - err := ds.Detector.Detect(req.Entry.Txn, req.Entry.WaitForTxn, req.Entry.KeyHash) + err := ds.Detector.Detect(req.Entry.Txn, req.Entry.WaitForTxn, req.Entry.KeyHash, diagnosticContext{ + key: req.Entry.Key, + resourceGroupTag: req.Entry.ResourceGroupTag, + }) if err != nil { resp := convertErrToResp(err, req.Entry.Txn, req.Entry.WaitForTxn, req.Entry.KeyHash) return resp @@ -178,30 +181,35 @@ func (dt *DetectorClient) recvLoop(streamCli deadlockPb.Deadlock_DetectClient) { } func (dt *DetectorClient) handleRemoteTask(requestType deadlockPb.DeadlockRequestType, - txnTs uint64, waitForTxnTs uint64, keyHash uint64) { + txnTs uint64, waitForTxnTs uint64, keyHash uint64, diagCtx diagnosticContext) { detectReq := &deadlockPb.DeadlockRequest{} detectReq.Tp = requestType detectReq.Entry.Txn = txnTs detectReq.Entry.WaitForTxn = waitForTxnTs detectReq.Entry.KeyHash = keyHash + detectReq.Entry.Key = diagCtx.key + detectReq.Entry.ResourceGroupTag = diagCtx.resourceGroupTag dt.sendCh <- detectReq } // CleanUp processes cleaup task on local detector // user interfaces func (dt *DetectorClient) CleanUp(startTs uint64) { - dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUp, startTs, 0, 0) + dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUp, startTs, 0, 0, diagnosticContext{}) } // CleanUpWaitFor cleans up the specific wait edge in detector's wait map func (dt *DetectorClient) CleanUpWaitFor(txnTs, waitForTxn, keyHash uint64) { - dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUpWaitFor, txnTs, waitForTxn, keyHash) + dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUpWaitFor, txnTs, waitForTxn, keyHash, diagnosticContext{}) } // Detect post the detection request to local deadlock detector or remote first region leader, // the caller should use `waiter.ch` to receive possible deadlock response -func (dt *DetectorClient) Detect(txnTs uint64, waitForTxnTs uint64, keyHash uint64) { - dt.handleRemoteTask(deadlockPb.DeadlockRequestType_Detect, txnTs, waitForTxnTs, keyHash) +func (dt *DetectorClient) Detect(txnTs uint64, waitForTxnTs uint64, keyHash uint64, key []byte, resourceGroupTag []byte) { + dt.handleRemoteTask(deadlockPb.DeadlockRequestType_Detect, txnTs, waitForTxnTs, keyHash, diagnosticContext{ + key: key, + resourceGroupTag: resourceGroupTag, + }) } // convertErrToResp converts `ErrDeadlock` to `DeadlockResponse` proto type @@ -213,6 +221,18 @@ func convertErrToResp(errDeadlock *ErrDeadlock, txnTs, waitForTxnTs, keyHash uin resp := &deadlockPb.DeadlockResponse{} resp.Entry = entry resp.DeadlockKeyHash = errDeadlock.DeadlockKeyHash + + resp.WaitChain = make([]*deadlockPb.WaitForEntry, 0, len(errDeadlock.WaitChain)) + for _, item := range errDeadlock.WaitChain { + resp.WaitChain = append(resp.WaitChain, &deadlockPb.WaitForEntry{ + Txn: item.Txn, + WaitForTxn: item.WaitForTxn, + KeyHash: item.KeyHash, + Key: item.Key, + ResourceGroupTag: item.ResourceGroupTag, + }) + } + return resp } diff --git a/store/mockstore/unistore/tikv/detector.go b/store/mockstore/unistore/tikv/detector.go index 0273bed5fe6a8..a27adb3f35b6c 100644 --- a/store/mockstore/unistore/tikv/detector.go +++ b/store/mockstore/unistore/tikv/detector.go @@ -30,6 +30,7 @@ import ( "sync" "time" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/log" "go.uber.org/zap" ) @@ -54,6 +55,12 @@ type txnKeyHashPair struct { txn uint64 keyHash uint64 registerTime time.Time + diagCtx diagnosticContext +} + +type diagnosticContext struct { + key []byte + resourceGroupTag []byte } func (p *txnKeyHashPair) isExpired(ttl time.Duration, nowTime time.Time) bool { @@ -75,13 +82,27 @@ func NewDetector(ttl time.Duration, urgentSize uint64, expireInterval time.Durat } // Detect detects deadlock for the sourceTxn on a locked key. -func (d *Detector) Detect(sourceTxn, waitForTxn, keyHash uint64) *ErrDeadlock { +func (d *Detector) Detect(sourceTxn, waitForTxn, keyHash uint64, diagCtx diagnosticContext) *ErrDeadlock { d.lock.Lock() nowTime := time.Now() d.activeExpire(nowTime) err := d.doDetect(nowTime, sourceTxn, waitForTxn) if err == nil { - d.register(sourceTxn, waitForTxn, keyHash) + d.register(sourceTxn, waitForTxn, keyHash, diagCtx) + } else { + // Reverse the wait chain so that the order will be each one waiting for the next one, and append the current + // entry that finally caused the deadlock. + for i := 0; i < len(err.WaitChain)/2; i++ { + j := len(err.WaitChain) - i - 1 + err.WaitChain[i], err.WaitChain[j] = err.WaitChain[j], err.WaitChain[i] + } + err.WaitChain = append(err.WaitChain, &deadlockPB.WaitForEntry{ + Txn: sourceTxn, + Key: diagCtx.key, + KeyHash: keyHash, + ResourceGroupTag: diagCtx.resourceGroupTag, + WaitForTxn: waitForTxn, + }) } d.lock.Unlock() return err @@ -103,9 +124,26 @@ func (d *Detector) doDetect(nowTime time.Time, sourceTxn, waitForTxn uint64) *Er continue } if keyHashPair.txn == sourceTxn { - return &ErrDeadlock{DeadlockKeyHash: keyHashPair.keyHash} + return &ErrDeadlock{DeadlockKeyHash: keyHashPair.keyHash, + WaitChain: []*deadlockPB.WaitForEntry{ + { + Txn: waitForTxn, + Key: keyHashPair.diagCtx.key, + KeyHash: keyHashPair.keyHash, + ResourceGroupTag: keyHashPair.diagCtx.resourceGroupTag, + WaitForTxn: keyHashPair.txn, + }, + }, + } } if err := d.doDetect(nowTime, sourceTxn, keyHashPair.txn); err != nil { + err.WaitChain = append(err.WaitChain, &deadlockPB.WaitForEntry{ + Txn: waitForTxn, + Key: keyHashPair.diagCtx.key, + KeyHash: keyHashPair.keyHash, + ResourceGroupTag: keyHashPair.diagCtx.resourceGroupTag, + WaitForTxn: keyHashPair.txn, + }) return err } } @@ -115,9 +153,9 @@ func (d *Detector) doDetect(nowTime time.Time, sourceTxn, waitForTxn uint64) *Er return nil } -func (d *Detector) register(sourceTxn, waitForTxn, keyHash uint64) { +func (d *Detector) register(sourceTxn, waitForTxn, keyHash uint64, diagCtx diagnosticContext) { val := d.waitForMap[sourceTxn] - pair := txnKeyHashPair{txn: waitForTxn, keyHash: keyHash, registerTime: time.Now()} + pair := txnKeyHashPair{txn: waitForTxn, keyHash: keyHash, registerTime: time.Now(), diagCtx: diagCtx} if val == nil { newList := &txnList{txns: list.New()} newList.txns.PushBack(&pair) diff --git a/store/mockstore/unistore/tikv/detector_test.go b/store/mockstore/unistore/tikv/detector_test.go index 1768cc377ec7c..b0d3a074ff840 100644 --- a/store/mockstore/unistore/tikv/detector_test.go +++ b/store/mockstore/unistore/tikv/detector_test.go @@ -31,6 +31,7 @@ import ( "time" . "github.com/pingcap/check" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" ) func TestT(t *testing.T) { @@ -42,19 +43,38 @@ var _ = Suite(&testDeadlockSuite{}) type testDeadlockSuite struct{} func (s *testDeadlockSuite) TestDeadlock(c *C) { + makeDiagCtx := func(key string, resourceGroupTag string) diagnosticContext { + return diagnosticContext{ + key: []byte(key), + resourceGroupTag: []byte(resourceGroupTag), + } + } + checkWaitChainEntry := func(entry *deadlockPB.WaitForEntry, txn, waitForTxn uint64, key, resourceGroupTag string) { + c.Assert(entry.Txn, Equals, txn) + c.Assert(entry.WaitForTxn, Equals, waitForTxn) + c.Assert(string(entry.Key), Equals, key) + c.Assert(string(entry.ResourceGroupTag), Equals, resourceGroupTag) + } + ttl := 50 * time.Millisecond expireInterval := 100 * time.Millisecond urgentSize := uint64(1) detector := NewDetector(ttl, urgentSize, expireInterval) - err := detector.Detect(1, 2, 100) + err := detector.Detect(1, 2, 100, makeDiagCtx("k1", "tag1")) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(1)) - err = detector.Detect(2, 3, 200) + err = detector.Detect(2, 3, 200, makeDiagCtx("k2", "tag2")) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(2)) - err = detector.Detect(3, 1, 300) + err = detector.Detect(3, 1, 300, makeDiagCtx("k3", "tag3")) c.Assert(err, NotNil) c.Assert(err.Error(), Equals, fmt.Sprintf("deadlock")) + c.Assert(len(err.WaitChain), Equals, 3) + // The order of entries in the wait chain is specific: each item is waiting for the next one. + checkWaitChainEntry(err.WaitChain[0], 1, 2, "k1", "tag1") + checkWaitChainEntry(err.WaitChain[1], 2, 3, "k2", "tag2") + checkWaitChainEntry(err.WaitChain[2], 3, 1, "k3", "tag3") + c.Assert(detector.totalSize, Equals, uint64(2)) detector.CleanUp(2) list2 := detector.waitForMap[2] @@ -62,20 +82,21 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { c.Assert(detector.totalSize, Equals, uint64(1)) // After cycle is broken, no deadlock now. - err = detector.Detect(3, 1, 300) + diagCtx := diagnosticContext{} + err = detector.Detect(3, 1, 300, diagCtx) c.Assert(err, IsNil) list3 := detector.waitForMap[3] c.Assert(list3.txns.Len(), Equals, 1) c.Assert(detector.totalSize, Equals, uint64(2)) // Different keyHash grows the list. - err = detector.Detect(3, 1, 400) + err = detector.Detect(3, 1, 400, diagCtx) c.Assert(err, IsNil) c.Assert(list3.txns.Len(), Equals, 2) c.Assert(detector.totalSize, Equals, uint64(3)) // Same waitFor and key hash doesn't grow the list. - err = detector.Detect(3, 1, 400) + err = detector.Detect(3, 1, 400, diagCtx) c.Assert(err, IsNil) c.Assert(list3.txns.Len(), Equals, 2) c.Assert(detector.totalSize, Equals, uint64(3)) @@ -90,7 +111,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { // after 100ms, all entries expired, detect non exist edges time.Sleep(100 * time.Millisecond) - err = detector.Detect(100, 200, 100) + err = detector.Detect(100, 200, 100, diagCtx) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(1)) c.Assert(len(detector.waitForMap), Equals, 1) @@ -98,7 +119,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { // expired entry should not report deadlock, detect will remove this entry // not dependent on expire check interval time.Sleep(60 * time.Millisecond) - err = detector.Detect(200, 100, 200) + err = detector.Detect(200, 100, 200, diagCtx) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(1)) c.Assert(len(detector.waitForMap), Equals, 1) diff --git a/store/mockstore/unistore/tikv/errors.go b/store/mockstore/unistore/tikv/errors.go index 01d28fb73c896..98a70951871d5 100644 --- a/store/mockstore/unistore/tikv/errors.go +++ b/store/mockstore/unistore/tikv/errors.go @@ -16,6 +16,7 @@ package tikv import ( "fmt" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/mockstore/unistore/tikv/mvcc" ) @@ -90,6 +91,7 @@ type ErrDeadlock struct { LockKey []byte LockTS uint64 DeadlockKeyHash uint64 + WaitChain []*deadlockPB.WaitForEntry } func (e ErrDeadlock) Error() string { diff --git a/store/mockstore/unistore/tikv/mvcc.go b/store/mockstore/unistore/tikv/mvcc.go index 4e3eb4f7d7df8..fe5a75b549945 100644 --- a/store/mockstore/unistore/tikv/mvcc.go +++ b/store/mockstore/unistore/tikv/mvcc.go @@ -239,7 +239,11 @@ func (store *MVCCStore) PessimisticLock(reqCtx *requestCtx, req *kvrpcpb.Pessimi for _, m := range mutations { lock, err := store.checkConflictInLockStore(reqCtx, m, startTS) if err != nil { - return store.handleCheckPessimisticErr(startTS, err, req.IsFirstLock, req.WaitTimeout) + var resourceGroupTag []byte = nil + if req.Context != nil { + resourceGroupTag = req.Context.ResourceGroupTag + } + return store.handleCheckPessimisticErr(startTS, err, req.IsFirstLock, req.WaitTimeout, m.Key, resourceGroupTag) } if lock != nil { if lock.Op != uint8(kvrpcpb.Op_PessimisticLock) { @@ -533,11 +537,13 @@ func (store *MVCCStore) CheckSecondaryLocks(reqCtx *requestCtx, keys [][]byte, s func (store *MVCCStore) normalizeWaitTime(lockWaitTime int64) time.Duration { if lockWaitTime > store.conf.PessimisticTxn.WaitForLockTimeout { lockWaitTime = store.conf.PessimisticTxn.WaitForLockTimeout + } else if lockWaitTime == 0 { + lockWaitTime = store.conf.PessimisticTxn.WaitForLockTimeout } return time.Duration(lockWaitTime) * time.Millisecond } -func (store *MVCCStore) handleCheckPessimisticErr(startTS uint64, err error, isFirstLock bool, lockWaitTime int64) (*lockwaiter.Waiter, error) { +func (store *MVCCStore) handleCheckPessimisticErr(startTS uint64, err error, isFirstLock bool, lockWaitTime int64, key []byte, resourceGroupTag []byte) (*lockwaiter.Waiter, error) { if locked, ok := err.(*ErrLocked); ok { if lockWaitTime != lockwaiter.LockNoWait { keyHash := farm.Fingerprint64(locked.Key) @@ -546,7 +552,7 @@ func (store *MVCCStore) handleCheckPessimisticErr(startTS uint64, err error, isF log.S().Debugf("%d blocked by %d on key %d", startTS, lock.StartTS, keyHash) waiter := store.lockWaiterManager.NewWaiter(startTS, lock.StartTS, keyHash, waitTimeDuration) if !isFirstLock { - store.DeadlockDetectCli.Detect(startTS, lock.StartTS, keyHash) + store.DeadlockDetectCli.Detect(startTS, lock.StartTS, keyHash, key, resourceGroupTag) } return waiter, err } diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index adf3049330897..036d824a39ff9 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -217,6 +217,7 @@ func (svr *Server) KvPessimisticLock(ctx context.Context, req *kvrpcpb.Pessimist LockKey: errLocked.Key, LockTS: errLocked.Lock.StartTS, DeadlockKeyHash: result.DeadlockResp.DeadlockKeyHash, + WaitChain: result.DeadlockResp.WaitChain, } resp.Errors, resp.RegionError = convertToPBErrors(deadlockErr) return resp, nil @@ -845,11 +846,6 @@ func (svr *Server) ReadIndex(context.Context, *kvrpcpb.ReadIndexRequest) (*kvrpc return &kvrpcpb.ReadIndexResponse{}, nil } -// GetLockWaitInfo implements implements the tikvpb.TikvServer interface. -func (svr *Server) GetLockWaitInfo(ctx context.Context, _ *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { - return &kvrpcpb.GetLockWaitInfoResponse{}, nil -} - // transaction debugger commands. // MvccGetByKey implements implements the tikvpb.TikvServer interface. @@ -976,6 +972,11 @@ func (svr *Server) GetStoreSafeTS(context.Context, *kvrpcpb.StoreSafeTSRequest) return &kvrpcpb.StoreSafeTSResponse{}, nil } +// GetLockWaitInfo implements the tikvpb.TikvServer interface. +func (svr *Server) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + panic("unimplemented") +} + func convertToKeyError(err error) *kvrpcpb.KeyError { if err == nil { return nil @@ -1011,6 +1012,7 @@ func convertToKeyError(err error) *kvrpcpb.KeyError { LockKey: x.LockKey, LockTs: x.LockTS, DeadlockKeyHash: x.DeadlockKeyHash, + WaitChain: x.WaitChain, }, } case *ErrCommitExpire: diff --git a/store/tikv/kv/kv.go b/store/tikv/kv/kv.go index 2b7e87ecd2e47..8ba36a749db4f 100644 --- a/store/tikv/kv/kv.go +++ b/store/tikv/kv/kv.go @@ -27,4 +27,5 @@ type LockCtx struct { ValuesLock sync.Mutex LockExpired *uint32 Stats *util.LockKeysDetails + ResourceGroupTag []byte } diff --git a/store/tikv/pessimistic.go b/store/tikv/pessimistic.go index 445ced93ff904..2da8e93dad946 100644 --- a/store/tikv/pessimistic.go +++ b/store/tikv/pessimistic.go @@ -101,7 +101,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * WaitTimeout: action.LockWaitTime, ReturnValues: action.ReturnValues, MinCommitTs: c.forUpdateTS + 1, - }, pb.Context{Priority: c.priority, SyncLog: c.syncLog}) + }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: action.LockCtx.ResourceGroupTag}) lockWaitStartTime := action.WaitStartTime for { // if lockWaitTime set, refine the request `WaitTimeout` field based on timeout limit diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index f7fd2a149060d..bbe7ff8d19479 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -463,10 +463,6 @@ func (s *mockTikvGrpcServer) SplitRegion(context.Context, *kvrpcpb.SplitRegionRe return nil, errors.New("unreachable") } -func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { - return nil, errors.New("unreachable") -} - func (s *mockTikvGrpcServer) CoprocessorStream(*coprocessor.Request, tikvpb.Tikv_CoprocessorStreamServer) error { return errors.New("unreachable") } @@ -495,6 +491,10 @@ func (s *mockTikvGrpcServer) CoprocessorV2(context.Context, *coprocessor_v2.RawC return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + return nil, errors.New("unreachable") +} + func (s *testRegionRequestToSingleStoreSuite) TestNoReloadRegionForGrpcWhenCtxCanceled(c *C) { // prepare a mock tikv grpc server addr := "localhost:56341" diff --git a/store/tikv/tests/lock_test.go b/store/tikv/tests/lock_test.go index f32991877fefd..d64c1d102e6d1 100644 --- a/store/tikv/tests/lock_test.go +++ b/store/tikv/tests/lock_test.go @@ -19,13 +19,17 @@ import ( "fmt" "math" "runtime" + "sync" "time" . "github.com/pingcap/check" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) @@ -640,3 +644,131 @@ func (s *testLockSuite) TestBatchResolveTxnFallenBackFromAsyncCommit(c *C) { _, err = t3.Get(context.Background(), []byte("fb2")) c.Assert(tikverr.IsErrNotFound(err), IsTrue) } + +func (s *testLockSuite) TestDeadlockReportWaitChain(c *C) { + // Utilities to make the test logic clear and simple. + type txnWrapper struct { + tikv.TxnProbe + wg sync.WaitGroup + } + + makeLockCtx := func(txn *txnWrapper, resourceGroupTag string) *kv.LockCtx { + return &kv.LockCtx{ + ForUpdateTS: txn.StartTS(), + WaitStartTime: time.Now(), + LockWaitTime: 1000, + ResourceGroupTag: []byte(resourceGroupTag), + } + } + + // Prepares several transactions and each locks a key. + prepareTxns := func(num int) []*txnWrapper { + res := make([]*txnWrapper, 0, num) + for i := 0; i < num; i++ { + txnProbe, err := s.store.Begin() + c.Assert(err, IsNil) + txn := &txnWrapper{TxnProbe: txnProbe} + txn.SetPessimistic(true) + tag := fmt.Sprintf("tag-init%v", i) + key := []byte{'k', byte(i)} + err = txn.LockKeys(context.Background(), makeLockCtx(txn, tag), key) + c.Assert(err, IsNil) + + res = append(res, txn) + } + return res + } + + // Let the i-th trnasaction lock the key that has been locked by j-th transaction + tryLock := func(txns []*txnWrapper, i int, j int) error { + c.Logf("txn %v try locking %v", i, j) + txn := txns[i] + tag := fmt.Sprintf("tag-%v-%v", i, j) + key := []byte{'k', byte(j)} + return txn.LockKeys(context.Background(), makeLockCtx(txn, tag), key) + } + + // Asserts the i-th transaction waits for the j-th transaction. + makeWaitFor := func(txns []*txnWrapper, i int, j int) { + txns[i].wg.Add(1) + go func() { + defer txns[i].wg.Done() + err := tryLock(txns, i, j) + // After the lock being waited for is released, the transaction returns a WriteConflict error + // unconditionally, which is by design. + c.Assert(err, NotNil) + c.Logf("txn %v wait for %v finished, err: %s", i, j, err.Error()) + _, ok := errors.Cause(err).(*tikverr.ErrWriteConflict) + c.Assert(ok, IsTrue) + }() + } + + waitAndRollback := func(txns []*txnWrapper, i int) { + // It's expected that each transaction should be rolled back after its blocker, so that `Rollback` will not + // run when there's concurrent `LockKeys` running. + // If it's blocked on the `Wait` forever, it means the transaction's blocker is not rolled back. + c.Logf("rollback txn %v", i) + txns[i].wg.Wait() + err := txns[i].Rollback() + c.Assert(err, IsNil) + } + + // Check the given WaitForEntry is caused by txn[i] waiting for txn[j]. + checkWaitChainEntry := func(txns []*txnWrapper, entry *deadlockPB.WaitForEntry, i, j int) { + c.Assert(entry.Txn, Equals, txns[i].StartTS()) + c.Assert(entry.WaitForTxn, Equals, txns[j].StartTS()) + c.Assert(entry.Key, BytesEquals, []byte{'k', byte(j)}) + c.Assert(string(entry.ResourceGroupTag), Equals, fmt.Sprintf("tag-%v-%v", i, j)) + } + + c.Log("test case 1: 1->0->1") + + txns := prepareTxns(2) + + makeWaitFor(txns, 0, 1) + // Sleep for a while to make sure it has been blocked. + time.Sleep(time.Millisecond * 100) + + // txn2 tries locking k1 and encounters deadlock error. + err := tryLock(txns, 1, 0) + c.Assert(err, NotNil) + dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock) + c.Assert(ok, IsTrue) + + waitChain := dl.GetWaitChain() + c.Assert(len(waitChain), Equals, 2) + checkWaitChainEntry(txns, waitChain[0], 0, 1) + checkWaitChainEntry(txns, waitChain[1], 1, 0) + + // Each transaction should be rolled back after its blocker being rolled back + waitAndRollback(txns, 1) + waitAndRollback(txns, 0) + + c.Log("test case 2: 3->2->0->1->3") + txns = prepareTxns(4) + + makeWaitFor(txns, 0, 1) + makeWaitFor(txns, 2, 0) + makeWaitFor(txns, 1, 3) + // Sleep for a while to make sure it has been blocked. + time.Sleep(time.Millisecond * 100) + + err = tryLock(txns, 3, 2) + c.Assert(err, NotNil) + dl, ok = errors.Cause(err).(*tikverr.ErrDeadlock) + c.Assert(ok, IsTrue) + + waitChain = dl.GetWaitChain() + c.Assert(len(waitChain), Equals, 4) + c.Logf("wait chain: \n** %v\n**%v\n**%v\n**%v\n", waitChain[0], waitChain[1], waitChain[2], waitChain[3]) + checkWaitChainEntry(txns, waitChain[0], 2, 0) + checkWaitChainEntry(txns, waitChain[1], 0, 1) + checkWaitChainEntry(txns, waitChain[2], 1, 3) + checkWaitChainEntry(txns, waitChain[3], 3, 2) + + // Each transaction should be rolled back after its blocker being rolled back + waitAndRollback(txns, 3) + waitAndRollback(txns, 1) + waitAndRollback(txns, 0) + waitAndRollback(txns, 2) +} diff --git a/store/tikv/txn.go b/store/tikv/txn.go index cba091cbdc8da..20bf0491ed294 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -613,15 +613,18 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput // If there is only 1 key and lock fails, no need to do pessimistic rollback. if len(keys) > 1 || keyMayBeLocked { wg := txn.asyncPessimisticRollback(ctx, keys) - if dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok && hashInKeys(dl.DeadlockKeyHash, keys) { - dl.IsRetryable = true - // Wait for the pessimistic rollback to finish before we retry the statement. - wg.Wait() - // Sleep a little, wait for the other transaction that blocked by this transaction to acquire the lock. - time.Sleep(time.Millisecond * 5) - failpoint.Inject("SingleStmtDeadLockRetrySleep", func() { - time.Sleep(300 * time.Millisecond) - }) + if dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok { + logutil.Logger(ctx).Debug("deadlock error received", zap.Uint64("startTS", txn.startTS), zap.Stringer("deadlockInfo", dl)) + if hashInKeys(dl.DeadlockKeyHash, keys) { + dl.IsRetryable = true + // Wait for the pessimistic rollback to finish before we retry the statement. + wg.Wait() + // Sleep a little, wait for the other transaction that blocked by this transaction to acquire the lock. + time.Sleep(time.Millisecond * 5) + failpoint.Inject("SingleStmtDeadLockRetrySleep", func() { + time.Sleep(300 * time.Millisecond) + }) + } } } if assignedPrimaryKey { diff --git a/util/resourcegrouptag/resource_group_tag.go b/util/resourcegrouptag/resource_group_tag.go new file mode 100644 index 0000000000000..cacbf574b91fb --- /dev/null +++ b/util/resourcegrouptag/resource_group_tag.go @@ -0,0 +1,85 @@ +package resourcegrouptag + +import ( + "encoding/hex" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + +const ( + resourceGroupTagPrefixSQLDigest = byte(1) +) + +// EncodeResourceGroupTag encodes sqlDigest into resource group tag. +// A resource group tag can be carried in the Context field of TiKV requests, which is a byte array, and sent to TiKV as +// diagnostic information. Currently it contains only the SQL Digest, and the codec method is naive but extendable. +// This function doesn't return error. When there's some error, which can only be caused by unexpected format of the +// arguments, it simply returns an empty result. +// The format: +// +-----------+-----------------------+----------------------------+---------------+----------------+---- +// | version=1 | field1 prefix (1byte) | field1 content (var bytes) | field2 prefix | field2 content | ... +// +-----------+-----------------------+----------------------------+---------------+----------------+---- +// The `version` section marks the codec version, which makes it easier for changing the format in the future. +// Each field starts with a byte to mark what field it is, and the length of the content depends on the field's +// definition. +// Currently there's only one field (SQL Digest), and its content starts with a byte `B` describing it's length, and +// then follows by exactly `B` bytes. +func EncodeResourceGroupTag(sqlDigest string) []byte { + if len(sqlDigest) == 0 { + return nil + } + if len(sqlDigest) >= 512 { + logutil.BgLogger().Warn("failed to encode sql digest to resource group tag: length too long", zap.String("sqlDigest", sqlDigest)) + return nil + } + + res := make([]byte, 3+len(sqlDigest)/2) + + const encodingVersion = 1 + res[0] = encodingVersion + + res[1] = resourceGroupTagPrefixSQLDigest + // The SQL Digest is expected to be a hex string. Convert it back to bytes to save half of the memory. + res[2] = byte(len(sqlDigest) / 2) + _, err := hex.Decode(res[3:], []byte(sqlDigest)) + if err != nil { + logutil.BgLogger().Warn("failed to encode sql digest to resource group tag: invalid hex string", zap.String("sqlDigest", sqlDigest)) + return nil + } + + return res +} + +// DecodeResourceGroupTag decodes a resource group tag into various information contained in it. Currently it contains +// only the SQL Digest. +func DecodeResourceGroupTag(data []byte) (sqlDigest string, err error) { + if len(data) == 0 { + return "", nil + } + + encodingVersion := data[0] + if encodingVersion != 1 { + return "", errors.Errorf("unsupported resource group tag version %v", data[0]) + } + rem := data[1:] + + for len(rem) > 0 { + switch rem[0] { + case resourceGroupTagPrefixSQLDigest: + // There must be one more byte at rem[1] to represent the content's length, and the remaining bytes should + // not be shorter than the length specified by rem[1]. + if len(rem) < 2 || len(rem)-2 < int(rem[1]) { + return "", errors.Errorf("cannot parse resource group tag: field length mismatch, tag: %v", hex.EncodeToString(data)) + } + fieldLen := int(rem[1]) + sqlDigest = hex.EncodeToString(rem[2 : 2+fieldLen]) + rem = rem[2+fieldLen:] + default: + return "", errors.Errorf("resource group tag field not recognized, prefix: %v, tag: %v", rem[0], hex.EncodeToString(data)) + } + } + + return +} diff --git a/util/resourcegrouptag/resource_group_tag_test.go b/util/resourcegrouptag/resource_group_tag_test.go new file mode 100644 index 0000000000000..a979b92fce315 --- /dev/null +++ b/util/resourcegrouptag/resource_group_tag_test.go @@ -0,0 +1,111 @@ +// Copyright 2021 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 resourcegrouptag + +import ( + "math/rand" + "testing" + + . "github.com/pingcap/check" +) + +type testUtilsSuite struct{} + +var _ = Suite(&testUtilsSuite{}) + +func TestT(t *testing.T) { + TestingT(t) +} + +func (s *testUtilsSuite) TestResourceGroupTagEncoding(c *C) { + sqlDigest := "" + tag := EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + decodedSQLDigest, err := DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(len(decodedSQLDigest), Equals, 0) + + sqlDigest = "aa" + tag = EncodeResourceGroupTag(sqlDigest) + // version(1) + prefix(1) + length(1) + content(2hex -> 1byte) + c.Assert(len(tag), Equals, 4) + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, sqlDigest) + + sqlDigest = genRandHex(64) + tag = EncodeResourceGroupTag(sqlDigest) + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, sqlDigest) + + sqlDigest = genRandHex(510) + tag = EncodeResourceGroupTag(sqlDigest) + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, sqlDigest) + + // The max supported length is 255 bytes (510 hex digits). + sqlDigest = genRandHex(512) + tag = EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + + // A hex string can't have odd length. + sqlDigest = genRandHex(15) + tag = EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + + // Non-hexadecimal character is invalid + sqlDigest = "aabbccddgg" + tag = EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + + // A tag should start with a supported version + tag = []byte("\x00") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) + + // The fields should have format like `[prefix, length, content...]`, otherwise decoding it should returns error. + tag = []byte("\x01\x01") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) + + tag = []byte("\x01\x01\x02") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) + + tag = []byte("\x01\x01\x02AB") + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, "4142") + + tag = []byte("\x01\x01\x00") + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(len(decodedSQLDigest), Equals, 0) + + // Unsupported field + tag = []byte("\x01\x99") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) +} + +func genRandHex(length int) string { + const chars = "0123456789abcdef" + res := make([]byte, length) + for i := 0; i < length; i++ { + res[i] = chars[rand.Intn(len(chars))] + } + return string(res) +} From b7c22aa31d062b77a96493519a6b50ca9b627971 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 21:29:39 +0800 Subject: [PATCH 05/16] kv: remove UnionStore interface (#24625) --- ddl/index.go | 2 +- ddl/index_change_test.go | 2 +- executor/admin.go | 2 +- executor/admin_test.go | 44 +++++++++++++-------------- executor/distsql_test.go | 2 +- executor/executor_test.go | 4 +-- kv/interface_mock_test.go | 4 --- kv/kv.go | 2 -- kv/union_store.go | 17 +---------- store/driver/txn/txn_driver.go | 4 --- store/driver/txn/unionstore_driver.go | 36 ---------------------- table/index.go | 6 ++-- table/tables/index.go | 27 ++++++++-------- table/tables/index_test.go | 12 ++++---- table/tables/tables.go | 4 +-- util/admin/admin.go | 2 +- util/mock/context.go | 8 ----- 17 files changed, 54 insertions(+), 124 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index b1b4303d7a0f1..fdba6c65008f6 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1343,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) } diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index 0a54b6b25e694..dfdfc7111c372 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -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) } diff --git a/executor/admin.go b/executor/admin.go index 7e15a24e667ca..be46e39a1d2a3 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -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++ diff --git a/executor/admin_test.go b/executor/admin_test.go index 20095eb59a0ba..35e3d08345d63 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -135,7 +135,7 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { sc := s.ctx.GetSessionVars().StmtCtx txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(1), kv.IntHandle(1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(1), kv.IntHandle(1)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -158,7 +158,7 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(10), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(10), kv.IntHandle(10)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -172,15 +172,15 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(1), kv.IntHandle(1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(1), kv.IntHandle(1)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(2), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(2), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(3), kv.IntHandle(3)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(3), kv.IntHandle(3)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(10), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(10), kv.IntHandle(10)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(20), kv.IntHandle(20)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(20)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -236,7 +236,7 @@ func (s *testSuite5) TestClusteredIndexAdminRecoverIndex(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) cHandle := testutil.MustNewCommonHandle(c, "1", "3") - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(2), cHandle) + err = indexOpr.Delete(sc, txn, types.MakeDatums(2), cHandle) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -269,7 +269,7 @@ func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { sc := s.ctx.GetSessionVars().StmtCtx txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(idxValue), kv.IntHandle(idxValue)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(idxValue), kv.IntHandle(idxValue)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -345,13 +345,13 @@ func (s *testSuite5) TestAdminRecoverIndex1(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("1"), kv.IntHandle(1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("1"), kv.IntHandle(1)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("2"), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("2"), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("3"), kv.IntHandle(3)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("3"), kv.IntHandle(3)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("10"), kv.IntHandle(4)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("10"), kv.IntHandle(4)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -745,7 +745,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo) txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(i), kv.IntHandle(i)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i), kv.IntHandle(i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -784,7 +784,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(i+8), kv.IntHandle(i+8)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), kv.IntHandle(i+8)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -807,7 +807,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(i+8), kv.IntHandle(i)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), kv.IntHandle(i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -842,7 +842,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Index c2 is missing 11. txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(-10), kv.IntHandle(-1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(-10), kv.IntHandle(-1)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -873,7 +873,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Index c2 has two more values than table data: 10, 13, and these handles have correlative record. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(0), kv.IntHandle(0)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(0), kv.IntHandle(0)) c.Assert(err, IsNil) // Make sure the index value "19" is smaller "21". Then we scan to "19" before "21". _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(19), kv.IntHandle(10), nil) @@ -890,9 +890,9 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Two indices have the same handle. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(13), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(13), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(12), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(12), kv.IntHandle(2)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -906,7 +906,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(12), kv.IntHandle(2), nil) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(20), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(10)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -917,7 +917,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Recover records. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(19), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(19), kv.IntHandle(10)) c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(20), kv.IntHandle(10), nil) c.Assert(err, IsNil) diff --git a/executor/distsql_test.go b/executor/distsql_test.go index eca6e1d016b40..6f06fd550f0b4 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -241,7 +241,7 @@ func (s *testSuite3) TestInconsistentIndex(c *C) { for i := 0; i < 10; i++ { txn, err := s.store.Begin() c.Assert(err, IsNil) - err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), types.MakeDatums(i+10), kv.IntHandle(100+i)) + err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(i+10), kv.IntHandle(100+i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/executor/executor_test.go b/executor/executor_test.go index 3b168636606ed..7b433e71e60ac 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3852,9 +3852,9 @@ func (s *testSuite) TestCheckIndex(c *C) { // table data (handle, data): (1, 10), (2, 20), (4, 40) txn, err = s.store.Begin() c.Assert(err, IsNil) - err = idx.Delete(sc, txn.GetUnionStore(), types.MakeDatums(int64(30)), kv.IntHandle(3)) + err = idx.Delete(sc, txn, types.MakeDatums(int64(30)), kv.IntHandle(3)) c.Assert(err, IsNil) - err = idx.Delete(sc, txn.GetUnionStore(), types.MakeDatums(int64(20)), kv.IntHandle(2)) + err = idx.Delete(sc, txn, types.MakeDatums(int64(20)), kv.IntHandle(2)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index 2388c4f48b9f3..e1d41f1693088 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -106,10 +106,6 @@ func (t *mockTxn) GetSnapshot() Snapshot { return nil } -func (t *mockTxn) GetUnionStore() UnionStore { - return nil -} - func (t *mockTxn) NewStagingBuffer() MemBuffer { return nil } diff --git a/kv/kv.go b/kv/kv.go index 1fad79d641009..e5ab4eed6f812 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -174,8 +174,6 @@ type Transaction interface { GetMemBuffer() MemBuffer // GetSnapshot returns the Snapshot binding to this transaction. GetSnapshot() Snapshot - // GetUnionStore returns the UnionStore binding to this transaction. - GetUnionStore() UnionStore // SetVars sets variables to the transaction. SetVars(vars interface{}) // GetVars gets variables from the transaction. diff --git a/kv/union_store.go b/kv/union_store.go index 0e9a6768c5ebc..1d12e0a7fd37b 100644 --- a/kv/union_store.go +++ b/kv/union_store.go @@ -15,21 +15,6 @@ package kv // UnionStore is a store that wraps a snapshot for read and a MemBuffer for buffered write. // Also, it provides some transaction related utilities. +// TODO: Remove after upgrading BR. type UnionStore interface { - Retriever - - // HasPresumeKeyNotExists returns whether the key presumed key not exists error for the lazy check. - HasPresumeKeyNotExists(k Key) bool - // UnmarkPresumeKeyNotExists deletes the key presume key not exists error flag for the lazy check. - UnmarkPresumeKeyNotExists(k Key) - - // SetOption sets an option with a value, when val is nil, uses the default - // value of this option. - SetOption(opt int, val interface{}) - // DelOption deletes an option. - DelOption(opt int) - // GetOption gets an option. - GetOption(opt int) interface{} - // GetMemBuffer return the MemBuffer binding to this unionStore. - GetMemBuffer() MemBuffer } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 4d5ce77034312..72c1aac8c8e71 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -120,10 +120,6 @@ func (txn *tikvTxn) GetMemBuffer() kv.MemBuffer { return newMemBuffer(txn.KVTxn.GetMemBuffer()) } -func (txn *tikvTxn) GetUnionStore() kv.UnionStore { - return &tikvUnionStore{txn.KVTxn.GetUnionStore()} -} - func (txn *tikvTxn) SetOption(opt int, val interface{}) { switch opt { case kv.BinlogInfo: diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index 5a2f56bfe4233..b54bffa139700 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -111,42 +111,6 @@ func (m *memBuffer) SnapshotGetter() kv.Getter { return newKVGetter(m.MemDB.SnapshotGetter()) } -//tikvUnionStore implements kv.UnionStore -type tikvUnionStore struct { - *unionstore.KVUnionStore -} - -func (u *tikvUnionStore) GetMemBuffer() kv.MemBuffer { - return newMemBuffer(u.KVUnionStore.GetMemBuffer()) -} - -func (u *tikvUnionStore) Get(ctx context.Context, k kv.Key) ([]byte, error) { - data, err := u.KVUnionStore.Get(ctx, k) - return data, derr.ToTiDBErr(err) -} - -func (u *tikvUnionStore) HasPresumeKeyNotExists(k kv.Key) bool { - return u.KVUnionStore.HasPresumeKeyNotExists(k) -} - -func (u *tikvUnionStore) UnmarkPresumeKeyNotExists(k kv.Key) { - u.KVUnionStore.UnmarkPresumeKeyNotExists(k) -} - -func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { - it, err := u.KVUnionStore.Iter(k, upperBound) - return newKVIterator(it), derr.ToTiDBErr(err) -} - -// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. -// The returned iterator will iterate from greater key to smaller key. -// If k is nil, the returned iterator will be positioned at the last key. -// TODO: Add lower bound limit -func (u *tikvUnionStore) IterReverse(k kv.Key) (kv.Iterator, error) { - it, err := u.KVUnionStore.IterReverse(k) - return newKVIterator(it), derr.ToTiDBErr(err) -} - type tikvGetter struct { unionstore.Getter } diff --git a/table/index.go b/table/index.go index 5a9f32fbbfd3f..336efb7f574c2 100644 --- a/table/index.go +++ b/table/index.go @@ -66,11 +66,11 @@ type Index interface { // Create supports insert into statement. Create(ctx sessionctx.Context, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle, handleRestoreData []types.Datum, opts ...CreateIdxOptFunc) (kv.Handle, error) // Delete supports delete from statement. - Delete(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) error + Delete(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) error // Drop supports drop table, drop index statements. - Drop(us kv.UnionStore) error + Drop(txn kv.Transaction) error // Exist supports check index exists or not. - Exist(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) + Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) // GenIndexKey generates an index key. GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h kv.Handle, buf []byte) (key []byte, distinct bool, err error) // Seek supports where clause. diff --git a/table/tables/index.go b/table/tables/index.go index 8b4630d47f70d..ab6296390fbfd 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -184,9 +184,8 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue return nil, err } - us := txn.GetUnionStore() if !distinct || skipCheck || opt.Untouched { - err = us.GetMemBuffer().Set(key, idxVal) + err = txn.GetMemBuffer().Set(key, idxVal) return nil, err } @@ -202,18 +201,18 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue var value []byte if sctx.GetSessionVars().LazyCheckKeyNotExists() { - value, err = us.GetMemBuffer().Get(ctx, key) + value, err = txn.GetMemBuffer().Get(ctx, key) } else { - value, err = us.Get(ctx, key) + value, err = txn.Get(ctx, key) } if err != nil && !kv.IsErrNotFound(err) { return nil, err } if err != nil || len(value) == 0 { if sctx.GetSessionVars().LazyCheckKeyNotExists() && err != nil { - err = us.GetMemBuffer().SetWithFlags(key, idxVal, tikvstore.SetPresumeKeyNotExists) + err = txn.GetMemBuffer().SetWithFlags(key, idxVal, tikvstore.SetPresumeKeyNotExists) } else { - err = us.GetMemBuffer().Set(key, idxVal) + err = txn.GetMemBuffer().Set(key, idxVal) } return nil, err } @@ -226,22 +225,22 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue } // Delete removes the entry for handle h and indexedValues from KV index. -func (c *index) Delete(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) error { +func (c *index) Delete(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) error { key, distinct, err := c.GenIndexKey(sc, indexedValues, h, nil) if err != nil { return err } if distinct { - err = us.GetMemBuffer().DeleteWithFlags(key, tikvstore.SetNeedLocked) + err = txn.GetMemBuffer().DeleteWithFlags(key, tikvstore.SetNeedLocked) } else { - err = us.GetMemBuffer().Delete(key) + err = txn.GetMemBuffer().Delete(key) } return err } // Drop removes the KV index from store. -func (c *index) Drop(us kv.UnionStore) error { - it, err := us.Iter(c.prefix, c.prefix.PrefixNext()) +func (c *index) Drop(txn kv.Transaction) error { + it, err := txn.Iter(c.prefix, c.prefix.PrefixNext()) if err != nil { return err } @@ -252,7 +251,7 @@ func (c *index) Drop(us kv.UnionStore) error { if !it.Key().HasPrefix(c.prefix) { break } - err := us.GetMemBuffer().Delete(it.Key()) + err := txn.GetMemBuffer().Delete(it.Key()) if err != nil { return err } @@ -298,13 +297,13 @@ func (c *index) SeekFirst(r kv.Retriever) (iter table.IndexIterator, err error) return &indexIter{it: it, idx: c, prefix: c.prefix, colInfos: colInfos, tps: tps}, nil } -func (c *index) Exist(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) { +func (c *index) Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) { key, distinct, err := c.GenIndexKey(sc, indexedValues, h, nil) if err != nil { return false, nil, err } - value, err := us.Get(context.TODO(), key) + value, err := txn.Get(context.TODO(), key) if kv.IsErrNotFound(err) { return false, nil, nil } diff --git a/table/tables/index_test.go b/table/tables/index_test.go index 9345e86bab185..2c0a417746d42 100644 --- a/table/tables/index_test.go +++ b/table/tables/index_test.go @@ -104,15 +104,15 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(h.IntValue(), Equals, int64(1)) it.Close() sc := &stmtctx.StatementContext{TimeZone: time.Local} - exist, _, err := index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(100)) + exist, _, err := index.Exist(sc, txn, values, kv.IntHandle(100)) c.Assert(err, IsNil) c.Assert(exist, IsFalse) - exist, _, err = index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(1)) + exist, _, err = index.Exist(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) c.Assert(exist, IsTrue) - err = index.Delete(sc, txn.GetUnionStore(), values, kv.IntHandle(1)) + err = index.Delete(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) it, err = index.SeekFirst(txn) @@ -132,7 +132,7 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(err, IsNil) c.Assert(hit, IsFalse) - err = index.Drop(txn.GetUnionStore()) + err = index.Drop(txn) c.Assert(err, IsNil) it, hit, err = index.Seek(sc, txn, values) @@ -194,12 +194,12 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(h.IntValue(), Equals, int64(1)) it.Close() - exist, h, err = index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(1)) + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) c.Assert(h.IntValue(), Equals, int64(1)) c.Assert(exist, IsTrue) - exist, h, err = index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(2)) + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(2)) c.Assert(err, NotNil) c.Assert(h.IntValue(), Equals, int64(1)) c.Assert(exist, IsTrue) diff --git a/table/tables/tables.go b/table/tables/tables.go index a6a4180401d6a..37e6ab1103a89 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1152,7 +1152,7 @@ func (t *TableCommon) removeRowIndices(ctx sessionctx.Context, h kv.Handle, rec logutil.BgLogger().Info("remove row index failed", zap.Any("index", v.Meta()), zap.Uint64("txnStartTS", txn.StartTS()), zap.String("handle", h.String()), zap.Any("record", rec), zap.Error(err)) return err } - if err = v.Delete(ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), vals, h); err != nil { + if err = v.Delete(ctx.GetSessionVars().StmtCtx, txn, vals, h); err != nil { if v.Meta().State != model.StatePublic && kv.ErrNotExist.Equal(err) { // If the index is not in public state, we may have not created the index, // or already deleted the index, so skip ErrNotExist error. @@ -1167,7 +1167,7 @@ func (t *TableCommon) removeRowIndices(ctx sessionctx.Context, h kv.Handle, rec // removeRowIndex implements table.Table RemoveRowIndex interface. func (t *TableCommon) removeRowIndex(sc *stmtctx.StatementContext, h kv.Handle, vals []types.Datum, idx table.Index, txn kv.Transaction) error { - return idx.Delete(sc, txn.GetUnionStore(), vals, h) + return idx.Delete(sc, txn, vals, h) } // buildIndexForRow implements table.Table BuildIndexForRow interface. diff --git a/util/admin/admin.go b/util/admin/admin.go index 20217a53c1b6d..608040a89d6b0 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -389,7 +389,7 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table vals1[i] = colDefVal } } - isExist, h2, err := idx.Exist(sc, txn.GetUnionStore(), vals1, h1) + isExist, h2, err := idx.Exist(sc, txn, vals1, h1) if kv.ErrKeyExists.Equal(err) { record1 := &RecordData{Handle: h1, Values: vals1} record2 := &RecordData{Handle: h2, Values: vals1} diff --git a/util/mock/context.go b/util/mock/context.go index 4b329e0ff1f55..d6a5f1d913902 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -59,14 +59,6 @@ func (txn *wrapTxn) Valid() bool { return txn.Transaction != nil && txn.Transaction.Valid() } -// GetUnionStore implements GetUnionStore -func (txn *wrapTxn) GetUnionStore() kv.UnionStore { - if txn.Transaction == nil { - return nil - } - return txn.Transaction.GetUnionStore() -} - func (txn *wrapTxn) CacheTableInfo(id int64, info *model.TableInfo) { if txn.Transaction == nil { return From 18cbfaac15f8478902726e1e64971aa96b862462 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 14 May 2021 09:15:38 +0800 Subject: [PATCH 06/16] *: enable gosimple linter (#24617) --- Makefile | 1 + ddl/db_test.go | 8 +++---- ddl/ddl_api.go | 2 +- ddl/serial_test.go | 10 ++++---- executor/aggfuncs/func_percentile.go | 4 +--- executor/concurrent_map.go | 1 - executor/delete.go | 5 +--- executor/executor_test.go | 4 ++-- executor/explainfor_test.go | 24 +++++++++---------- executor/grant_test.go | 4 ++-- executor/infoschema_reader.go | 1 - executor/insert_test.go | 2 +- executor/join.go | 2 +- executor/memtable_reader.go | 4 ++-- executor/merge_join_test.go | 4 ++-- executor/parallel_apply_test.go | 1 - executor/partition_table_test.go | 2 +- executor/show.go | 8 +++---- expression/integration_test.go | 2 +- go.mod | 2 +- go.sum | 5 ++-- infoschema/metrics_schema.go | 14 +++-------- planner/cascades/implementation_rules.go | 10 ++------ planner/cascades/transformation_rules.go | 5 +--- planner/core/exhaust_physical_plans.go | 1 + planner/core/partition_pruner_test.go | 8 +++---- planner/core/pb_to_plan.go | 22 ++++++++--------- planner/core/physical_plans.go | 4 +--- planner/core/rule_partition_processor.go | 8 +++---- planner/core/stringer.go | 12 +++++----- planner/core/util.go | 2 +- plugin/conn_ip_example/conn_ip_example.go | 5 ---- server/http_handler_test.go | 6 ++--- server/server_test.go | 6 ++--- sessionctx/binloginfo/binloginfo.go | 20 +++++++--------- statistics/cmsketch.go | 9 +++---- statistics/feedback.go | 2 -- statistics/handle/handle.go | 1 - statistics/handle/handle_test.go | 2 +- statistics/handle/update_test.go | 4 ++-- store/copr/mpp.go | 8 ++----- .../unistore/cophandler/closure_exec.go | 4 +--- .../unistore/tikv/dbreader/db_reader.go | 1 - store/mockstore/unistore/tikv/detector.go | 5 +--- .../mockstore/unistore/tikv/detector_test.go | 3 +-- store/tikv/region_request.go | 5 +--- store/tikv/txn.go | 2 +- util/chunk/row_container.go | 2 -- util/execdetails/execdetails.go | 4 +--- util/expensivequery/memory_usage_alarm.go | 1 - util/profile/trackerRecorder.go | 10 ++++---- util/rowcodec/decoder.go | 5 +--- 52 files changed, 111 insertions(+), 176 deletions(-) diff --git a/Makefile b/Makefile index dd0e5f521acab..d6681083dc284 100644 --- a/Makefile +++ b/Makefile @@ -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 diff --git a/ddl/db_test.go b/ddl/db_test.go index eddad6d0d635d..ecef0d0144215 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -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. @@ -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 { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e6d77c9e674e9..528a6087638fc 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -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) diff --git a/ddl/serial_test.go b/ddl/serial_test.go index e532bfc2352af..a8d245007f39c 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -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 diff --git a/executor/aggfuncs/func_percentile.go b/executor/aggfuncs/func_percentile.go index 31855f791fb0f..ff13392276c61 100644 --- a/executor/aggfuncs/func_percentile.go +++ b/executor/aggfuncs/func_percentile.go @@ -53,9 +53,7 @@ func (e *basePercentile) AllocPartialResult() (pr PartialResult, memDelta int64) return } -func (e *basePercentile) ResetPartialResult(pr PartialResult) { - return -} +func (e *basePercentile) ResetPartialResult(pr PartialResult) {} func (e *basePercentile) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) (memDelta int64, err error) { return diff --git a/executor/concurrent_map.go b/executor/concurrent_map.go index 27f13a4f21dcb..3d6ef1082f605 100644 --- a/executor/concurrent_map.go +++ b/executor/concurrent_map.go @@ -56,7 +56,6 @@ func (m concurrentMap) Insert(key uint64, value *entry) { shard.items[key] = value } shard.Unlock() - return } // UpsertCb : Callback to return new element to be inserted into the map diff --git a/executor/delete.go b/executor/delete.go index 1fe9c26b8ac82..16f0e9c421b19 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -188,10 +188,7 @@ func (e *DeleteExec) removeRowsInTblRowMap(tblRowMap tableRowMapType) error { var err error rowMap.Range(func(h kv.Handle, val interface{}) bool { err = e.removeRow(e.ctx, e.tblID2Table[id], h, val.([]types.Datum)) - if err != nil { - return false - } - return true + return err == nil }) if err != nil { return err diff --git a/executor/executor_test.go b/executor/executor_test.go index 7b433e71e60ac..65af164174e6f 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5809,7 +5809,7 @@ func (s *testRecoverTable) TestRecoverTable(c *C) { // Test for recover one table multiple time. tk.MustExec("drop table t_recover") tk.MustExec("flashback table t_recover to t_recover_tmp") - _, err = tk.Exec(fmt.Sprintf("recover table t_recover")) + _, err = tk.Exec("recover table t_recover") c.Assert(infoschema.ErrTableExists.Equal(err), IsTrue) gcEnable, err := gcutil.CheckGCEnable(tk.Se) @@ -5876,7 +5876,7 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { tk.MustQuery("select a,_tidb_rowid from t_flashback2;").Check(testkit.Rows("1 1", "2 2", "3 3", "4 5001", "5 5002", "6 5003", "7 10001", "8 10002", "9 10003")) // Test for flashback one table multiple time. - _, err = tk.Exec(fmt.Sprintf("flashback table t_flashback to t_flashback4")) + _, err = tk.Exec("flashback table t_flashback to t_flashback4") c.Assert(infoschema.ErrTableExists.Equal(err), IsTrue) // Test for flashback truncated table to new table. diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index e29a7a3e24cee..46df545b1ff47 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -181,11 +181,11 @@ func (s *testSuite) TestExplainMemTablePredicate(c *C) { func (s *testSuite) TestExplainClusterTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where type in ('tikv', 'tidb')")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where type in ('tikv', 'tidb')").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG node_types:["tidb","tikv"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where instance='192.168.1.7:2379'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where instance='192.168.1.7:2379'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG instances:["192.168.1.7:2379"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where type='tidb' and instance='192.168.1.7:2379'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where type='tidb' and instance='192.168.1.7:2379'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG node_types:["tidb"], instances:["192.168.1.7:2379"]`)) } @@ -203,11 +203,11 @@ func (s *testSuite) TestInspectionResultTable(c *C) { func (s *testSuite) TestInspectionRuleTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES node_types:["inspection"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection' or type='summary'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection' or type='summary'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES node_types:["inspection","summary"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection' and type='summary'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection' and type='summary'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES skip_request: true`)) } @@ -355,12 +355,12 @@ func (s *testPrepareSerialSuite) TestExplainDotForQuery(c *C) { func (s *testSuite) TestExplainTableStorage(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"]"))) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_NAME = 'schemata'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]"))) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]"))) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"]")) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_NAME = 'schemata'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]")) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]")) } func (s *testSuite) TestInspectionSummaryTable(c *C) { diff --git a/executor/grant_test.go b/executor/grant_test.go index 13686494feddc..0e41c75e44353 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -68,7 +68,7 @@ func (s *testSuite3) TestGrantDBScope(c *C) { createUserSQL := `CREATE USER 'testDB'@'localhost' IDENTIFIED BY '123';` tk.MustExec(createUserSQL) // Make sure all the db privs for new user is empty. - sql := fmt.Sprintf("SELECT * FROM mysql.db WHERE User=\"testDB\" and host=\"localhost\"") + sql := `SELECT * FROM mysql.db WHERE User="testDB" and host="localhost"` tk.MustQuery(sql).Check(testkit.Rows()) // Grant each priv to the user. @@ -101,7 +101,7 @@ func (s *testSuite3) TestWithGrantOption(c *C) { createUserSQL := `CREATE USER 'testWithGrant'@'localhost' IDENTIFIED BY '123';` tk.MustExec(createUserSQL) // Make sure all the db privs for new user is empty. - sql := fmt.Sprintf("SELECT * FROM mysql.db WHERE User=\"testWithGrant\" and host=\"localhost\"") + sql := `SELECT * FROM mysql.db WHERE User="testWithGrant" and host="localhost"` tk.MustQuery(sql).Check(testkit.Rows()) // Grant select priv to the user, with grant option. diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index ae338bdd644d2..4d56cc55accac 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1856,7 +1856,6 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, s } } e.rows = rows - return } func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, tableName string) error { diff --git a/executor/insert_test.go b/executor/insert_test.go index ffcfdc214bdb9..351b337bd84ae 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -426,7 +426,7 @@ func (s *testSuite3) TestInsertDateTimeWithTimeZone(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (ts timestamp)") tk.MustExec("insert into t values ('2020-10-22T12:00:00Z'), ('2020-10-22T13:00:00Z'), ('2020-10-22T14:00:00Z')") - tk.MustQuery(fmt.Sprintf("select count(*) from t where ts > '2020-10-22T12:00:00Z'")).Check(testkit.Rows("2")) + tk.MustQuery("select count(*) from t where ts > '2020-10-22T12:00:00Z'").Check(testkit.Rows("2")) // test for datetime with fsp fspCases := []struct { diff --git a/executor/join.go b/executor/join.go index c1a8045aba9a3..1a3f62de47ac1 100644 --- a/executor/join.go +++ b/executor/join.go @@ -1078,7 +1078,7 @@ func (e *joinRuntimeStats) String() string { if e.cache.useCache { buf.WriteString(fmt.Sprintf(", cache:ON, cacheHitRatio:%.3f%%", e.cache.hitRatio*100)) } else { - buf.WriteString(fmt.Sprintf(", cache:OFF")) + buf.WriteString(", cache:OFF") } } if e.hasHashStat { diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index 24e2001131580..76ea478d91e71 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -230,9 +230,9 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String continue } var str string - switch val.(type) { + switch val := val.(type) { case string: // remove quotes - str = val.(string) + str = val default: tmp, err := json.Marshal(val) if err != nil { diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 94b9c33a5a9d7..cf5328fc93263 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -277,7 +277,6 @@ func (s *testSerialSuite1) TestShuffleMergeJoinInDisk(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), Greater, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.BytesConsumed(), Equals, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), Greater, int64(0)) - return } func (s *testSerialSuite1) TestMergeJoinInDisk(c *C) { defer config.RestoreFunc()() @@ -313,7 +312,6 @@ func (s *testSerialSuite1) TestMergeJoinInDisk(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), Greater, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.BytesConsumed(), Equals, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), Greater, int64(0)) - return } func (s *testSuite2) TestMergeJoin(c *C) { @@ -726,6 +724,7 @@ func (s *testSuite2) TestMergeJoinDifferentTypes(c *C) { } // TestVectorizedMergeJoin is used to test vectorized merge join with some corner cases. +//nolint:gosimple // generates false positive fmt.Sprintf warnings which keep aligned func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -841,6 +840,7 @@ func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) { } // TestVectorizedShuffleMergeJoin is used to test vectorized shuffle merge join with some corner cases. +//nolint:gosimple // generates false positive fmt.Sprintf warnings which keep aligned func (s *testSuiteJoin3) TestVectorizedShuffleMergeJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@session.tidb_merge_join_concurrency = 4;") diff --git a/executor/parallel_apply_test.go b/executor/parallel_apply_test.go index b849d3d961043..c0ecb19783273 100644 --- a/executor/parallel_apply_test.go +++ b/executor/parallel_apply_test.go @@ -43,7 +43,6 @@ func checkApplyPlan(c *C, tk *testkit.TestKit, sql string, parallel int) { } } c.Assert(containApply, IsTrue) - return } func (s *testSuite) TestParallelApply(c *C) { diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 5be39c3a04d54..ab5a19ca823e9 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -263,7 +263,7 @@ func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { } // test empty PointGet - queryHash := fmt.Sprintf("select a from thash where a=200") + queryHash := "select a from thash where a=200" c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used tk.MustQuery(queryHash).Check(testkit.Rows()) diff --git a/executor/show.go b/executor/show.go index c5df3bb290268..2bd9b786fcffb 100644 --- a/executor/show.go +++ b/executor/show.go @@ -862,10 +862,10 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } if pkCol != nil { - // If PKIsHanle, pk info is not in tb.Indices(). We should handle it here. + // If PKIsHandle, pk info is not in tb.Indices(). We should handle it here. buf.WriteString(",\n") fmt.Fprintf(buf, " PRIMARY KEY (%s)", stringutil.Escape(pkCol.Name.O, sqlMode)) - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] CLUSTERED */")) + buf.WriteString(" /*T![clustered_index] CLUSTERED */") } publicIndices := make([]*model.IndexInfo, 0, len(tableInfo.Indices)) @@ -906,9 +906,9 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } if idxInfo.Primary { if tableInfo.PKIsHandle || tableInfo.IsCommonHandle { - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] CLUSTERED */")) + buf.WriteString(" /*T![clustered_index] CLUSTERED */") } else { - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] NONCLUSTERED */")) + buf.WriteString(" /*T![clustered_index] NONCLUSTERED */") } } if i != len(publicIndices)-1 { diff --git a/expression/integration_test.go b/expression/integration_test.go index a3d983069cce9..8e0f70823f327 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -5833,7 +5833,7 @@ func (s *testIntegrationSuite) TestDecodetoChunkReuse(c *C) { tk.MustExec("create table chk (a int,b varchar(20))") for i := 0; i < 200; i++ { if i%5 == 0 { - tk.MustExec(fmt.Sprintf("insert chk values (NULL,NULL)")) + tk.MustExec("insert chk values (NULL,NULL)") continue } tk.MustExec(fmt.Sprintf("insert chk values (%d,'%s')", i, strconv.Itoa(i))) diff --git a/go.mod b/go.mod index c4b668a81dcf1..fe8e08ae42e47 100644 --- a/go.mod +++ b/go.mod @@ -82,7 +82,7 @@ require ( gopkg.in/natefinch/lumberjack.v2 v2.0.0 gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 // indirect - honnef.co/go/tools v0.1.3 // indirect + honnef.co/go/tools v0.1.4 // indirect modernc.org/mathutil v1.2.2 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 diff --git a/go.sum b/go.sum index 212917e414404..3ee71da011a54 100644 --- a/go.sum +++ b/go.sum @@ -500,7 +500,6 @@ github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQD github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= -github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A= @@ -933,8 +932,8 @@ honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.1.3 h1:qTakTkI6ni6LFD5sBwwsdSO+AQqbSIxOauHTTQKZ/7o= -honnef.co/go/tools v0.1.3/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= +honnef.co/go/tools v0.1.4 h1:SadWOkti5uVN1FAMgxn165+Mw00fuQKyk4Gyn/inxNQ= +honnef.co/go/tools v0.1.4/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= modernc.org/mathutil v1.2.2 h1:+yFk8hBprV+4c0U9GjFtL+dV3N8hOJ8JCituQcMShFY= modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= diff --git a/infoschema/metrics_schema.go b/infoschema/metrics_schema.go index 49a57e4ac9eeb..3b4654f90f7f2 100644 --- a/infoschema/metrics_schema.go +++ b/infoschema/metrics_schema.go @@ -100,17 +100,9 @@ func (def *MetricTableDef) genColumnInfos() []columnInfo { // GenPromQL generates the promQL. func (def *MetricTableDef) GenPromQL(sctx sessionctx.Context, labels map[string]set.StringSet, quantile float64) string { promQL := def.PromQL - if strings.Contains(promQL, promQLQuantileKey) { - promQL = strings.Replace(promQL, promQLQuantileKey, strconv.FormatFloat(quantile, 'f', -1, 64), -1) - } - - if strings.Contains(promQL, promQLLabelConditionKey) { - promQL = strings.Replace(promQL, promQLLabelConditionKey, def.genLabelCondition(labels), -1) - } - - if strings.Contains(promQL, promQRangeDurationKey) { - promQL = strings.Replace(promQL, promQRangeDurationKey, strconv.FormatInt(sctx.GetSessionVars().MetricSchemaRangeDuration, 10)+"s", -1) - } + promQL = strings.Replace(promQL, promQLQuantileKey, strconv.FormatFloat(quantile, 'f', -1, 64), -1) + promQL = strings.Replace(promQL, promQLLabelConditionKey, def.genLabelCondition(labels), -1) + promQL = strings.Replace(promQL, promQRangeDurationKey, strconv.FormatInt(sctx.GetSessionVars().MetricSchemaRangeDuration, 10)+"s", -1) return promQL } diff --git a/planner/cascades/implementation_rules.go b/planner/cascades/implementation_rules.go index d7a08b4fabaab..56c2141cfb213 100644 --- a/planner/cascades/implementation_rules.go +++ b/planner/cascades/implementation_rules.go @@ -95,10 +95,7 @@ type ImplTableDual struct { // Match implements ImplementationRule Match interface. func (r *ImplTableDual) Match(expr *memo.GroupExpr, prop *property.PhysicalProperty) (matched bool) { - if !prop.IsEmpty() { - return false - } - return true + return prop.IsEmpty() } // OnImplement implements ImplementationRule OnImplement interface. @@ -116,10 +113,7 @@ type ImplMemTableScan struct { // Match implements ImplementationRule Match interface. func (r *ImplMemTableScan) Match(expr *memo.GroupExpr, prop *property.PhysicalProperty) (matched bool) { - if !prop.IsEmpty() { - return false - } - return true + return prop.IsEmpty() } // OnImplement implements ImplementationRule OnImplement interface. diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 6d23e063f5877..9961509299a52 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -1503,10 +1503,7 @@ func NewRuleMergeAggregationProjection() Transformation { // Match implements Transformation interface. func (r *MergeAggregationProjection) Match(old *memo.ExprIter) bool { proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - if plannercore.ExprsHasSideEffects(proj.Exprs) { - return false - } - return true + return !plannercore.ExprsHasSideEffects(proj.Exprs) } // OnTransform implements Transformation interface. diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index cd227657a75d9..7c7493eedd5c5 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1416,6 +1416,7 @@ func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges []*ranger.Range, p func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool) (ranges []*ranger.Range, emptyRange bool, err error) { pointLength := matchedKeyCnt + len(eqAndInFuncs) + //nolint:gosimple // false positive unnecessary nil check if nextColRange != nil { for _, colRan := range nextColRange { // The range's exclude status is the same with last col's. diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index dd614239de55d..e151efdecce03 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -327,7 +327,7 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPrunerRandom(c *C) { tk1.MustExec(insert) // Test query without condition - query := fmt.Sprintf("select * from t1 order by id,a,b") + query := "select * from t1 order by id,a,b" tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) } @@ -467,9 +467,9 @@ func (s *testPartitionPruneSuit) TestRangePartitionPredicatePruner(c *C) { tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec(`create table t (a int(11) default null) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin partition by range(a) ( - partition p0 values less than (1), - partition p1 values less than (2), - partition p2 values less than (3), + partition p0 values less than (1), + partition p1 values less than (2), + partition p2 values less than (3), partition p_max values less than (maxvalue));`) var input []string diff --git a/planner/core/pb_to_plan.go b/planner/core/pb_to_plan.go index 3cfaf6708affe..a453596a0f288 100644 --- a/planner/core/pb_to_plan.go +++ b/planner/core/pb_to_plan.go @@ -255,15 +255,15 @@ func (b *PBPlanBuilder) pbToKill(e *tipb.Executor) (PhysicalPlan, error) { return &PhysicalSimpleWrapper{Inner: simple}, nil } -func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { - if p == nil { - return predicates, p +func (b *PBPlanBuilder) predicatePushDown(physicalPlan PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { + if physicalPlan == nil { + return predicates, physicalPlan } - switch p.(type) { + switch plan := physicalPlan.(type) { case *PhysicalMemTable: - memTable := p.(*PhysicalMemTable) + memTable := plan if memTable.Extractor == nil { - return predicates, p + return predicates, plan } names := make([]*types.FieldName, 0, len(memTable.Columns)) for _, col := range memTable.Columns { @@ -284,8 +284,8 @@ func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expressio predicates = memTable.Extractor.Extract(b.sctx, memTable.schema, names, predicates) return predicates, memTable case *PhysicalSelection: - selection := p.(*PhysicalSelection) - conditions, child := b.predicatePushDown(p.Children()[0], selection.Conditions) + selection := plan + conditions, child := b.predicatePushDown(plan.Children()[0], selection.Conditions) if len(conditions) > 0 { selection.Conditions = conditions selection.SetChildren(child) @@ -293,10 +293,10 @@ func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expressio } return predicates, child default: - if children := p.Children(); len(children) > 0 { + if children := plan.Children(); len(children) > 0 { _, child := b.predicatePushDown(children[0], nil) - p.SetChildren(child) + plan.SetChildren(child) } - return predicates, p + return predicates, plan } } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 43304971b4680..f4e6769ee8d70 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -911,9 +911,7 @@ func (p *PhysicalMergeJoin) Clone() (PhysicalPlan, error) { return nil, err } cloned.basePhysicalJoin = *base - for _, cf := range p.CompareFuncs { - cloned.CompareFuncs = append(cloned.CompareFuncs, cf) - } + cloned.CompareFuncs = append(cloned.CompareFuncs, p.CompareFuncs...) cloned.Desc = p.Desc return cloned, nil } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 57858679b7795..17f8c5b2a88b1 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -297,7 +297,7 @@ func (s *partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types. }) continue } - return nil, errors.New(fmt.Sprintf("information of column %v is not found", colExpr.String())) + return nil, fmt.Errorf("information of column %v is not found", colExpr.String()) } return names, nil } @@ -1345,9 +1345,9 @@ func appendWarnForUnknownPartitions(ctx sessionctx.Context, hintName string, unk if len(unknownPartitions) == 0 { return } - ctx.GetSessionVars().StmtCtx.AppendWarning( - errors.New(fmt.Sprintf("Unknown partitions (%s) in optimizer hint %s", - strings.Join(unknownPartitions, ","), hintName))) + + warning := fmt.Errorf("Unknown partitions (%s) in optimizer hint %s", strings.Join(unknownPartitions, ","), hintName) + ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } func (s *partitionProcessor) checkHintsApplicable(ds *DataSource, partitionSet set.StringSet) { diff --git a/planner/core/stringer.go b/planner/core/stringer.go index 4c63f6ff244b9..346b5b50e5742 100644 --- a/planner/core/stringer.go +++ b/planner/core/stringer.go @@ -270,31 +270,31 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { case *PhysicalShuffleReceiverStub: str = fmt.Sprintf("PartitionReceiverStub(%s)", x.ExplainInfo()) case *PointGetPlan: - str = fmt.Sprintf("PointGet(") + str = "PointGet(" if x.IndexInfo != nil { str += fmt.Sprintf("Index(%s.%s)%v)", x.TblInfo.Name.L, x.IndexInfo.Name.L, x.IndexValues) } else { str += fmt.Sprintf("Handle(%s.%s)%v)", x.TblInfo.Name.L, x.TblInfo.GetPkName().L, x.Handle) } case *BatchPointGetPlan: - str = fmt.Sprintf("BatchPointGet(") + str = "BatchPointGet(" if x.IndexInfo != nil { str += fmt.Sprintf("Index(%s.%s)%v)", x.TblInfo.Name.L, x.IndexInfo.Name.L, x.IndexValues) } else { str += fmt.Sprintf("Handle(%s.%s)%v)", x.TblInfo.Name.L, x.TblInfo.GetPkName().L, x.Handles) } case *PhysicalExchangeReceiver: - str = fmt.Sprintf("Recv(") + str = "Recv(" for _, task := range x.Tasks { str += fmt.Sprintf("%d, ", task.ID) } - str += fmt.Sprintf(")") + str += ")" case *PhysicalExchangeSender: - str = fmt.Sprintf("Send(") + str = "Send(" for _, task := range x.TargetTasks { str += fmt.Sprintf("%d, ", task.ID) } - str += fmt.Sprintf(")") + str += ")" default: str = fmt.Sprintf("%T", in) } diff --git a/planner/core/util.go b/planner/core/util.go index 753445f07ee42..19ce0a47673a2 100644 --- a/planner/core/util.go +++ b/planner/core/util.go @@ -291,7 +291,7 @@ func extractStringFromStringSet(set set.StringSet) string { l = append(l, fmt.Sprintf(`"%s"`, k)) } sort.Strings(l) - return fmt.Sprintf("%s", strings.Join(l, ",")) + return strings.Join(l, ",") } func tableHasDirtyContent(ctx sessionctx.Context, tableInfo *model.TableInfo) bool { diff --git a/plugin/conn_ip_example/conn_ip_example.go b/plugin/conn_ip_example/conn_ip_example.go index bae1b3ff37497..24d0bf04b0309 100644 --- a/plugin/conn_ip_example/conn_ip_example.go +++ b/plugin/conn_ip_example/conn_ip_example.go @@ -62,19 +62,14 @@ func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugi switch event { case plugin.Log: fmt.Println("---- event: Log") - break case plugin.Error: fmt.Println("---- event: Error") - break case plugin.Result: fmt.Println("---- event: Result") - break case plugin.Status: fmt.Println("---- event: Status") - break default: fmt.Println("---- event: unrecognized") - break } fmt.Printf("---- cmd: %s\n", cmd) } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index a7495b987f24a..7cdd7137138bd 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -578,7 +578,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { ts.prepareData(c) defer ts.stopServer(c) - resp, err := ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1")) + resp, err := ts.fetchStatus("/mvcc/key/tidb/test/1") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) var data mvccKV @@ -621,7 +621,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { c.Assert(err, IsNil) c.Assert(data2, DeepEquals, data) - resp, err = ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1?decode=true")) + resp, err = ts.fetchStatus("/mvcc/key/tidb/test/1?decode=true") c.Assert(err, IsNil) decoder = json.NewDecoder(resp.Body) var data3 map[string]interface{} @@ -667,7 +667,7 @@ func (ts *HTTPHandlerTestSuite) TestGetMVCCNotFound(c *C) { ts.startServer(c) ts.prepareData(c) defer ts.stopServer(c) - resp, err := ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1234")) + resp, err := ts.fetchStatus("/mvcc/key/tidb/test/1234") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) var data mvccKV diff --git a/server/server_test.go b/server/server_test.go index 20c0c2b508213..a342dab77d79a 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -499,14 +499,14 @@ func (cli *testServerClient) runTestLoadDataForSlowLog(c *C, server *Server) { } // Test for record slow log for load data statement. - rows := dbt.mustQuery(fmt.Sprintf("select plan from information_schema.slow_query where query like 'load data local infile %% into table t_slow;' order by time desc limit 1")) + rows := dbt.mustQuery("select plan from information_schema.slow_query where query like 'load data local infile % into table t_slow;' order by time desc limit 1") expectedPlan := ".*LoadData.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.* commit_txn.*" checkPlan(rows, expectedPlan) // Test for record statements_summary for load data statement. - rows = dbt.mustQuery(fmt.Sprintf("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %%' limit 1")) + rows = dbt.mustQuery("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %' limit 1") checkPlan(rows, expectedPlan) // Test log normal statement after executing load date. - rows = dbt.mustQuery(fmt.Sprintf("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1")) + rows = dbt.mustQuery("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1") expectedPlan = ".*Insert.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.*" checkPlan(rows, expectedPlan) }) diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 163c22e4a6fb7..5645b7496ebcc 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -154,17 +154,15 @@ func WaitBinlogRecover(timeout time.Duration) error { defer ticker.Stop() start := time.Now() for { - select { - case <-ticker.C: - if atomic.LoadInt32(&skippedCommitterCounter) == 0 { - logutil.BgLogger().Warn("[binloginfo] binlog recovered") - return nil - } - if time.Since(start) > timeout { - logutil.BgLogger().Warn("[binloginfo] waiting for binlog recovering timed out", - zap.Duration("duration", timeout)) - return errors.New("timeout") - } + <-ticker.C + if atomic.LoadInt32(&skippedCommitterCounter) == 0 { + logutil.BgLogger().Warn("[binloginfo] binlog recovered") + return nil + } + if time.Since(start) > timeout { + logutil.BgLogger().Warn("[binloginfo] waiting for binlog recovering timed out", + zap.Duration("duration", timeout)) + return errors.New("timeout") } } } diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index f682a1507a4bf..68791a7a0787f 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -818,14 +818,11 @@ func MergeTopN(topNs []*TopN, n uint32) (*TopN, []TopNMeta) { } func checkEmptyTopNs(topNs []*TopN) bool { - totCnt := uint64(0) + count := uint64(0) for _, topN := range topNs { - totCnt += topN.TotalCount() + count += topN.TotalCount() } - if totCnt == 0 { - return true - } - return false + return count == 0 } func getMergedTopNFromSortedSlice(sorted []TopNMeta, n uint32) (*TopN, []TopNMeta) { diff --git a/statistics/feedback.go b/statistics/feedback.go index 89aeab32152b3..5b69163e21b2b 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -107,7 +107,6 @@ func (m *QueryFeedbackMap) Append(q *QueryFeedback) { Tp: q.Tp, } m.append(k, []*QueryFeedback{q}) - return } // MaxQueryFeedbackCount is the max number of feedbacks that are cached in memory. @@ -136,7 +135,6 @@ func (m *QueryFeedbackMap) Merge(r *QueryFeedbackMap) { break } } - return } var ( diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index a8f51f8924e81..fe3644739c1b8 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -554,7 +554,6 @@ func (sc statsCache) initMemoryUsage() { sum += tb.MemoryUsage() } sc.memUsage = sum - return } // update updates the statistics table cache using copy on write. diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 2bf1d93ff4246..6d39e42b076e0 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2377,7 +2377,7 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { // test NDV checkNDV := func(rows, ndv int) { tk.MustExec("analyze table t") - rs := tk.MustQuery(fmt.Sprintf("select value from mysql.stats_fm_sketch")).Rows() + rs := tk.MustQuery("select value from mysql.stats_fm_sketch").Rows() c.Assert(len(rs), Equals, rows) for i := range rs { fm, err := statistics.DecodeFMSketch([]byte(rs[i][0].(string))) diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index b105738098f4b..a1de28e78eeef 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -577,8 +577,8 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeOnEmptyTable(c *C) { // test if it will be limited by the time range c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsFalse) - tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='00:00 +0000'")) - tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='23:59 +0000'")) + tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'") + tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'") c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsTrue) } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 9869fa501d430..2aaf4223ed8e5 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -144,14 +144,10 @@ func (m *mppIterator) run(ctx context.Context) { break } m.mu.Lock() - switch task.State { - case kv.MppTaskReady: + if task.State == kv.MppTaskReady { task.State = kv.MppTaskRunning - m.mu.Unlock() - default: - m.mu.Unlock() - break } + m.mu.Unlock() m.wg.Add(1) bo := backoff.NewBackoffer(ctx, copNextMaxBackoff) go m.handleDispatchReq(ctx, bo, task) diff --git a/store/mockstore/unistore/cophandler/closure_exec.go b/store/mockstore/unistore/cophandler/closure_exec.go index 152362c13d5e3..696a1497b068e 100644 --- a/store/mockstore/unistore/cophandler/closure_exec.go +++ b/store/mockstore/unistore/cophandler/closure_exec.go @@ -124,9 +124,7 @@ func buildClosureExecutorFromExecutorList(dagCtx *dagContext, executors []*tipb. outputFieldTypes = append(outputFieldTypes, originalOutputFieldTypes[idx]) } } else { - for _, tp := range originalOutputFieldTypes { - outputFieldTypes = append(outputFieldTypes, tp) - } + outputFieldTypes = append(outputFieldTypes, originalOutputFieldTypes...) } if len(executors) == 1 { ce.resultFieldType = outputFieldTypes diff --git a/store/mockstore/unistore/tikv/dbreader/db_reader.go b/store/mockstore/unistore/tikv/dbreader/db_reader.go index 158f295cc1bdf..6e3909ada740f 100644 --- a/store/mockstore/unistore/tikv/dbreader/db_reader.go +++ b/store/mockstore/unistore/tikv/dbreader/db_reader.go @@ -165,7 +165,6 @@ func (r *DBReader) BatchGet(keys [][]byte, startTS uint64, f BatchGetFunc) { } f(key, val, err) } - return } // ErrScanBreak is returned by ScanFunc to break the scan loop. diff --git a/store/mockstore/unistore/tikv/detector.go b/store/mockstore/unistore/tikv/detector.go index a27adb3f35b6c..6a56a0d9fd73c 100644 --- a/store/mockstore/unistore/tikv/detector.go +++ b/store/mockstore/unistore/tikv/detector.go @@ -64,10 +64,7 @@ type diagnosticContext struct { } func (p *txnKeyHashPair) isExpired(ttl time.Duration, nowTime time.Time) bool { - if p.registerTime.Add(ttl).Before(nowTime) { - return true - } - return false + return p.registerTime.Add(ttl).Before(nowTime) } // NewDetector creates a new Detector. diff --git a/store/mockstore/unistore/tikv/detector_test.go b/store/mockstore/unistore/tikv/detector_test.go index b0d3a074ff840..189a8b00b8217 100644 --- a/store/mockstore/unistore/tikv/detector_test.go +++ b/store/mockstore/unistore/tikv/detector_test.go @@ -26,7 +26,6 @@ package tikv import ( - "fmt" "testing" "time" @@ -68,7 +67,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { c.Assert(detector.totalSize, Equals, uint64(2)) err = detector.Detect(3, 1, 300, makeDiagCtx("k3", "tag3")) c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, fmt.Sprintf("deadlock")) + c.Assert(err.Error(), Equals, "deadlock") c.Assert(len(err.WaitChain), Equals, 3) // The order of entries in the wait chain is specific: each item is waiting for the next one. checkWaitChainEntry(err.WaitChain[0], 1, 2, "k1", "tag1") diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index ec5b92f81c5ad..b8b61aac05fc8 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -477,10 +477,7 @@ func (s *RegionRequestSender) sendReqToRegion(bo *Backoffer, rpcCtx *RPCContext, if val.(bool) { ctx1, cancel := context.WithCancel(context.Background()) cancel() - select { - case <-ctx1.Done(): - } - + <-ctx1.Done() ctx = ctx1 err = ctx.Err() resp = nil diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 20bf0491ed294..e444f5adda7f6 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -642,7 +642,7 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput // PointGet and BatchPointGet will return value in pessimistic lock response, the value may not exist. // For other lock modes, the locked key values always exist. if lockCtx.ReturnValues { - val, _ := lockCtx.Values[string(key)] + val := lockCtx.Values[string(key)] if len(val.Value) == 0 { valExists = tikv.SetKeyLockedValueNotExists } diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index f67cbb36b76da..0ef0b573e1bb2 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -90,7 +90,6 @@ func (c *RowContainer) SpillToDisk() { } } c.m.records.Clear() - return } // Reset resets RowContainer. @@ -447,7 +446,6 @@ func (c *SortedRowContainer) Sort() { func (c *SortedRowContainer) sortAndSpillToDisk() { c.Sort() c.RowContainer.SpillToDisk() - return } // Add appends a chunk into the SortedRowContainer. diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index ca045352dbd33..676f0a241d489 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -703,9 +703,7 @@ func (e *RuntimeStatsWithConcurrencyInfo) SetConcurrencyInfo(infos ...*Concurren e.Lock() defer e.Unlock() e.concurrency = e.concurrency[:0] - for _, info := range infos { - e.concurrency = append(e.concurrency, info) - } + e.concurrency = append(e.concurrency, infos...) } // Clone implements the RuntimeStats interface. diff --git a/util/expensivequery/memory_usage_alarm.go b/util/expensivequery/memory_usage_alarm.go index fbe9b6c5ff438..03d53c12342c9 100644 --- a/util/expensivequery/memory_usage_alarm.go +++ b/util/expensivequery/memory_usage_alarm.go @@ -83,7 +83,6 @@ func (record *memoryUsageAlarm) initMemoryUsageAlarmRecord() { } } record.initialized = true - return } // If Performance.ServerMemoryQuota is set, use `ServerMemoryQuota * MemoryUsageAlarmRatio` to check oom risk. diff --git a/util/profile/trackerRecorder.go b/util/profile/trackerRecorder.go index c5e5390a51bff..7c449ffa9113c 100644 --- a/util/profile/trackerRecorder.go +++ b/util/profile/trackerRecorder.go @@ -29,12 +29,10 @@ func HeapProfileForGlobalMemTracker(d time.Duration) { t := time.NewTicker(d) defer t.Stop() for { - select { - case <-t.C: - err := heapProfileForGlobalMemTracker() - if err != nil { - log.Warn("profile memory into tracker failed", zap.Error(err)) - } + <-t.C + err := heapProfileForGlobalMemTracker() + if err != nil { + log.Warn("profile memory into tracker failed", zap.Error(err)) } } } diff --git a/util/rowcodec/decoder.go b/util/rowcodec/decoder.go index 69a78d1de7d43..0efd50ecaf27c 100644 --- a/util/rowcodec/decoder.go +++ b/util/rowcodec/decoder.go @@ -260,10 +260,7 @@ func (decoder *ChunkDecoder) tryAppendHandleColumn(colIdx int, col *ColInfo, han } coder := codec.NewDecoder(chk, decoder.loc) _, err := coder.DecodeOne(handle.EncodedCol(i), colIdx, col.Ft) - if err != nil { - return false - } - return true + return err == nil } } return false From e628bad698b58f8b1268c7fdf5305f25d16b1c84 Mon Sep 17 00:00:00 2001 From: cfzjywxk Date: Fri, 14 May 2021 10:31:38 +0800 Subject: [PATCH 07/16] txn: avoid the gc resolving pessimistic locks of ongoing transactions (#24601) --- store/gcworker/gc_worker.go | 13 ++++++-- store/gcworker/gc_worker_test.go | 51 +++++++++++++++++++++++++++++++- 2 files changed, 61 insertions(+), 3 deletions(-) diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index e0aa993558b6c..b408f279be98a 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -409,12 +409,21 @@ func (w *GCWorker) calcSafePointByMinStartTS(ctx context.Context, safePoint uint return safePoint } - if globalMinStartTS < safePoint { + // If the lock.ts <= max_ts(safePoint), it will be collected and resolved by the gc worker, + // the locks of ongoing pessimistic transactions could be resolved by the gc worker and then + // the transaction is aborted, decrement the value by 1 to avoid this. + globalMinStartAllowedTS := globalMinStartTS + if globalMinStartTS > 0 { + globalMinStartAllowedTS = globalMinStartTS - 1 + } + + if globalMinStartAllowedTS < safePoint { logutil.Logger(ctx).Info("[gc worker] gc safepoint blocked by a running session", zap.String("uuid", w.uuid), zap.Uint64("globalMinStartTS", globalMinStartTS), + zap.Uint64("globalMinStartAllowedTS", globalMinStartAllowedTS), zap.Uint64("safePoint", safePoint)) - safePoint = globalMinStartTS + safePoint = globalMinStartAllowedTS } return safePoint } diff --git a/store/gcworker/gc_worker_test.go b/store/gcworker/gc_worker_test.go index bc09651e0d379..39abe369f82fb 100644 --- a/store/gcworker/gc_worker_test.go +++ b/store/gcworker/gc_worker_test.go @@ -262,7 +262,7 @@ func (s *testGCWorkerSuite) TestMinStartTS(c *C) { strconv.FormatUint(now-oracle.EncodeTSO(20000), 10)) c.Assert(err, IsNil) sp = s.gcWorker.calcSafePointByMinStartTS(ctx, now-oracle.EncodeTSO(10000)) - c.Assert(sp, Equals, now-oracle.EncodeTSO(20000)) + c.Assert(sp, Equals, now-oracle.EncodeTSO(20000)-1) } func (s *testGCWorkerSuite) TestPrepareGC(c *C) { @@ -1589,3 +1589,52 @@ func (s *testGCWorkerSuite) TestGCPlacementRules(c *C) { c.Assert(pid, Equals, int64(1)) c.Assert(err, IsNil) } + +func (s *testGCWorkerSuite) TestGCWithPendingTxn(c *C) { + ctx := context.Background() + gcSafePointCacheInterval = 0 + err := s.gcWorker.saveValueToSysTable(gcEnableKey, booleanFalse) + c.Assert(err, IsNil) + + k1 := []byte("tk1") + v1 := []byte("v1") + txn, err := s.store.Begin() + c.Assert(err, IsNil) + txn.SetOption(kv.Pessimistic, true) + lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()} + + // Lock the key. + err = txn.Set(k1, v1) + c.Assert(err, IsNil) + err = txn.LockKeys(ctx, lockCtx, k1) + c.Assert(err, IsNil) + + // Prepare to run gc with txn's startTS as the safepoint ts. + spkv := s.tikvStore.GetSafePointKV() + err = spkv.Put(fmt.Sprintf("%s/%s", infosync.ServerMinStartTSPath, "a"), strconv.FormatUint(txn.StartTS(), 10)) + c.Assert(err, IsNil) + s.mustSetTiDBServiceSafePoint(c, txn.StartTS(), txn.StartTS()) + veryLong := gcDefaultLifeTime * 100 + err = s.gcWorker.saveTime(gcLastRunTimeKey, oracle.GetTimeFromTS(s.mustAllocTs(c)).Add(-veryLong)) + c.Assert(err, IsNil) + s.gcWorker.lastFinish = time.Now().Add(-veryLong) + s.oracle.AddOffset(time.Minute * 10) + err = s.gcWorker.saveValueToSysTable(gcEnableKey, booleanTrue) + c.Assert(err, IsNil) + + // Trigger the tick let the gc job start. + err = s.gcWorker.leaderTick(ctx) + c.Assert(err, IsNil) + // Wait for GC finish + select { + case err = <-s.gcWorker.done: + s.gcWorker.gcIsRunning = false + break + case <-time.After(time.Second * 10): + err = errors.New("receive from s.gcWorker.done timeout") + } + c.Assert(err, IsNil) + + err = txn.Commit(ctx) + c.Assert(err, IsNil) +} From d169a57ba5a4e677196d12b7fef797561315b29b Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Fri, 14 May 2021 11:55:38 +0800 Subject: [PATCH 08/16] util: fix wrong enum building for index range (#24632) --- expression/integration_test.go | 10 ++++++++++ util/ranger/points.go | 2 +- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 8e0f70823f327..095e21119ab30 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9344,4 +9344,14 @@ func (s *testIntegrationSuite) TestEnumIndex(c *C) { testkit.Rows( "OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A -30 ", "ZOHBSCRMZPOI`IVTSEZAIDAF7DS@1TT20AP9 -30 ")) + + // issue 24576 + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(col1 enum('a','b','c'), col2 enum('a','b','c'), col3 int, index idx(col1,col2));") + tk.MustExec("insert into t values(1,1,1),(2,2,2),(3,3,3);") + tk.MustQuery("select /*+ use_index(t,idx) */ col3 from t where col2 between 'b' and 'b' and col1 is not null;").Check( + testkit.Rows("2")) + tk.MustQuery("select /*+ use_index(t,idx) */ col3 from t where col2 = 'b' and col1 is not null;").Check( + testkit.Rows("2")) } diff --git a/util/ranger/points.go b/util/ranger/points.go index 9c33ccef7feb3..46a4283dd3222 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -459,7 +459,7 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val tmpEnum := types.Enum{} for i := range ft.Elems { tmpEnum.Name = ft.Elems[i] - tmpEnum.Value = uint64(i) + tmpEnum.Value = uint64(i) + 1 d := types.NewMysqlEnumDatum(tmpEnum) if v, err := d.CompareDatum(sc, &val); err == nil { switch op { From 80a557eb4956da36f21ba1c81ad12595dcf716b1 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Thu, 13 May 2021 22:43:38 -0600 Subject: [PATCH 09/16] sessionctx: change innodb large prefix default (#24555) --- expression/integration_test.go | 6 ++++++ sessionctx/variable/noop.go | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 095e21119ab30..b1bded2a17719 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9086,9 +9086,15 @@ func (s *testIntegrationSuite) TestEnumPushDown(c *C) { func (s *testIntegrationSuite) TestJiraSetInnoDBDefaultRowFormat(c *C) { // For issue #23541 // JIRA needs to be able to set this to be happy. + // See: https://nova.moe/run-jira-on-tidb/ tk := testkit.NewTestKit(c, s.store) tk.MustExec("set global innodb_default_row_format = dynamic") tk.MustExec("set global innodb_default_row_format = 'dynamic'") + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_default_row_format'").Check(testkit.Rows("innodb_default_row_format dynamic")) + tk.MustQuery("SHOW VARIABLES LIKE 'character_set_server'").Check(testkit.Rows("character_set_server utf8mb4")) + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_file_format'").Check(testkit.Rows("innodb_file_format Barracuda")) + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_large_prefix'").Check(testkit.Rows("innodb_large_prefix ON")) + } func (s *testIntegrationSerialSuite) TestCollationForBinaryLiteral(c *C) { diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index c510d9c73ce3a..1ad37b512d807 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -312,7 +312,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "datetime_format", Value: "%Y-%m-%d %H:%i:%s"}, {Scope: ScopeGlobal, Name: "log_syslog", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "transaction_alloc_block_size", Value: "8192"}, - {Scope: ScopeGlobal, Name: "innodb_large_prefix", Type: TypeBool, Value: Off}, + {Scope: ScopeGlobal, Name: "innodb_large_prefix", Type: TypeBool, Value: On}, {Scope: ScopeNone, Name: "performance_schema_max_cond_classes", Value: "80"}, {Scope: ScopeGlobal, Name: "innodb_io_capacity", Value: "200"}, {Scope: ScopeGlobal, Name: "max_binlog_cache_size", Value: "18446744073709547520"}, From ea7f0ca1ba80674a1bec449ca2c3d0f79d48b735 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Fri, 14 May 2021 15:49:38 +0800 Subject: [PATCH 10/16] store: fix data race about KVStore.tikvClient (#24655) --- store/tikv/commit.go | 2 +- store/tikv/kv.go | 25 ++++++++++++++++--------- store/tikv/prewrite.go | 2 +- store/tikv/scan.go | 2 +- store/tikv/split_region.go | 2 +- 5 files changed, 20 insertions(+), 13 deletions(-) diff --git a/store/tikv/commit.go b/store/tikv/commit.go index 449081860c029..10c60d9f6d4bd 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -48,7 +48,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch CommitVersion: c.commitTS, }, pb.Context{Priority: c.priority, SyncLog: c.syncLog}) - sender := NewRegionRequestSender(c.store.regionCache, c.store.client) + sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) // If we fail to receive response for the request that commits primary key, it will be undetermined whether this diff --git a/store/tikv/kv.go b/store/tikv/kv.go index f61db4168ef7d..bbf8517a42a8c 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -66,10 +66,13 @@ var oracleUpdateInterval = 2000 // KVStore contains methods to interact with a TiKV cluster. type KVStore struct { - clusterID uint64 - uuid string - oracle oracle.Oracle - client Client + clusterID uint64 + uuid string + oracle oracle.Oracle + clientMu struct { + sync.RWMutex + client Client + } pdClient pd.Client regionCache *RegionCache lockResolver *LockResolver @@ -133,7 +136,6 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client clusterID: pdClient.GetClusterID(context.TODO()), uuid: uuid, oracle: o, - client: reqCollapse{client}, pdClient: pdClient, regionCache: NewRegionCache(pdClient), kv: spkv, @@ -142,6 +144,7 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client closed: make(chan struct{}), replicaReadSeed: rand.Uint32(), } + store.clientMu.client = reqCollapse{client} store.lockResolver = newLockResolver(store) go store.runSafePointChecker() @@ -205,7 +208,7 @@ func (s *KVStore) Close() error { s.pdClient.Close() close(s.closed) - if err := s.client.Close(); err != nil { + if err := s.GetTiKVClient().Close(); err != nil { return errors.Trace(err) } @@ -312,7 +315,7 @@ func (s *KVStore) SupportDeleteRange() (supported bool) { // SendReq sends a request to region. func (s *KVStore) SendReq(bo *Backoffer, req *tikvrpc.Request, regionID RegionVerID, timeout time.Duration) (*tikvrpc.Response, error) { - sender := NewRegionRequestSender(s.regionCache, s.client) + sender := NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) return sender.SendReq(bo, req, regionID, timeout) } @@ -343,12 +346,16 @@ func (s *KVStore) SetOracle(oracle oracle.Oracle) { // SetTiKVClient resets the client instance. func (s *KVStore) SetTiKVClient(client Client) { - s.client = client + s.clientMu.Lock() + defer s.clientMu.Unlock() + s.clientMu.client = client } // GetTiKVClient gets the client instance. func (s *KVStore) GetTiKVClient() (client Client) { - return s.client + s.clientMu.RLock() + defer s.clientMu.RUnlock() + return s.clientMu.client } func (s *KVStore) getSafeTS(storeID uint64) uint64 { diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index 7097ba5dbcd3e..ffb47e1fb46fa 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -157,7 +157,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff req := c.buildPrewriteRequest(batch, txnSize) for { - sender := NewRegionRequestSender(c.store.regionCache, c.store.client) + sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) // If we fail to receive response for async commit prewrite, it will be undetermined whether this diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 6c43b7bdee7cd..035291a783aec 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -164,7 +164,7 @@ func (s *Scanner) getData(bo *Backoffer) error { zap.String("nextEndKey", kv.StrKey(s.nextEndKey)), zap.Bool("reverse", s.reverse), zap.Uint64("txnStartTS", s.startTS())) - sender := NewRegionRequestSender(s.snapshot.store.regionCache, s.snapshot.store.client) + sender := NewRegionRequestSender(s.snapshot.store.regionCache, s.snapshot.store.GetTiKVClient()) var reqEndKey, reqStartKey []byte var loc *KeyLocation var err error diff --git a/store/tikv/split_region.go b/store/tikv/split_region.go index 38ce24917d1cf..c33a89efc19be 100644 --- a/store/tikv/split_region.go +++ b/store/tikv/split_region.go @@ -123,7 +123,7 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool Priority: kvrpcpb.CommandPri_Normal, }) - sender := NewRegionRequestSender(s.regionCache, s.client) + sender := NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.regionID, ReadTimeoutShort) batchResp := singleBatchResp{resp: resp} From d9f28c7f765662a575e8a2bf6f139229cce1d360 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Fri, 14 May 2021 02:07:39 -0600 Subject: [PATCH 11/16] executor, privileges: Add dynamic privileges to SHOW PRIVILEGES (#24646) --- executor/executor_test.go | 9 ++++++++- executor/show.go | 4 ++++ privilege/privileges/privileges.go | 11 +++++++++++ privilege/privileges/privileges_test.go | 12 ++++++++++++ 4 files changed, 35 insertions(+), 1 deletion(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 65af164174e6f..d67bb6b48b8f4 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -350,7 +350,14 @@ func (s *testSuiteP1) TestShow(c *C) { "Trigger Tables To use triggers", "Create tablespace Server Admin To create/alter/drop tablespaces", "Update Tables To update existing rows", - "Usage Server Admin No privileges - allow connect only")) + "Usage Server Admin No privileges - allow connect only", + "BACKUP_ADMIN Server Admin ", + "SYSTEM_VARIABLES_ADMIN Server Admin ", + "ROLE_ADMIN Server Admin ", + "CONNECTION_ADMIN Server Admin ", + "RESTRICTED_TABLES_ADMIN Server Admin ", + "RESTRICTED_STATUS_ADMIN Server Admin ", + )) c.Assert(len(tk.MustQuery("show table status").Rows()), Equals, 1) } diff --git a/executor/show.go b/executor/show.go index 2bd9b786fcffb..ab08e5ba4cbf5 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1412,6 +1412,10 @@ func (e *ShowExec) fetchShowPrivileges() error { e.appendRow([]interface{}{"Create tablespace", "Server Admin", "To create/alter/drop tablespaces"}) e.appendRow([]interface{}{"Update", "Tables", "To update existing rows"}) e.appendRow([]interface{}{"Usage", "Server Admin", "No privileges - allow connect only"}) + + for _, priv := range privileges.GetDynamicPrivileges() { + e.appendRow([]interface{}{priv, "Server Admin", ""}) + } return nil } diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index c5ec2f8394385..6ac58e04e44e4 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -535,3 +535,14 @@ func RegisterDynamicPrivilege(privNameInUpper string) error { dynamicPrivs = append(dynamicPrivs, privNameInUpper) return nil } + +// GetDynamicPrivileges returns the list of registered DYNAMIC privileges +// for use in meta data commands (i.e. SHOW PRIVILEGES) +func GetDynamicPrivileges() []string { + dynamicPrivLock.Lock() + defer dynamicPrivLock.Unlock() + + privCopy := make([]string, len(dynamicPrivs)) + copy(privCopy, dynamicPrivs) + return privCopy +} diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 3038aad397076..c7a825a4d894d 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1427,3 +1427,15 @@ func (s *testPrivilegeSuite) TestViewDefiner(c *C) { tk.MustExec("select * from test_view") tk.MustExec("select * from test_view2") } + +func (s *testPrivilegeSuite) TestDynamicPrivsRegistration(c *C) { + se := newSession(c, s.store, s.dbName) + pm := privilege.GetPrivilegeManager(se) + + count := len(privileges.GetDynamicPrivileges()) + + c.Assert(pm.IsDynamicPrivilege("ACDC_ADMIN"), IsFalse) + privileges.RegisterDynamicPrivilege("ACDC_ADMIN") + c.Assert(pm.IsDynamicPrivilege("ACDC_ADMIN"), IsTrue) + c.Assert(len(privileges.GetDynamicPrivileges()), Equals, count+1) +} From 031a9fa18b909c1501c99d6a8a98448beb94a962 Mon Sep 17 00:00:00 2001 From: xhe Date: Fri, 14 May 2021 17:05:38 +0800 Subject: [PATCH 12/16] ddl: refactor rule [4/6] (#24007) --- ddl/ddl_api.go | 14 +-- ddl/placement/errors.go | 6 + ddl/placement/rule.go | 132 +++++++++++++++++++++ ddl/placement/rule_test.go | 206 +++++++++++++++++++++++++++++++++ ddl/placement/types.go | 36 ------ ddl/placement/types_test.go | 12 -- ddl/placement/utils.go | 4 +- ddl/placement/utils_test.go | 14 +-- ddl/placement_rule_test.go | 36 +++--- ddl/placement_sql_test.go | 6 +- executor/infoschema_reader.go | 2 +- expression/integration_test.go | 2 +- infoschema/tables_test.go | 2 +- session/session_test.go | 2 +- 14 files changed, 385 insertions(+), 89 deletions(-) create mode 100644 ddl/placement/rule.go create mode 100644 ddl/placement/rule_test.go diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 528a6087638fc..a3f8bb7f9c622 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5895,8 +5895,8 @@ func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]* } rules = append(rules, &placement.Rule{ - Count: int(replicas), - LabelConstraints: labelConstraints, + Count: int(replicas), + Constraints: labelConstraints, }) return rules, nil @@ -5925,8 +5925,8 @@ func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]* } rules = append(rules, &placement.Rule{ - Count: cnt, - LabelConstraints: labelConstraints, + Count: cnt, + Constraints: labelConstraints, }) } @@ -6051,14 +6051,14 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, newRules := bundle.Rules[:0] for i, rule := range bundle.Rules { // merge all empty constraints - if len(rule.LabelConstraints) == 0 { + if len(rule.Constraints) == 0 { extraCnt[rule.Role] += rule.Count continue } // refer to tidb#22065. // add -engine=tiflash to every rule to avoid schedules to tiflash instances. // placement rules in SQL is not compatible with `set tiflash replica` yet - if err := rule.LabelConstraints.Add(placement.Constraint{ + if err := rule.Constraints.Add(placement.Constraint{ Op: placement.NotIn, Key: placement.EngineLabelKey, Values: []string{placement.EngineLabelTiFlash}, @@ -6083,7 +6083,7 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, Count: cnt, StartKeyHex: startKey, EndKeyHex: endKey, - LabelConstraints: []placement.Constraint{{ + Constraints: []placement.Constraint{{ Op: placement.NotIn, Key: placement.EngineLabelKey, Values: []string{placement.EngineLabelTiFlash}, diff --git a/ddl/placement/errors.go b/ddl/placement/errors.go index 19797022a609c..95fce4591c961 100644 --- a/ddl/placement/errors.go +++ b/ddl/placement/errors.go @@ -24,4 +24,10 @@ var ( ErrUnsupportedConstraint = errors.New("unsupported label constraint") // ErrConflictingConstraints is from constraints.go. ErrConflictingConstraints = errors.New("conflicting label constraints") + // ErrInvalidConstraintsMapcnt is from rule.go. + ErrInvalidConstraintsMapcnt = errors.New("label constraints in map syntax have invalid replicas") + // ErrInvalidConstraintsFormat is from rule.go. + ErrInvalidConstraintsFormat = errors.New("invalid label constraints format") + // ErrInvalidConstraintsRelicas is from rule.go. + ErrInvalidConstraintsRelicas = errors.New("label constraints with invalid REPLICAS") ) diff --git a/ddl/placement/rule.go b/ddl/placement/rule.go new file mode 100644 index 0000000000000..134bdd5a610f9 --- /dev/null +++ b/ddl/placement/rule.go @@ -0,0 +1,132 @@ +// Copyright 2021 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 placement + +import ( + "fmt" + "strings" + + "github.com/go-yaml/yaml" +) + +// PeerRoleType is the expected peer type of the placement rule. +type PeerRoleType string + +const ( + // Voter can either match a leader peer or follower peer. + Voter PeerRoleType = "voter" + // Leader matches a leader. + Leader PeerRoleType = "leader" + // Follower matches a follower. + Follower PeerRoleType = "follower" + // Learner matches a learner. + Learner PeerRoleType = "learner" +) + +// Rule is the core placement rule struct. Check https://github.com/tikv/pd/blob/master/server/schedule/placement/rule.go. +type Rule struct { + GroupID string `json:"group_id"` + ID string `json:"id"` + Index int `json:"index,omitempty"` + Override bool `json:"override,omitempty"` + StartKeyHex string `json:"start_key"` + EndKeyHex string `json:"end_key"` + Role PeerRoleType `json:"role"` + Count int `json:"count"` + Constraints Constraints `json:"label_constraints,omitempty"` + LocationLabels []string `json:"location_labels,omitempty"` + IsolationLevel string `json:"isolation_level,omitempty"` +} + +// NewRules constructs []*Rule from a yaml-compatible representation of +// array or map of constraints. It converts 'CONSTRAINTS' field in RFC +// https://github.com/pingcap/tidb/blob/master/docs/design/2020-06-24-placement-rules-in-sql.md to structs. +func NewRules(replicas uint64, cnstr string) ([]*Rule, error) { + rules := []*Rule{} + + cnstbytes := []byte(cnstr) + + constraints1 := []string{} + err1 := yaml.UnmarshalStrict(cnstbytes, &constraints1) + if err1 == nil { + // can not emit REPLICAS with an array or empty label + if replicas == 0 { + return rules, fmt.Errorf("%w: should be positive", ErrInvalidConstraintsRelicas) + } + + labelConstraints, err := NewConstraints(constraints1) + if err != nil { + return rules, err + } + + rules = append(rules, &Rule{ + Count: int(replicas), + Constraints: labelConstraints, + }) + + return rules, nil + } + + constraints2 := map[string]int{} + err2 := yaml.UnmarshalStrict(cnstbytes, &constraints2) + if err2 == nil { + ruleCnt := 0 + for labels, cnt := range constraints2 { + if cnt <= 0 { + return rules, fmt.Errorf("%w: count of labels '%s' should be positive, but got %d", ErrInvalidConstraintsMapcnt, labels, cnt) + } + ruleCnt += cnt + } + + if replicas == 0 { + replicas = uint64(ruleCnt) + } + + if int(replicas) < ruleCnt { + return rules, fmt.Errorf("%w: should be larger or equal to the number of total replicas, but REPLICAS=%d < total=%d", ErrInvalidConstraintsRelicas, replicas, ruleCnt) + } + + for labels, cnt := range constraints2 { + labelConstraints, err := NewConstraints(strings.Split(labels, ",")) + if err != nil { + return rules, err + } + + rules = append(rules, &Rule{ + Count: cnt, + Constraints: labelConstraints, + }) + } + + remain := int(replicas) - ruleCnt + if remain > 0 { + rules = append(rules, &Rule{ + Count: remain, + }) + } + + return rules, nil + } + + return nil, fmt.Errorf("%w: should be [constraint1, ...] (error %s), {constraint1: cnt1, ...} (error %s), or any yaml compatible representation", ErrInvalidConstraintsFormat, err1, err2) +} + +// Clone is used to duplicate a RuleOp for safe modification. +// Note that it is a shallow copy: LocationLabels and Constraints +// is not cloned. +func (r *Rule) Clone() *Rule { + n := &Rule{} + *n = *r + return n +} diff --git a/ddl/placement/rule_test.go b/ddl/placement/rule_test.go new file mode 100644 index 0000000000000..85dd492f348e7 --- /dev/null +++ b/ddl/placement/rule_test.go @@ -0,0 +1,206 @@ +// Copyright 2021 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 placement + +import ( + "encoding/json" + "errors" + + . "github.com/pingcap/check" +) + +var _ = Suite(&testRuleSuite{}) + +type testRuleSuite struct{} + +func (t *testRuleSuite) TestClone(c *C) { + rule := &Rule{ID: "434"} + newRule := rule.Clone() + newRule.ID = "121" + + c.Assert(rule, DeepEquals, &Rule{ID: "434"}) + c.Assert(newRule, DeepEquals, &Rule{ID: "121"}) +} + +func matchRule(r1 *Rule, t2 []*Rule) bool { + for _, r2 := range t2 { + if ok, _ := DeepEquals.Check([]interface{}{r1, r2}, nil); ok { + return true + } + } + return false +} + +func matchRules(t1, t2 []*Rule, prefix string, c *C) { + expected, err := json.Marshal(t1) + c.Assert(err, IsNil) + got, err := json.Marshal(t2) + c.Assert(err, IsNil) + comment := Commentf("%s, expected %s\nbut got %s", prefix, expected, got) + c.Assert(len(t1), Equals, len(t2), comment) + for _, r1 := range t1 { + c.Assert(matchRule(r1, t2), IsTrue, comment) + } +} + +func (t *testRuleSuite) TestNewRules(c *C) { + type TestCase struct { + name string + input string + replicas uint64 + output []*Rule + err error + } + tests := []TestCase{} + + tests = append(tests, TestCase{ + name: "empty constraints", + input: "", + replicas: 3, + output: []*Rule{ + { + Count: 3, + Constraints: Constraints{}, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "zero replicas", + input: "", + replicas: 0, + err: ErrInvalidConstraintsRelicas, + }) + + labels, err := NewConstraints([]string{"+zone=sh", "+zone=sh"}) + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal array constraints", + input: `["+zone=sh", "+zone=sh"]`, + replicas: 3, + output: []*Rule{ + { + Count: 3, + Constraints: labels, + }, + }, + }) + + labels1, err := NewConstraints([]string{"+zone=sh", "-zone=bj"}) + c.Assert(err, IsNil) + labels2, err := NewConstraints([]string{"+zone=sh"}) + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal object constraints", + input: `{"+zone=sh,-zone=bj":2, "+zone=sh": 1}`, + replicas: 3, + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "normal object constraints, with extra count", + input: "{'+zone=sh,-zone=bj':2, '+zone=sh': 1}", + replicas: 4, + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + { + Count: 1, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "normal object constraints, without count", + input: "{'+zone=sh,-zone=bj':2, '+zone=sh': 1}", + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "zero count in object constraints", + input: `{"+zone=sh,-zone=bj":0, "+zone=sh": 1}`, + replicas: 3, + err: ErrInvalidConstraintsMapcnt, + }) + + tests = append(tests, TestCase{ + name: "overlarge total count in object constraints", + input: `{"+ne=sh,-zone=bj":1, "+zone=sh": 4}`, + replicas: 3, + err: ErrInvalidConstraintsRelicas, + }) + + tests = append(tests, TestCase{ + name: "invalid array", + input: `["+ne=sh", "+zone=sh"`, + replicas: 3, + err: ErrInvalidConstraintsFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid array constraints", + input: `["ne=sh", "+zone=sh"]`, + replicas: 3, + err: ErrInvalidConstraintFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid map", + input: `{+ne=sh,-zone=bj:1, "+zone=sh": 4`, + replicas: 5, + err: ErrInvalidConstraintsFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid map constraints", + input: `{"nesh,-zone=bj":1, "+zone=sh": 4}`, + replicas: 6, + err: ErrInvalidConstraintFormat, + }) + + for _, t := range tests { + comment := Commentf("%s", t.name) + output, err := NewRules(t.replicas, t.input) + if t.err == nil { + c.Assert(err, IsNil, comment) + matchRules(t.output, output, comment.CheckCommentString(), c) + } else { + c.Assert(errors.Is(err, t.err), IsTrue, comment) + } + } +} diff --git a/ddl/placement/types.go b/ddl/placement/types.go index 3bb9da96e3890..72093a2c19c78 100644 --- a/ddl/placement/types.go +++ b/ddl/placement/types.go @@ -22,42 +22,6 @@ import ( // After all, placement rules are communicated using an HTTP API. Loose // coupling is a good feature. -// PeerRoleType is the expected peer type of the placement rule. -type PeerRoleType string - -const ( - // Voter can either match a leader peer or follower peer. - Voter PeerRoleType = "voter" - // Leader matches a leader. - Leader PeerRoleType = "leader" - // Follower matches a follower. - Follower PeerRoleType = "follower" - // Learner matches a learner. - Learner PeerRoleType = "learner" -) - -// Rule is the placement rule. Check https://github.com/tikv/pd/blob/master/server/schedule/placement/rule.go. -type Rule struct { - GroupID string `json:"group_id"` - ID string `json:"id"` - Index int `json:"index,omitempty"` - Override bool `json:"override,omitempty"` - StartKeyHex string `json:"start_key"` - EndKeyHex string `json:"end_key"` - Role PeerRoleType `json:"role"` - Count int `json:"count"` - LabelConstraints Constraints `json:"label_constraints,omitempty"` - LocationLabels []string `json:"location_labels,omitempty"` - IsolationLevel string `json:"isolation_level,omitempty"` -} - -// Clone is used to duplicate a RuleOp for safe modification. -func (r *Rule) Clone() *Rule { - n := &Rule{} - *n = *r - return n -} - // Bundle is a group of all rules and configurations. It is used to support rule cache. type Bundle struct { ID string `json:"group_id"` diff --git a/ddl/placement/types_test.go b/ddl/placement/types_test.go index 77153cb29b692..93ed1a5a80f43 100644 --- a/ddl/placement/types_test.go +++ b/ddl/placement/types_test.go @@ -18,7 +18,6 @@ import ( ) var _ = Suite(&testBundleSuite{}) -var _ = Suite(&testRuleSuite{}) type testBundleSuite struct{} @@ -49,14 +48,3 @@ func (t *testBundleSuite) TestClone(c *C) { c.Assert(bundle, DeepEquals, &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}}) c.Assert(newBundle, DeepEquals, &Bundle{ID: GroupID(2), Rules: []*Rule{{ID: "121"}}}) } - -type testRuleSuite struct{} - -func (t *testRuleSuite) TestClone(c *C) { - rule := &Rule{ID: "434"} - newRule := rule.Clone() - newRule.ID = "121" - - c.Assert(rule, DeepEquals, &Rule{ID: "434"}) - c.Assert(newRule, DeepEquals, &Rule{ID: "121"}) -} diff --git a/ddl/placement/utils.go b/ddl/placement/utils.go index 16c0a424dde53..5b12f10e2d243 100644 --- a/ddl/placement/utils.go +++ b/ddl/placement/utils.go @@ -61,7 +61,7 @@ func BuildPlacementCopyBundle(oldBundle *Bundle, newID int64) *Bundle { func GetLeaderDCByBundle(bundle *Bundle, dcLabelKey string) (string, bool) { for _, rule := range bundle.Rules { if isValidLeaderRule(rule, dcLabelKey) { - return rule.LabelConstraints[0].Values[0], true + return rule.Constraints[0].Values[0], true } } return "", false @@ -69,7 +69,7 @@ func GetLeaderDCByBundle(bundle *Bundle, dcLabelKey string) (string, bool) { func isValidLeaderRule(rule *Rule, dcLabelKey string) bool { if rule.Role == Leader && rule.Count == 1 { - for _, con := range rule.LabelConstraints { + for _, con := range rule.Constraints { if con.Op == In && con.Key == dcLabelKey && len(con.Values) == 1 { return true } diff --git a/ddl/placement/utils_test.go b/ddl/placement/utils_test.go index 964382846485e..10941e0663455 100644 --- a/ddl/placement/utils_test.go +++ b/ddl/placement/utils_test.go @@ -58,7 +58,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -84,7 +84,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Voter, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -110,7 +110,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -127,7 +127,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Voter, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -153,7 +153,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "fake", Op: In, @@ -179,7 +179,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: NotIn, @@ -205,7 +205,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, diff --git a/ddl/placement_rule_test.go b/ddl/placement_rule_test.go index b051092a776e9..a9a916cb5a199 100644 --- a/ddl/placement_rule_test.go +++ b/ddl/placement_rule_test.go @@ -52,7 +52,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, @@ -67,9 +67,9 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { Constraints: "", }}, output: []*placement.Rule{{ - Role: placement.Voter, - Count: 3, - LabelConstraints: []placement.Constraint{}, + Role: placement.Voter, + Count: 3, + Constraints: []placement.Constraint{}, }}, }, @@ -83,14 +83,14 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, { Role: placement.Voter, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, @@ -108,7 +108,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -127,7 +127,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -154,7 +154,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -162,7 +162,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Follower, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "in", Values: []string{"bj"}}, }, @@ -189,7 +189,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "in", Values: []string{"bj"}}, }, @@ -214,14 +214,14 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { }, output: []*placement.Rule{ { - Role: placement.Voter, - Count: 1, - LabelConstraints: []placement.Constraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, + Role: placement.Voter, + Count: 1, + Constraints: []placement.Constraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, }, { - Role: placement.Voter, - Count: 1, - LabelConstraints: []placement.Constraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, + Role: placement.Voter, + Count: 1, + Constraints: []placement.Constraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, }, { Role: placement.Voter, @@ -306,7 +306,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index e77b0ba99d5cf..fb7158681714f 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -404,7 +404,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -423,7 +423,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Follower, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -619,7 +619,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 4d56cc55accac..4f788a3d7bd1d 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1915,7 +1915,7 @@ func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) er continue } for _, rule := range bundle.Rules { - constraint, err := rule.LabelConstraints.Restore() + constraint, err := rule.Constraints.Restore() if err != nil { return errors.Wrapf(err, "Restore rule %s in bundle %s failed", rule.ID, bundle.ID) } diff --git a/expression/integration_test.go b/expression/integration_test.go index b1bded2a17719..80e39b76ce746 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -8755,7 +8755,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 6cc24300c1be4..ebe4a0620256f 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1449,7 +1449,7 @@ func (s *testTableSuite) TestPlacementPolicy(c *C) { ID: "0", Role: "voter", Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: "zone", Op: "in", diff --git a/session/session_test.go b/session/session_test.go index 3baee4f0ef6f1..a8861dfddd79c 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -3319,7 +3319,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, From 392df99f04d5bc51560993408f39de81ffd3c672 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Fri, 14 May 2021 17:21:38 +0800 Subject: [PATCH 13/16] cmd: ddl_test modify retryCnt from 5 to 20 (#24662) --- cmd/ddltest/ddl_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/ddltest/ddl_test.go b/cmd/ddltest/ddl_test.go index 52009b10de142..36922638950a8 100644 --- a/cmd/ddltest/ddl_test.go +++ b/cmd/ddltest/ddl_test.go @@ -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() From 4d7fcbb3dffb7372150e47257842eb30efd33bf7 Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Fri, 14 May 2021 17:33:38 +0800 Subject: [PATCH 14/16] executor: add correctness tests about direct reading with ORDER BY and LIMIT (#24455) --- executor/partition_table_test.go | 79 ++++++++++++++++++++++++++++++++ 1 file changed, 79 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index ab5a19ca823e9..df0c52cfb55cc 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -227,6 +227,85 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows()) } +func (s *partitionTableSuite) TestOrderByandLimit(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_orderby_limit") + tk.MustExec("use test_orderby_limit") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // range partition table + tk.MustExec(`create table trange(a int, b int, index idx_a(a)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table + tk.MustExec("create table thash(a int, b int, index idx_a(a), index idx_b(b)) partition by hash(a) partitions 4;") + + // regular table + tk.MustExec("create table tregular(a int, b int, index idx_a(a))") + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000))) + } + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular values " + strings.Join(vals, ",")) + + // test indexLookUp + for i := 0; i < 100; i++ { + // explain select * from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select * from t where a > {y} use index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select * from trange use index(idx_a) where a > %v order by a, b limit %v;", x, y) + queryRegular := fmt.Sprintf("select * from tregular use index(idx_a) where a > %v order by a, b limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "IndexLookUp"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test tableReader + for i := 0; i < 100; i++ { + // explain select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select * from trange ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + queryRegular := fmt.Sprintf("select * from tregular ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "TableReader"), IsTrue) // check if tableReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test indexReader + for i := 0; i < 100; i++ { + // explain select a from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select a from t where a > {y} use index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select a from trange use index(idx_a) where a > %v order by a limit %v;", x, y) + queryRegular := fmt.Sprintf("select a from tregular use index(idx_a) where a > %v order by a limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "IndexReader"), IsTrue) // check if indexReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test indexMerge + for i := 0; i < 100; i++ { + // explain select /*+ use_index_merge(t) */ * from t where a > 2 or b < 5 order by a limit {x}; // check if IndexMerge is used + // select /*+ use_index_merge(t) */ * from t where a > 2 or b < 5 order by a limit {x}; // can return the correct value + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > 2 or b < 5 order by a, b limit %v;", y) + queryRegular := fmt.Sprintf("select * from tregular where a > 2 or b < 5 order by a, b limit %v;", y) + c.Assert(tk.HasPlan(queryPartition, "IndexMerge"), IsTrue) // check if indexMerge is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } +} + func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 42b12f7cc324c18e31e64e7c3829588cb4c96237 Mon Sep 17 00:00:00 2001 From: disksing Date: Fri, 14 May 2021 18:01:38 +0800 Subject: [PATCH 15/16] store/tikv: remove options from unionstore (#24629) --- store/driver/txn/txn_driver.go | 6 +----- store/tikv/txn.go | 16 ---------------- store/tikv/unionstore/union_store.go | 24 ------------------------ 3 files changed, 1 insertion(+), 45 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 72c1aac8c8e71..7c940561f82fe 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -168,8 +168,6 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.GetSnapshot().SetIsStatenessReadOnly(val.(bool)) case kv.MatchStoreLabels: txn.KVTxn.GetSnapshot().SetMatchStoreLabels(val.([]*metapb.StoreLabel)) - default: - txn.KVTxn.SetOption(opt, val) } } @@ -180,7 +178,7 @@ func (txn *tikvTxn) GetOption(opt int) interface{} { case kv.TxnScope: return txn.KVTxn.GetScope() default: - return txn.KVTxn.GetOption(opt) + return nil } } @@ -188,8 +186,6 @@ func (txn *tikvTxn) DelOption(opt int) { switch opt { case kv.CollectRuntimeStats: txn.KVTxn.GetSnapshot().SetRuntimeStats(nil) - default: - txn.KVTxn.DelOption(opt) } } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index e444f5adda7f6..aafaa2b323d24 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -221,22 +221,6 @@ func (txn *KVTxn) Delete(k []byte) error { return txn.us.GetMemBuffer().Delete(k) } -// SetOption sets an option with a value, when val is nil, uses the default -// value of this option. -func (txn *KVTxn) SetOption(opt int, val interface{}) { - txn.us.SetOption(opt, val) -} - -// GetOption returns the option -func (txn *KVTxn) GetOption(opt int) interface{} { - return txn.us.GetOption(opt) -} - -// DelOption deletes an option. -func (txn *KVTxn) DelOption(opt int) { - txn.us.DelOption(opt) -} - // SetSchemaLeaseChecker sets a hook to check schema version. func (txn *KVTxn) SetSchemaLeaseChecker(checker SchemaLeaseChecker) { txn.schemaLeaseChecker = checker diff --git a/store/tikv/unionstore/union_store.go b/store/tikv/unionstore/union_store.go index f9a077d1c1352..08354975e38c5 100644 --- a/store/tikv/unionstore/union_store.go +++ b/store/tikv/unionstore/union_store.go @@ -59,7 +59,6 @@ type uSnapshot interface { type KVUnionStore struct { memBuffer *MemDB snapshot uSnapshot - opts options } // NewUnionStore builds a new unionStore. @@ -67,7 +66,6 @@ func NewUnionStore(snapshot uSnapshot) *KVUnionStore { return &KVUnionStore{ snapshot: snapshot, memBuffer: newMemDB(), - opts: make(map[int]interface{}), } } @@ -131,30 +129,8 @@ func (us *KVUnionStore) UnmarkPresumeKeyNotExists(k []byte) { us.memBuffer.UpdateFlags(k, kv.DelPresumeKeyNotExists) } -// SetOption implements the unionStore SetOption interface. -func (us *KVUnionStore) SetOption(opt int, val interface{}) { - us.opts[opt] = val -} - -// DelOption implements the unionStore DelOption interface. -func (us *KVUnionStore) DelOption(opt int) { - delete(us.opts, opt) -} - -// GetOption implements the unionStore GetOption interface. -func (us *KVUnionStore) GetOption(opt int) interface{} { - return us.opts[opt] -} - // SetEntrySizeLimit sets the size limit for each entry and total buffer. func (us *KVUnionStore) SetEntrySizeLimit(entryLimit, bufferLimit uint64) { us.memBuffer.entrySizeLimit = entryLimit us.memBuffer.bufferSizeLimit = bufferLimit } - -type options map[int]interface{} - -func (opts options) Get(opt int) (interface{}, bool) { - v, ok := opts[opt] - return v, ok -} From e92df204b53310029b7c32c5143ab6cebbf7ff18 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Fri, 14 May 2021 19:09:38 +0800 Subject: [PATCH 16/16] planner: fix wrongly check for update statement (#24614) --- executor/write_test.go | 5 ++-- planner/core/integration_test.go | 14 ++++++++++ planner/core/logical_plan_builder.go | 39 +++++++++++++++------------- planner/core/logical_plan_test.go | 1 - 4 files changed, 37 insertions(+), 22 deletions(-) diff --git a/executor/write_test.go b/executor/write_test.go index 27ea70ae748a5..b832e52a9935c 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1554,7 +1554,7 @@ func (s *testSuite8) TestUpdate(c *C) { _, err = tk.Exec("UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2") c.Assert(err, IsNil) - tk.MustExec("update (select * from t) t set c1 = 1111111") + tk.MustGetErrCode("update (select * from t) t set c1 = 1111111", mysql.ErrNonUpdatableTable) // test update ignore for bad null error tk.MustExec("drop table if exists t;") @@ -1604,8 +1604,7 @@ func (s *testSuite8) TestUpdate(c *C) { tk.MustExec("drop view v") tk.MustExec("create sequence seq") - _, err = tk.Exec("update seq set minvalue=1") - c.Assert(err.Error(), Equals, "update sequence seq is not supported now.") + tk.MustGetErrCode("update seq set minvalue=1", mysql.ErrBadField) tk.MustExec("drop sequence seq") tk.MustExec("drop table if exists t1, t2") diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 9e8eaa9204af9..62e5b032c86df 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -196,6 +196,20 @@ func (s *testIntegrationSuite) TestIssue22298(c *C) { tk.MustGetErrMsg(`select * from t where 0 and c = 10;`, "[planner:1054]Unknown column 'c' in 'where clause'") } +func (s *testIntegrationSuite) TestIssue24571(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec(`create view v as select 1 as b;`) + tk.MustExec(`create table t (a int);`) + tk.MustExec(`update v, t set a=2;`) + tk.MustGetErrCode(`update v, t set b=2;`, mysql.ErrNonUpdatableTable) + tk.MustExec("create database db1") + tk.MustExec("use db1") + tk.MustExec("update test.t, (select 1 as a) as t set test.t.a=1;") + // bug in MySQL: ERROR 1288 (HY000): The target table t of the UPDATE is not updatable + tk.MustExec("update (select 1 as a) as t, test.t set test.t.a=1;") +} + func (s *testIntegrationSuite) TestIssue22828(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 7ff0e2ac6c6aa..64bc0c41407e1 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4239,17 +4239,6 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( b.popTableHints() }() - // update subquery table should be forbidden - var notUpdatableTbl []string - notUpdatableTbl = extractTableSourceAsNames(update.TableRefs.TableRefs, notUpdatableTbl, true) - for _, asName := range notUpdatableTbl { - for _, assign := range update.List { - if assign.Column.Table.L == asName { - return nil, ErrNonUpdatableTable.GenWithStackByArgs(asName, "UPDATE") - } - } - } - b.inUpdateStmt = true b.isForUpdateRead = true @@ -4265,12 +4254,6 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( if dbName == "" { dbName = b.ctx.GetSessionVars().CurrentDB } - if t.TableInfo.IsView() { - return nil, errors.Errorf("update view %s is not supported now.", t.Name.O) - } - if t.TableInfo.IsSequence() { - return nil, errors.Errorf("update sequence %s is not supported now.", t.Name.O) - } b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, dbName, t.Name.L, "", nil) } @@ -4314,6 +4297,10 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( proj.SetChildren(p) p = proj + // update subquery table should be forbidden + var notUpdatableTbl []string + notUpdatableTbl = extractTableSourceAsNames(update.TableRefs.TableRefs, notUpdatableTbl, true) + var updateTableList []*ast.TableName updateTableList = extractTableList(update.TableRefs.TableRefs, updateTableList, true) orderedList, np, allAssignmentsAreConstant, err := b.buildUpdateLists(ctx, updateTableList, update.List, p, notUpdatableTbl) @@ -4417,6 +4404,21 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab columnsIdx[assign.Column] = idx } name := p.OutputNames()[idx] + for _, tl := range tableList { + if (tl.Schema.L == "" || tl.Schema.L == name.DBName.L) && (tl.Name.L == name.TblName.L) { + if tl.TableInfo.IsView() || tl.TableInfo.IsSequence() { + return nil, nil, false, ErrNonUpdatableTable.GenWithStackByArgs(name.TblName.O, "UPDATE") + } + // may be a subquery + if tl.Schema.L == "" { + for _, nTbl := range notUpdatableTbl { + if nTbl == name.TblName.L { + return nil, nil, false, ErrNonUpdatableTable.GenWithStackByArgs(name.TblName.O, "UPDATE") + } + } + } + } + } columnFullName := fmt.Sprintf("%s.%s.%s", name.DBName.L, name.TblName.L, name.ColName.L) // We save a flag for the column in map `modifyColumns` // This flag indicated if assign keyword `DEFAULT` to the column @@ -4439,9 +4441,10 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab break } } - if !updatable { + if !updatable || tn.TableInfo.IsView() || tn.TableInfo.IsSequence() { continue } + tableInfo := tn.TableInfo tableVal, found := b.is.TableByID(tableInfo.ID) if !found { diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 11a116bb4fac8..921f1c99b34ec 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1459,7 +1459,6 @@ func (s *testPlanSuite) TestNameResolver(c *C) { {"delete a from (select * from t ) as a, t", "[planner:1288]The target table a of the DELETE is not updatable"}, {"delete b from (select * from t ) as a, t", "[planner:1109]Unknown table 'b' in MULTI DELETE"}, {"select '' as fakeCol from t group by values(fakeCol)", "[planner:1054]Unknown column '' in 'VALUES() function'"}, - {"update t, (select * from t) as b set b.a = t.a", "[planner:1288]The target table b of the UPDATE is not updatable"}, {"select row_number() over () from t group by 1", "[planner:1056]Can't group on 'row_number() over ()'"}, {"select row_number() over () as x from t group by 1", "[planner:1056]Can't group on 'x'"}, {"select sum(a) as x from t group by 1", "[planner:1056]Can't group on 'x'"},