From 0cb32a128b902b2ff829afcb0bca664942d8be52 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=99=8E?= Date: Wed, 28 Apr 2021 14:15:56 +0800 Subject: [PATCH 01/36] executor: fix projection executor panic and add failpoint test (#24231) --- executor/aggregate.go | 69 +++++++++++++++++++++++++-------------- executor/executor.go | 12 +++++-- executor/executor_test.go | 37 +++++++++++++++++++++ executor/projection.go | 10 +++++- 4 files changed, 100 insertions(+), 28 deletions(-) diff --git a/executor/aggregate.go b/executor/aggregate.go index 46bec05302ace..7cd0165a17691 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -186,9 +186,10 @@ type HashAggExec struct { isChildReturnEmpty bool // After we support parallel execution for aggregation functions with distinct, // we can remove this attribute. - isUnparallelExec bool - prepared bool - executed bool + isUnparallelExec bool + parallelExecInitialized bool + prepared bool + executed bool memTracker *memory.Tracker // track memory usage. @@ -226,38 +227,42 @@ func (d *HashAggIntermData) getPartialResultBatch(sc *stmtctx.StatementContext, // Close implements the Executor Close interface. func (e *HashAggExec) Close() error { if e.isUnparallelExec { - e.memTracker.Consume(-e.childResult.MemoryUsage()) e.childResult = nil e.groupSet, _ = set.NewStringSetWithMemoryUsage() e.partialResultMap = nil - e.memTracker.ReplaceBytesUsed(0) + if e.memTracker != nil { + e.memTracker.ReplaceBytesUsed(0) + } return e.baseExecutor.Close() } - // `Close` may be called after `Open` without calling `Next` in test. - if !e.prepared { - close(e.inputCh) + if e.parallelExecInitialized { + // `Close` may be called after `Open` without calling `Next` in test. + if !e.prepared { + close(e.inputCh) + for _, ch := range e.partialOutputChs { + close(ch) + } + for _, ch := range e.partialInputChs { + close(ch) + } + close(e.finalOutputCh) + } + close(e.finishCh) for _, ch := range e.partialOutputChs { - close(ch) + for range ch { + } } for _, ch := range e.partialInputChs { - close(ch) + for range ch { + } } - close(e.finalOutputCh) - } - close(e.finishCh) - for _, ch := range e.partialOutputChs { - for range ch { + for range e.finalOutputCh { } - } - for _, ch := range e.partialInputChs { - for chk := range ch { - e.memTracker.Consume(-chk.MemoryUsage()) + e.executed = false + if e.memTracker != nil { + e.memTracker.ReplaceBytesUsed(0) } } - for range e.finalOutputCh { - } - e.executed = false - e.memTracker.ReplaceBytesUsed(0) return e.baseExecutor.Close() } @@ -266,6 +271,11 @@ func (e *HashAggExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { return err } + failpoint.Inject("mockHashAggExecBaseExecutorOpenReturnedError", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.New("mock HashAggExec.baseExecutor.Open returned error")) + } + }) e.prepared = false e.memTracker = memory.NewTracker(e.id, -1) @@ -368,6 +378,8 @@ func (e *HashAggExec) initForParallelExec(ctx sessionctx.Context) { e.finalWorkers[i] = w e.finalWorkers[i].finalResultHolderCh <- newFirstChunk(e) } + + e.parallelExecInitialized = true } func (w *HashAggPartialWorker) getChildInput() bool { @@ -1092,6 +1104,11 @@ func (e *StreamAggExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { return err } + failpoint.Inject("mockStreamAggExecBaseExecutorOpenReturnedError", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.New("mock StreamAggExec.baseExecutor.Open returned error")) + } + }) e.childResult = newFirstChunk(e.children[0]) e.executed = false e.isChildReturnEmpty = true @@ -1117,8 +1134,10 @@ func (e *StreamAggExec) Open(ctx context.Context) error { // Close implements the Executor Close interface. func (e *StreamAggExec) Close() error { - e.memTracker.Consume(-e.childResult.MemoryUsage() - e.memUsageOfInitialPartialResult) - e.childResult = nil + if e.childResult != nil { + e.memTracker.Consume(-e.childResult.MemoryUsage() - e.memUsageOfInitialPartialResult) + e.childResult = nil + } e.groupChecker.reset() return e.baseExecutor.Close() } diff --git a/executor/executor.go b/executor/executor.go index 6bb07b93eb46c..e5d5d44efefe3 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -28,6 +28,7 @@ import ( "github.com/cznic/mathutil" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" @@ -1217,6 +1218,11 @@ func (e *SelectionExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { return err } + failpoint.Inject("mockSelectionExecBaseExecutorOpenReturnedError", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.New("mock SelectionExec.baseExecutor.Open returned error")) + } + }) return e.open(ctx) } @@ -1236,8 +1242,10 @@ func (e *SelectionExec) open(ctx context.Context) error { // Close implements plannercore.Plan Close interface. func (e *SelectionExec) Close() error { - e.memTracker.Consume(-e.childResult.MemoryUsage()) - e.childResult = nil + if e.childResult != nil { + e.memTracker.Consume(-e.childResult.MemoryUsage()) + e.childResult = nil + } e.selected = nil return e.baseExecutor.Close() } diff --git a/executor/executor_test.go b/executor/executor_test.go index aaebfd0ada142..80056439ec7c6 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8098,3 +8098,40 @@ func (s *testSuite1) TestIssue24091(c *C) { tk.MustQuery("select null div 0;").Check(testkit.Rows("")) tk.MustQuery("select * from t;").Check(testkit.Rows("")) } + +func (s *testSerialSuite) TestIssue24210(c *C) { + tk := testkit.NewTestKit(c, s.store) + + // for ProjectionExec + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockProjectionExecBaseExecutorOpenReturnedError", `return(true)`), IsNil) + _, err := tk.Exec("select a from (select 1 as a, 2 as b) t") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "mock ProjectionExec.baseExecutor.Open returned error") + err = failpoint.Disable("github.com/pingcap/tidb/executor/mockProjectionExecBaseExecutorOpenReturnedError") + c.Assert(err, IsNil) + + // for HashAggExec + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockHashAggExecBaseExecutorOpenReturnedError", `return(true)`), IsNil) + _, err = tk.Exec("select sum(a) from (select 1 as a, 2 as b) t group by b") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "mock HashAggExec.baseExecutor.Open returned error") + err = failpoint.Disable("github.com/pingcap/tidb/executor/mockHashAggExecBaseExecutorOpenReturnedError") + c.Assert(err, IsNil) + + // for StreamAggExec + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStreamAggExecBaseExecutorOpenReturnedError", `return(true)`), IsNil) + _, err = tk.Exec("select sum(a) from (select 1 as a, 2 as b) t") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "mock StreamAggExec.baseExecutor.Open returned error") + err = failpoint.Disable("github.com/pingcap/tidb/executor/mockStreamAggExecBaseExecutorOpenReturnedError") + c.Assert(err, IsNil) + + // for SelectionExec + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError", `return(true)`), IsNil) + _, err = tk.Exec("select * from (select 1 as a) t where a > 0") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "mock SelectionExec.baseExecutor.Open returned error") + err = failpoint.Disable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError") + c.Assert(err, IsNil) + +} diff --git a/executor/projection.go b/executor/projection.go index 3e055a4845964..5e0f54d258a26 100644 --- a/executor/projection.go +++ b/executor/projection.go @@ -21,6 +21,7 @@ import ( "sync/atomic" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util" @@ -85,6 +86,11 @@ func (e *ProjectionExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { return err } + failpoint.Inject("mockProjectionExecBaseExecutorOpenReturnedError", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.New("mock ProjectionExec.baseExecutor.Open returned error")) + } + }) return e.open(ctx) } @@ -290,7 +296,9 @@ func (e *ProjectionExec) drainOutputCh(ch chan *projectionOutput) { // Close implements the Executor Close interface. func (e *ProjectionExec) Close() error { - if e.isUnparallelExec() { + // if e.baseExecutor.Open returns error, e.childResult will be nil, see https://github.com/pingcap/tidb/issues/24210 + // for more information + if e.isUnparallelExec() && e.childResult != nil { e.memTracker.Consume(-e.childResult.MemoryUsage()) e.childResult = nil } From f07b62fcdfc085eacdecb938b2c5c3fea31b899a Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 28 Apr 2021 15:09:56 +0800 Subject: [PATCH 02/36] executor: fix data race of parallel apply operator (#24257) --- executor/parallel_apply.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/executor/parallel_apply.go b/executor/parallel_apply.go index d5fc7df17663e..d02ebac9e5349 100644 --- a/executor/parallel_apply.go +++ b/executor/parallel_apply.go @@ -157,12 +157,14 @@ func (e *ParallelNestedLoopApplyExec) Next(ctx context.Context, req *chunk.Chunk // Close implements the Executor interface. func (e *ParallelNestedLoopApplyExec) Close() error { e.memTracker = nil - err := e.outerExec.Close() if atomic.LoadUint32(&e.started) == 1 { close(e.exit) e.notifyWg.Wait() e.started = 0 } + // Wait all workers to finish before Close() is called. + // Otherwise we may got data race. + err := e.outerExec.Close() if e.runtimeStats != nil { runtimeStats := newJoinRuntimeStats() From 619c4720059ea619081b01644ef3084b426d282f Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 28 Apr 2021 15:25:56 +0800 Subject: [PATCH 03/36] store/tikv: remove IsolationLevel option (#24335) --- distsql/request_builder.go | 8 ++++---- distsql/request_builder_test.go | 2 +- executor/analyze.go | 4 ++-- kv/kv.go | 12 +++++++++++- session/session_test.go | 4 ++-- store/copr/batch_coprocessor.go | 2 +- store/copr/coprocessor.go | 13 ++++++++++++- store/driver/txn/snapshot.go | 22 ++++++++++++++++++++++ store/driver/txn/txn_driver.go | 3 +++ store/tikv/kv/kv.go | 10 ---------- store/tikv/region_request.go | 12 ------------ store/tikv/scan.go | 2 +- store/tikv/snapshot.go | 25 ++++++++++++++++++++++--- 13 files changed, 81 insertions(+), 38 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index ac53de8a0c4a3..9dfd4124304b4 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -145,7 +145,7 @@ func (builder *RequestBuilder) SetAnalyzeRequest(ana *tipb.AnalyzeReq) *RequestB builder.Request.Tp = kv.ReqTypeAnalyze builder.Request.Data, builder.err = ana.Marshal() builder.Request.NotFillCache = true - builder.Request.IsolationLevel = tikvstore.RC + builder.Request.IsolationLevel = kv.RC builder.Request.Priority = tikvstore.PriorityLow } @@ -199,12 +199,12 @@ func (builder *RequestBuilder) SetAllowBatchCop(batchCop bool) *RequestBuilder { return builder } -func (builder *RequestBuilder) getIsolationLevel() tikvstore.IsoLevel { +func (builder *RequestBuilder) getIsolationLevel() kv.IsoLevel { switch builder.Tp { case kv.ReqTypeAnalyze: - return tikvstore.RC + return kv.RC } - return tikvstore.SI + return kv.SI } func (builder *RequestBuilder) getKVPriority(sv *variable.SessionVars) int { diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 5b945a9ea7986..18c1ee8fc24e4 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -533,7 +533,7 @@ func (s *testSuite) TestRequestBuilder5(c *C) { KeepOrder: true, Desc: false, Concurrency: 15, - IsolationLevel: tikvstore.RC, + IsolationLevel: kv.RC, Priority: 1, NotFillCache: true, SyncLog: false, diff --git a/executor/analyze.go b/executor/analyze.go index a08145aab0702..f595931d0c036 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -965,7 +965,7 @@ func (e *AnalyzeFastExec) activateTxnForRowCount() (rollbackFn func() error, err } } txn.SetOption(tikvstore.Priority, tikvstore.PriorityLow) - txn.SetOption(tikvstore.IsolationLevel, tikvstore.RC) + txn.SetOption(tikvstore.IsolationLevel, kv.RC) txn.SetOption(tikvstore.NotFillCache, true) return rollbackFn, nil } @@ -1185,7 +1185,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { defer e.wg.Done() snapshot := e.ctx.GetStore().GetSnapshot(kv.MaxVersion) snapshot.SetOption(tikvstore.NotFillCache, true) - snapshot.SetOption(tikvstore.IsolationLevel, tikvstore.RC) + snapshot.SetOption(tikvstore.IsolationLevel, kv.RC) snapshot.SetOption(tikvstore.Priority, tikvstore.PriorityLow) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) diff --git a/kv/kv.go b/kv/kv.go index 711dd4e3ee860..4f020bbc9b3f9 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -257,7 +257,7 @@ type Request struct { // sent to multiple storage units concurrently. Concurrency int // IsolationLevel is the isolation level, default is SI. - IsolationLevel tikvstore.IsoLevel + IsolationLevel IsoLevel // Priority is the priority of this KV request, its value may be PriorityNormal/PriorityLow/PriorityHigh. Priority int // memTracker is used to trace and control memory usage in co-processor layer. @@ -438,3 +438,13 @@ type SplittableStore interface { WaitScatterRegionFinish(ctx context.Context, regionID uint64, backOff int) error CheckRegionInScattering(regionID uint64) (bool, error) } + +// IsoLevel is the transaction's isolation level. +type IsoLevel int + +const ( + // SI stands for 'snapshot isolation'. + SI IsoLevel = iota + // RC stands for 'read committed'. + RC +) diff --git a/session/session_test.go b/session/session_test.go index 9ce875fa07868..91dd04abded29 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2624,13 +2624,13 @@ func (s *testSessionSuite3) TestSetTransactionIsolationOneShot(c *C) { // Check isolation level is set to read committed. ctx := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { - c.Assert(req.IsolationLevel, Equals, tikvstore.SI) + c.Assert(req.IsolationLevel, Equals, kv.SI) }) tk.Se.Execute(ctx, "select * from t where k = 1") // Check it just take effect for one time. ctx = context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { - c.Assert(req.IsolationLevel, Equals, tikvstore.SI) + c.Assert(req.IsolationLevel, Equals, kv.SI) }) tk.Se.Execute(ctx, "select * from t where k = 1") diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 6e3b8d9b6e903..af6fb47676d1a 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -345,7 +345,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *tikv.Backoffe } req := tikvrpc.NewRequest(task.cmdType, &copReq, kvrpcpb.Context{ - IsolationLevel: tikv.IsolationLevelToPB(b.req.IsolationLevel), + IsolationLevel: isolationLevelToPB(b.req.IsolationLevel), Priority: tikv.PriorityToPB(b.req.Priority), NotFillCache: b.req.NotFillCache, RecordTimeStat: true, diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index dd1555f0c614d..dbb9dc807d5e8 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -697,7 +697,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *tikv.Backoffer, task *copTas } req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, worker.req.ReplicaRead, &worker.replicaReadSeed, kvrpcpb.Context{ - IsolationLevel: tikv.IsolationLevelToPB(worker.req.IsolationLevel), + IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel), Priority: tikv.PriorityToPB(worker.req.Priority), NotFillCache: worker.req.NotFillCache, RecordTimeStat: true, @@ -1191,3 +1191,14 @@ func (e *rateLimitAction) setEnabled(enabled bool) { func (e *rateLimitAction) isEnabled() bool { return atomic.LoadUint32(&e.enabled) > 0 } + +func isolationLevelToPB(level kv.IsoLevel) kvrpcpb.IsolationLevel { + switch level { + case kv.RC: + return kvrpcpb.IsolationLevel_RC + case kv.SI: + return kvrpcpb.IsolationLevel_SI + default: + return kvrpcpb.IsolationLevel_SI + } +} diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index afcb9eaee3f58..5fa8f67f90905 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" + tikvstore "github.com/pingcap/tidb/store/tikv/kv" ) type tikvSnapshot struct { @@ -61,7 +62,28 @@ func (s *tikvSnapshot) IterReverse(k kv.Key) (kv.Iterator, error) { return &tikvScanner{scanner.(*tikv.Scanner)}, err } +func (s *tikvSnapshot) SetOption(opt int, val interface{}) { + switch opt { + case tikvstore.IsolationLevel: + level := getTiKVIsolationLevel(val.(kv.IsoLevel)) + s.KVSnapshot.SetIsolationLevel(level) + default: + s.KVSnapshot.SetOption(opt, val) + } +} + func toTiKVKeys(keys []kv.Key) [][]byte { bytesKeys := *(*[][]byte)(unsafe.Pointer(&keys)) return bytesKeys } + +func getTiKVIsolationLevel(level kv.IsoLevel) tikv.IsoLevel { + switch level { + case kv.SI: + return tikv.SI + case kv.RC: + return tikv.RC + default: + return tikv.SI + } +} diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 6b39c1cca1e24..a9e09a271ce5e 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -129,6 +129,9 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn: txn.KVTxn, binInfo: val.(*binloginfo.BinlogInfo), // val cannot be other type. }) + case tikvstore.IsolationLevel: + level := getTiKVIsolationLevel(val.(kv.IsoLevel)) + txn.KVTxn.GetSnapshot().SetIsolationLevel(level) default: txn.KVTxn.SetOption(opt, val) } diff --git a/store/tikv/kv/kv.go b/store/tikv/kv/kv.go index 28c919a95627f..2b7e87ecd2e47 100644 --- a/store/tikv/kv/kv.go +++ b/store/tikv/kv/kv.go @@ -7,16 +7,6 @@ import ( "github.com/pingcap/tidb/store/tikv/util" ) -// IsoLevel is the transaction's isolation level. -type IsoLevel int - -const ( - // SI stands for 'snapshot isolation'. - SI IsoLevel = iota - // RC stands for 'read committed'. - RC -) - // ReturnedValue pairs the Value and AlreadyLocked flag for PessimisticLock return values result. type ReturnedValue struct { Value []byte diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 9b501a8a31c31..cad0ed0379e96 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -737,15 +737,3 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed } return false, nil } - -// IsolationLevelToPB converts isolation level to wire type. -func IsolationLevelToPB(level kv.IsoLevel) kvrpcpb.IsolationLevel { - switch level { - case kv.RC: - return kvrpcpb.IsolationLevel_RC - case kv.SI: - return kvrpcpb.IsolationLevel_SI - default: - return kvrpcpb.IsolationLevel_SI - } -} diff --git a/store/tikv/scan.go b/store/tikv/scan.go index cc8d825384a68..d2d151f59627f 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -191,7 +191,7 @@ func (s *Scanner) getData(bo *Backoffer) error { Context: &pb.Context{ Priority: s.snapshot.priority, NotFillCache: s.snapshot.notFillCache, - IsolationLevel: IsolationLevelToPB(s.snapshot.isolationLevel), + IsolationLevel: s.snapshot.isolationLevel.ToPB(), }, StartKey: s.nextStartKey, EndKey: reqEndKey, diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index a02cc23a5db66..571a6b0d3da5a 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -26,6 +26,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" tikverr "github.com/pingcap/tidb/store/tikv/error" @@ -44,11 +45,26 @@ const ( maxTimestamp = math.MaxUint64 ) +// IsoLevel is the transaction's isolation level. +type IsoLevel kvrpcpb.IsolationLevel + +const ( + // SI stands for 'snapshot isolation'. + SI IsoLevel = IsoLevel(kvrpcpb.IsolationLevel_SI) + // RC stands for 'read committed'. + RC IsoLevel = IsoLevel(kvrpcpb.IsolationLevel_RC) +) + +// ToPB converts isolation level to wire type. +func (l IsoLevel) ToPB() kvrpcpb.IsolationLevel { + return kvrpcpb.IsolationLevel(l) +} + // KVSnapshot implements the tidbkv.Snapshot interface. type KVSnapshot struct { store *KVStore version uint64 - isolationLevel kv.IsoLevel + isolationLevel IsoLevel priority pb.CommandPri notFillCache bool syncLog bool @@ -535,8 +551,6 @@ func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) { // value of this option. Only ReplicaRead is supported for snapshot func (s *KVSnapshot) SetOption(opt int, val interface{}) { switch opt { - case kv.IsolationLevel: - s.isolationLevel = val.(kv.IsoLevel) case kv.Priority: s.priority = PriorityToPB(val.(int)) case kv.NotFillCache: @@ -588,6 +602,11 @@ func (s *KVSnapshot) DelOption(opt int) { } } +// SetIsolationLevel sets the isolation level used to scan data from tikv. +func (s *KVSnapshot) SetIsolationLevel(level IsoLevel) { + s.isolationLevel = level +} + // SnapCacheHitCount gets the snapshot cache hit count. Only for test. func (s *KVSnapshot) SnapCacheHitCount() int { return int(atomic.LoadInt64(&s.mu.hitCnt)) From 8da5a1f113caf24929f2c7535fe077c7692d73a6 Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Wed, 28 Apr 2021 15:59:56 +0800 Subject: [PATCH 04/36] planner, expression: support enum index scan (#22691) --- expression/integration_test.go | 32 ++++ planner/core/physical_plan_test.go | 33 +++++ planner/core/testdata/plan_suite_in.json | 23 +++ planner/core/testdata/plan_suite_out.json | 170 ++++++++++++++++++++++ util/ranger/checker.go | 4 - util/ranger/points.go | 48 ++++++ util/ranger/ranger.go | 10 ++ 7 files changed, 316 insertions(+), 4 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index c5b3bdd6d25aa..587b8903fb075 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -19,6 +19,7 @@ import ( "encoding/hex" "fmt" "math" + "math/rand" "sort" "strconv" "strings" @@ -9287,3 +9288,34 @@ func (s *testIntegrationSuite) TestRefineArgNullValues(c *C) { "", )) } + +func (s *testIntegrationSuite) TestEnumIndex(c *C) { + defer s.cleanEnv(c) + + elems := []string{"\"a\"", "\"b\"", "\"c\""} + rand.Shuffle(len(elems), func(i, j int) { + elems[i], elems[j] = elems[j], elems[i] + }) + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t,tidx") + tk.MustExec("create table t(e enum(" + strings.Join(elems, ",") + "))") + tk.MustExec("create table tidx(e enum(" + strings.Join(elems, ",") + "), index idx(e))") + + nRows := 50 + values := make([]string, 0, nRows) + for i := 0; i < nRows; i++ { + values = append(values, fmt.Sprintf("(%v)", rand.Intn(len(elems))+1)) + } + tk.MustExec(fmt.Sprintf("insert into t values %v", strings.Join(values, ", "))) + tk.MustExec(fmt.Sprintf("insert into tidx values %v", strings.Join(values, ", "))) + + ops := []string{"=", "!=", ">", ">=", "<", "<="} + testElems := []string{"\"a\"", "\"b\"", "\"c\"", "\"d\"", "\"\"", "1", "2", "3", "4", "0", "-1"} + for i := 0; i < nRows; i++ { + cond := fmt.Sprintf("e" + ops[rand.Intn(len(ops))] + testElems[rand.Intn(len(testElems))]) + result := tk.MustQuery("select * from t where " + cond).Sort().Rows() + tk.MustQuery("select * from tidx where " + cond).Sort().Check(result) + } +} diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index c74365bb66384..d3dbeefa1d36b 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -1700,3 +1700,36 @@ func (s *testPlanSuite) TestNthPlanHintWithExplain(c *C) { // hold in the future, you may need to modify this. tk.MustQuery("explain format = 'brief' select * from test.tt where a=1 and b=1").Check(testkit.Rows(output[1].Plan...)) } + +func (s *testPlanSuite) TestEnumIndex(c *C) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + s.testData.GetTestCases(c, &input, &output) + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(e enum('c','b','a'), index idx(e))") + tk.MustExec("insert into t values(1),(2),(3);") + + for i, ts := range input { + s.testData.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) + } +} diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index ea257f22a332c..64a5c973e89cd 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -676,5 +676,28 @@ "select a from t2 where t2.a < (select t1.a from t1 where t1.b = t2.b and t1.a is null);", "select a from t2 where t2.a < (select t3.a from t3 where t3.a = t2.a);" ] + }, + { + "name": "TestEnumIndex", + "cases": [ + "select e from t where e = 'b'", + "select e from t where e != 'b'", + "select e from t where e > 'b'", + "select e from t where e >= 'b'", + "select e from t where e < 'b'", + "select e from t where e <= 'b'", + "select e from t where e = 2", + "select e from t where e != 2", + "select e from t where e > 2", + "select e from t where e >= 2", + "select e from t where e < 2", + "select e from t where e <= 2", + + // Out of range + "select e from t where e > ''", + "select e from t where e > 'd'", + "select e from t where e > -1", + "select e from t where e > 5" + ] } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 2b3c55eddd5bf..a0125e601c94a 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -2280,5 +2280,175 @@ "Result": null } ] + }, + { + "Name": "TestEnumIndex", + "Cases": [ + { + "SQL": "select e from t where e = 'b'", + "Plan": [ + "IndexReader 10.00 root index:IndexRangeScan", + "└─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(e) range:[\"b\",\"b\"], keep order:false, stats:pseudo" + ], + "Result": [ + "b" + ] + }, + { + "SQL": "select e from t where e != 'b'", + "Plan": [ + "IndexReader 20.00 root index:IndexRangeScan", + "└─IndexRangeScan 20.00 cop[tikv] table:t, index:idx(e) range:[\"c\",\"c\"], [\"a\",\"a\"], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "c" + ] + }, + { + "SQL": "select e from t where e > 'b'", + "Plan": [ + "IndexReader 10.00 root index:IndexRangeScan", + "└─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(e) range:[\"c\",\"c\"], keep order:false, stats:pseudo" + ], + "Result": [ + "c" + ] + }, + { + "SQL": "select e from t where e >= 'b'", + "Plan": [ + "IndexReader 20.00 root index:IndexRangeScan", + "└─IndexRangeScan 20.00 cop[tikv] table:t, index:idx(e) range:[\"c\",\"c\"], [\"b\",\"b\"], keep order:false, stats:pseudo" + ], + "Result": [ + "b", + "c" + ] + }, + { + "SQL": "select e from t where e < 'b'", + "Plan": [ + "IndexReader 10.00 root index:IndexRangeScan", + "└─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(e) range:[\"a\",\"a\"], keep order:false, stats:pseudo" + ], + "Result": [ + "a" + ] + }, + { + "SQL": "select e from t where e <= 'b'", + "Plan": [ + "IndexReader 20.00 root index:IndexRangeScan", + "└─IndexRangeScan 20.00 cop[tikv] table:t, index:idx(e) range:[\"b\",\"b\"], [\"a\",\"a\"], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "b" + ] + }, + { + "SQL": "select e from t where e = 2", + "Plan": [ + "IndexReader 10.00 root index:IndexRangeScan", + "└─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(e) range:[\"b\",\"b\"], keep order:false, stats:pseudo" + ], + "Result": [ + "b" + ] + }, + { + "SQL": "select e from t where e != 2", + "Plan": [ + "IndexReader 6656.67 root index:IndexRangeScan", + "└─IndexRangeScan 6656.67 cop[tikv] table:t, index:idx(e) range:[-inf,\"b\"), (\"b\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "c" + ] + }, + { + "SQL": "select e from t where e > 2", + "Plan": [ + "IndexReader 3333.33 root index:IndexRangeScan", + "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:(\"b\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "a" + ] + }, + { + "SQL": "select e from t where e >= 2", + "Plan": [ + "IndexReader 3333.33 root index:IndexRangeScan", + "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:[\"b\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "b" + ] + }, + { + "SQL": "select e from t where e < 2", + "Plan": [ + "IndexReader 3323.33 root index:IndexRangeScan", + "└─IndexRangeScan 3323.33 cop[tikv] table:t, index:idx(e) range:[-inf,\"b\"), keep order:false, stats:pseudo" + ], + "Result": [ + "c" + ] + }, + { + "SQL": "select e from t where e <= 2", + "Plan": [ + "IndexReader 3323.33 root index:IndexRangeScan", + "└─IndexRangeScan 3323.33 cop[tikv] table:t, index:idx(e) range:[-inf,\"b\"], keep order:false, stats:pseudo" + ], + "Result": [ + "b", + "c" + ] + }, + { + "SQL": "select e from t where e > ''", + "Plan": [ + "IndexReader 30.00 root index:IndexRangeScan", + "└─IndexRangeScan 30.00 cop[tikv] table:t, index:idx(e) range:[\"c\",\"c\"], [\"b\",\"b\"], [\"a\",\"a\"], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "b", + "c" + ] + }, + { + "SQL": "select e from t where e > 'd'", + "Plan": [ + "TableDual 0.00 root rows:0" + ], + "Result": null + }, + { + "SQL": "select e from t where e > -1", + "Plan": [ + "IndexReader 3333.33 root index:IndexRangeScan", + "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:[\"\",+inf], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "b", + "c" + ] + }, + { + "SQL": "select e from t where e > 5", + "Plan": [ + "IndexReader 3333.33 root index:IndexRangeScan", + "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:(\"a\",+inf], keep order:false, stats:pseudo" + ], + "Result": null + } + ] } ] diff --git a/util/ranger/checker.go b/util/ranger/checker.go index 3f702b35d18b6..97a0c782ce3dc 100644 --- a/util/ranger/checker.go +++ b/util/ranger/checker.go @@ -15,7 +15,6 @@ package ranger import ( "github.com/pingcap/parser/ast" - "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" @@ -162,8 +161,5 @@ func (c *conditionChecker) checkColumn(expr expression.Expression) bool { if !ok { return false } - if col.GetType().Tp == mysql.TypeEnum { - return false - } return c.colUniqueID == col.UniqueID } diff --git a/util/ranger/points.go b/util/ranger/points.go index e12233062b8a2..19d7c18d9ecf7 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -319,6 +319,10 @@ func (r *builder) buildFormBinOp(expr *expression.ScalarFunction) []*point { return nil } + if ft.Tp == mysql.TypeEnum && ft.EvalType() == types.ETString { + return handleEnumFromBinOp(r.sc, ft, value, op) + } + switch op { case ast.NullEQ: if value.IsNull() { @@ -433,6 +437,50 @@ func handleBoundCol(ft *types.FieldType, val types.Datum, op string) (types.Datu return val, op, true } +func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val types.Datum, op string) []*point { + res := make([]*point, 0, len(ft.Elems)*2) + appendPointFunc := func(d types.Datum) { + res = append(res, &point{value: d, excl: false, start: true}) + res = append(res, &point{value: d, excl: false, start: false}) + } + + tmpEnum := types.Enum{} + for i := range ft.Elems { + tmpEnum.Name = ft.Elems[i] + tmpEnum.Value = uint64(i) + d := types.NewMysqlEnumDatum(tmpEnum) + if v, err := d.CompareDatum(sc, &val); err == nil { + switch op { + case ast.LT: + if v < 0 { + appendPointFunc(d) + } + case ast.LE: + if v <= 0 { + appendPointFunc(d) + } + case ast.GT: + if v > 0 { + appendPointFunc(d) + } + case ast.GE: + if v >= 0 { + appendPointFunc(d) + } + case ast.EQ: + if v == 0 { + appendPointFunc(d) + } + case ast.NE: + if v != 0 { + appendPointFunc(d) + } + } + } + } + return res +} + func (r *builder) buildFromIsTrue(expr *expression.ScalarFunction, isNot int, keepNull bool) []*point { if isNot == 1 { if keepNull { diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index a414fb7fec75a..4f1efef7a7d90 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -101,6 +101,16 @@ func convertPoint(sc *stmtctx.StatementContext, point *point, tp *types.FieldTyp // Ignore the types.ErrOverflow when we convert TypeNewDecimal values. // A trimmed valid boundary point value would be returned then. Accordingly, the `excl` of the point // would be adjusted. Impossible ranges would be skipped by the `validInterval` call later. + } else if tp.Tp == mysql.TypeEnum && terror.ErrorEqual(err, types.ErrTruncated) { + // Ignore the types.ErrorTruncated when we convert TypeEnum values. + // We should cover Enum upper overflow, and convert to the biggest value. + if point.value.GetInt64() > 0 { + upperEnum, err := types.ParseEnumValue(tp.Elems, uint64(len(tp.Elems))) + if err != nil { + return nil, err + } + casted.SetMysqlEnum(upperEnum, tp.Collate) + } } else { return point, errors.Trace(err) } From b7838d885aefa042646dc423342d70c333f10c2c Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Wed, 28 Apr 2021 16:23:56 +0800 Subject: [PATCH 05/36] planner: clone possible properties before saving them (#24204) --- planner/core/physical_plan_test.go | 19 +++++++++++++++++ planner/core/property_cols_prune.go | 33 ++++++++++++++++++++++++----- 2 files changed, 47 insertions(+), 5 deletions(-) diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index d3dbeefa1d36b..6a8993286e36a 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -1733,3 +1733,22 @@ func (s *testPlanSuite) TestEnumIndex(c *C) { tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) } } + +func (s *testPlanSuite) TestPossibleProperties(c *C) { + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists student, sc") + tk.MustExec("create table student(id int primary key auto_increment, name varchar(4) not null)") + tk.MustExec("create table sc(id int primary key auto_increment, student_id int not null, course_id int not null, score int not null)") + tk.MustExec("insert into student values (1,'s1'), (2,'s2')") + tk.MustExec("insert into sc (student_id, course_id, score) values (1,1,59), (1,2,57), (1,3,76), (2,1,99), (2,2,100), (2,3,100)") + tk.MustQuery("select /*+ stream_agg() */ a.id, avg(b.score) as afs from student a join sc b on a.id = b.student_id where b.score < 60 group by a.id having count(b.course_id) >= 2").Check(testkit.Rows( + "1 58.0000", + )) +} diff --git a/planner/core/property_cols_prune.go b/planner/core/property_cols_prune.go index c8296aa799283..91a9f34fb9017 100644 --- a/planner/core/property_cols_prune.go +++ b/planner/core/property_cols_prune.go @@ -165,13 +165,27 @@ func (p *LogicalProjection) PreparePossibleProperties(schema *expression.Schema, return childProperties } +func clonePossibleProperties(props [][]*expression.Column) [][]*expression.Column { + res := make([][]*expression.Column, len(props)) + for i, prop := range props { + clonedProp := make([]*expression.Column, len(prop)) + for j, col := range prop { + clonedProp[j] = col.Clone().(*expression.Column) + } + res[i] = clonedProp + } + return res +} + // PreparePossibleProperties implements LogicalPlan PreparePossibleProperties interface. func (p *LogicalJoin) PreparePossibleProperties(schema *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column { leftProperties := childrenProperties[0] rightProperties := childrenProperties[1] // TODO: We should consider properties propagation. - p.leftProperties = leftProperties - p.rightProperties = rightProperties + // Clone the Columns in the property before saving them, otherwise the upper Projection may + // modify them and lead to unexpected results. + p.leftProperties = clonePossibleProperties(leftProperties) + p.rightProperties = clonePossibleProperties(rightProperties) if p.JoinType == LeftOuterJoin || p.JoinType == LeftOuterSemiJoin { rightProperties = nil } else if p.JoinType == RightOuterJoin { @@ -200,13 +214,22 @@ func (la *LogicalAggregation) PreparePossibleProperties(schema *expression.Schem return nil } resultProperties := make([][]*expression.Column, 0, len(childProps)) + clonedProperties := make([][]*expression.Column, 0, len(childProps)) groupByCols := la.GetGroupByCols() for _, possibleChildProperty := range childProps { sortColOffsets := getMaxSortPrefix(possibleChildProperty, groupByCols) if len(sortColOffsets) == len(groupByCols) { - resultProperties = append(resultProperties, possibleChildProperty[:len(groupByCols)]) + prop := possibleChildProperty[:len(groupByCols)] + resultProperties = append(resultProperties, prop) + // Clone the Columns in the property before saving them, otherwise the upper Projection may + // modify them and lead to unexpected results. + clonedProp := make([]*expression.Column, len(prop)) + for i, col := range prop { + clonedProp[i] = col.Clone().(*expression.Column) + } + clonedProperties = append(clonedProperties, clonedProp) } } - la.possibleProperties = resultProperties - return la.possibleProperties + la.possibleProperties = clonedProperties + return resultProperties } From e79fa8c6b654e5b94e9ed0a1c0f997d6564e95be Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 28 Apr 2021 16:43:56 +0800 Subject: [PATCH 06/36] planner: let CopTiFlashConcurrencyFactor inflence the cost of whole plan (#24157) --- planner/core/task.go | 7 ++++--- .../core/testdata/integration_serial_suite_out.json | 12 ++++++------ 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/planner/core/task.go b/planner/core/task.go index 68b6c627e9165..82045579be155 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -2028,11 +2028,12 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { StoreType: kv.TiFlash, }.Init(ctx, t.p.SelectBlockOffset()) p.stats = t.p.statsInfo() - p.cost = t.cost() - return &rootTask{ + rt := &rootTask{ p: p, - cst: t.cst / 20, // TODO: This is tricky because mpp doesn't run in a coprocessor way. + cst: t.cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor, } + p.cost = rt.cost() + return rt } func (t *mppTask) needEnforce(prop *property.PhysicalProperty) bool { diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 9512f3bbd1fcc..fcd6bfd7dee44 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -102,7 +102,7 @@ { "SQL": "explain format = 'verbose' select count(*) from t2 group by a", "Plan": [ - "TableReader_24 3.00 77.00 root data:ExchangeSender_23", + "TableReader_24 3.00 3.21 root data:ExchangeSender_23", "└─ExchangeSender_23 3.00 77.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection_22 3.00 0.00 batchCop[tiflash] Column#4", " └─HashAgg_8 3.00 77.00 batchCop[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", @@ -152,8 +152,8 @@ { "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a", "Plan": [ - "StreamAgg_12 1.00 20.77 root funcs:count(1)->Column#7", - "└─TableReader_44 3.00 235.38 root data:ExchangeSender_43", + "StreamAgg_12 1.00 18.81 root funcs:count(1)->Column#7", + "└─TableReader_44 3.00 9.81 root data:ExchangeSender_43", " └─ExchangeSender_43 3.00 235.38 cop[tiflash] ExchangeType: PassThrough", " └─HashJoin_40 3.00 235.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─ExchangeReceiver_19(Build) 3.00 77.00 cop[tiflash] ", @@ -167,11 +167,11 @@ { "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", "Plan": [ - "StreamAgg_15 1.00 62.68 root funcs:count(1)->Column#10", - "└─HashJoin_65 3.00 53.68 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + "StreamAgg_15 1.00 60.48 root funcs:count(1)->Column#10", + "└─HashJoin_65 3.00 51.48 root inner join, equal:[eq(test.t1.b, test.t3.b)]", " ├─IndexReader_53(Build) 3.00 11.66 root index:IndexFullScan_52", " │ └─IndexFullScan_52 3.00 150.50 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_39(Probe) 3.00 264.38 root data:ExchangeSender_38", + " └─TableReader_39(Probe) 3.00 11.02 root data:ExchangeSender_38", " └─ExchangeSender_38 3.00 264.38 cop[tiflash] ExchangeType: PassThrough", " └─HashJoin_29 3.00 264.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─ExchangeReceiver_35(Build) 3.00 106.00 cop[tiflash] ", From 0d76e899f42510f458b99f2c9083ce53d434f0e9 Mon Sep 17 00:00:00 2001 From: Zhuomin Liu Date: Wed, 28 Apr 2021 17:25:56 +0800 Subject: [PATCH 07/36] expression: fix wrong type infer for agg function when type is null (#24290) --- executor/window_test.go | 28 ++++++++++++++++++++++++++++ expression/aggregation/base_func.go | 3 +++ 2 files changed, 31 insertions(+) diff --git a/executor/window_test.go b/executor/window_test.go index 219113cf38c0c..e530aa03f0549 100644 --- a/executor/window_test.go +++ b/executor/window_test.go @@ -402,3 +402,31 @@ func baseTestSlidingWindowFunctions(tk *testkit.TestKit) { result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") result.Check(testkit.Rows(" 11", " 11", "M 5", "F 5", "F 4", "F 3", "M 2")) } + +func (s *testSuite7) TestIssue24264(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists tbl_2") + tk.MustExec("create table tbl_2 ( col_10 char(65) collate utf8mb4_unicode_ci not null , col_11 bigint not null , col_12 datetime not null , col_13 bigint unsigned default 327695751717730004 , col_14 timestamp default '2010-11-18' not null , primary key idx_5 ( col_11,col_13 ) /*T![clustered_index] clustered */ , unique key idx_6 ( col_10,col_11,col_13 ) , unique key idx_7 ( col_14,col_12,col_13 ) )") + tk.MustExec("insert into tbl_2 values ( 'RmF',-5353757041350034197,'1996-01-22',1866803697729291364,'1996-09-11' )") + tk.MustExec("insert into tbl_2 values ( 'xEOGaB',-6602924241498980347,'2019-02-22',8297270320597030697,'1972-04-04' )") + tk.MustExec("insert into tbl_2 values ( 'dvUztqgTPAhLdzgEsV',3316448219481769821,'2034-09-12',937089564901142512,'2030-12-04' )") + tk.MustExec("insert into tbl_2 values ( 'mNoyfbT',-6027094365061219400,'2035-10-10',1752804734961508175,'1992-08-09' )") + tk.MustExec("insert into tbl_2 values ( 'BDPJMhLYXuKB',6823702503458376955,'2015-04-09',737914379167848827,'2026-04-29' )") + tk.MustExec("insert into tbl_2 values ( 'WPiaVfPstGohvHd',1308183537252932688,'2020-05-03',5364104746649397703,'1979-01-28' )") + tk.MustExec("insert into tbl_2 values ( 'lrm',4642935044097656317,'1973-04-29',149081313305673035,'2013-02-03' )") + tk.MustExec("insert into tbl_2 values ( '',-7361040853169906422,'2024-10-22',6308270832310351889,'1981-02-01' )") + tk.MustExec("insert into tbl_2 values ( 'uDANahGcLwpSssabD',2235074865448210231,'1992-10-10',7140606140672586593,'1992-11-25' )") + tk.MustExec("insert into tbl_2 values ( 'TDH',-1911014243756021618,'2013-01-26',2022218243939205750,'1982-04-04' )") + tk.MustQuery("select lead(col_13,1,NULL) over w from tbl_2 window w as (order by col_13)").Check(testkit.Rows( + "737914379167848827", + "937089564901142512", + "1752804734961508175", + "1866803697729291364", + "2022218243939205750", + "5364104746649397703", + "6308270832310351889", + "7140606140672586593", + "8297270320597030697", + "")) +} diff --git a/expression/aggregation/base_func.go b/expression/aggregation/base_func.go index 6abd5e8cfc0aa..9a5ef95d49bb2 100644 --- a/expression/aggregation/base_func.go +++ b/expression/aggregation/base_func.go @@ -411,6 +411,9 @@ func (a *baseFuncDesc) WrapCastForAggArgs(ctx sessionctx.Context) { if i == 1 && (a.Name == ast.WindowFuncLead || a.Name == ast.WindowFuncLag || a.Name == ast.WindowFuncNthValue) { continue } + if a.Args[i].GetType().Tp == mysql.TypeNull { + continue + } a.Args[i] = castFunc(ctx, a.Args[i]) if a.Name != ast.AggFuncAvg && a.Name != ast.AggFuncSum { continue From c347c6ced85192b04d63675c3495ea1605ed461e Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 28 Apr 2021 17:43:57 +0800 Subject: [PATCH 08/36] executor: Use TableReader directly when accessing TiFlash partition table instead of using PartitionTableExec (#24344) --- executor/builder.go | 25 --------------------- executor/partition_table.go | 44 ------------------------------------- 2 files changed, 69 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 51971b5f8e0a9..45e5447f252c1 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2686,31 +2686,6 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E } if v.StoreType == kv.TiFlash { sctx.IsTiFlash.Store(true) - partsExecutor := make([]Executor, 0, len(partitions)) - for _, part := range partitions { - exec, err := buildNoRangeTableReader(b, v) - if err != nil { - b.err = err - return nil - } - exec.ranges = ts.Ranges - nexec, err := nextPartitionForTableReader{exec: exec}.nextPartition(context.Background(), part) - if err != nil { - b.err = err - return nil - } - partsExecutor = append(partsExecutor, nexec) - } - if len(partsExecutor) == 0 { - return &TableDualExec{baseExecutor: *ret.base()} - } - if len(partsExecutor) == 1 { - return partsExecutor[0] - } - return &UnionExec{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), partsExecutor...), - concurrency: b.ctx.GetSessionVars().UnionConcurrency(), - } } if len(partitions) == 0 { diff --git a/executor/partition_table.go b/executor/partition_table.go index 41dbe64ba7972..3cebdec6c14e6 100644 --- a/executor/partition_table.go +++ b/executor/partition_table.go @@ -40,33 +40,6 @@ type nextPartition interface { nextPartition(context.Context, table.PhysicalTable) (Executor, error) } -// nolint:structcheck -type innerPartitionInfo struct { - isFullPartition bool -} - -type nextPartitionForTableReader struct { - *innerPartitionInfo - rangeBuilders map[int64]kvRangeBuilder - exec *TableReaderExecutor -} - -func (n nextPartitionForTableReader) GetInnerPartitionInfo() *innerPartitionInfo { - return n.innerPartitionInfo -} - -func (n nextPartitionForTableReader) nextPartition(ctx context.Context, tbl table.PhysicalTable) (Executor, error) { - n.exec.table = tbl - n.exec.kvRanges = n.exec.kvRanges[:0] - if n.innerPartitionInfo != nil && !n.isFullPartition { - n.exec.kvRangeBuilder = n.rangeBuilders[tbl.GetPhysicalID()] - } - if err := updateDAGRequestTableID(ctx, n.exec.dagPB, tbl.GetPhysicalID()); err != nil { - return nil, err - } - return n.exec, nil -} - type nextPartitionForUnionScan struct { b *executorBuilder us *plannercore.PhysicalUnionScan @@ -96,23 +69,6 @@ func nextPartitionWithTrace(ctx context.Context, n nextPartition, tbl table.Phys return n.nextPartition(ctx, tbl) } -// updateDAGRequestTableID update the table ID in the DAG request to partition ID. -// TiKV only use that table ID for log, but TiFlash use it. -func updateDAGRequestTableID(ctx context.Context, dag *tipb.DAGRequest, partitionID int64) error { - // TiFlash set RootExecutor field and ignore Executors field. - if dag.RootExecutor != nil { - return updateExecutorTableID(ctx, dag.RootExecutor, partitionID, true) - } - for i := 0; i < len(dag.Executors); i++ { - exec := dag.Executors[i] - err := updateExecutorTableID(ctx, exec, partitionID, false) - if err != nil { - return err - } - } - return nil -} - func updateExecutorTableID(ctx context.Context, exec *tipb.Executor, partitionID int64, recursive bool) error { var child *tipb.Executor switch exec.Tp { From 83ebd6b8d04a7976c21c8460192769ac511b8f5e Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 28 Apr 2021 19:15:57 +0800 Subject: [PATCH 09/36] store/tikv: remove kvfilter option (#24303) --- store/driver/txn/txn_driver.go | 2 +- store/tikv/2pc.go | 2 +- store/tikv/kv/option.go | 2 -- store/tikv/tests/2pc_test.go | 2 +- store/tikv/txn.go | 16 +++++++--------- 5 files changed, 10 insertions(+), 14 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index a9e09a271ce5e..8992866877d4d 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -35,7 +35,7 @@ type tikvTxn struct { // NewTiKVTxn returns a new Transaction. func NewTiKVTxn(txn *tikv.KVTxn) kv.Transaction { - txn.SetOption(tikvstore.KVFilter, TiDBKVFilter{}) + txn.SetKVFilter(TiDBKVFilter{}) entryLimit := atomic.LoadUint64(&kv.TxnEntrySizeLimit) totalLimit := atomic.LoadUint64(&kv.TxnTotalSizeLimit) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index c33bf6d0f6d1f..dafd0847af797 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -327,7 +327,7 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error { sizeHint := txn.us.GetMemBuffer().Len() c.mutations = newMemBufferMutations(sizeHint, memBuf) c.isPessimistic = txn.IsPessimistic() - filter := txn.getKVFilter() + filter := txn.kvFilter var err error for it := memBuf.IterWithFlags(nil, nil); it.Valid(); err = it.Next() { diff --git a/store/tikv/kv/option.go b/store/tikv/kv/option.go index e9479cf609502..a87ddcb7214a4 100644 --- a/store/tikv/kv/option.go +++ b/store/tikv/kv/option.go @@ -59,8 +59,6 @@ const ( IsStalenessReadOnly // MatchStoreLabels indicates the labels the store should be matched MatchStoreLabels - // KVFilter filters out the key-value pairs in the memBuf that is unnecessary to be committed - KVFilter ) // Priority value for transaction priority. diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 9313e6f0abeb8..60af78db111ff 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -1032,7 +1032,7 @@ func (s *testCommitterSuite) TestResolvePessimisticLock(c *C) { untouchedIndexValue := []byte{0, 0, 0, 0, 0, 0, 0, 1, 49} noValueIndexKey := []byte("t00000001_i000000002") txn := s.begin(c) - txn.SetOption(kv.KVFilter, drivertxn.TiDBKVFilter{}) + txn.SetKVFilter(drivertxn.TiDBKVFilter{}) err := txn.Set(untouchedIndexKey, untouchedIndexValue) c.Assert(err, IsNil) lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now(), LockWaitTime: tikv.LockNoWait} diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 5133a6deb35f6..e78eaba81febe 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -73,8 +73,8 @@ type KVTxn struct { schemaAmender SchemaAmender // commitCallback is called after current transaction gets committed commitCallback func(info string, err error) - - binlog BinlogExecutor + binlog BinlogExecutor + kvFilter KVFilter } func newTiKVTxn(store *KVStore, txnScope string) (*KVTxn, error) { @@ -198,18 +198,16 @@ func (txn *KVTxn) DelOption(opt int) { txn.us.DelOption(opt) } +// SetKVFilter sets the filter to ignore key-values in memory buffer. +func (txn *KVTxn) SetKVFilter(filter KVFilter) { + txn.kvFilter = filter +} + // IsPessimistic returns true if it is pessimistic. func (txn *KVTxn) IsPessimistic() bool { return txn.us.GetOption(kv.Pessimistic) != nil } -func (txn *KVTxn) getKVFilter() KVFilter { - if filter := txn.us.GetOption(kv.KVFilter); filter != nil { - return filter.(KVFilter) - } - return nil -} - // Commit commits the transaction operations to KV store. func (txn *KVTxn) Commit(ctx context.Context) error { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { From 7df2289f3b147552da0fa13f4bb5b809c3998d8e Mon Sep 17 00:00:00 2001 From: Shirly Date: Wed, 28 Apr 2021 19:31:56 +0800 Subject: [PATCH 10/36] store/tikv: remove err kv.ErrNotImplemented from tikv package (#24336) --- store/driver/tikv_driver.go | 5 +++++ store/mockstore/mockstorage/storage.go | 6 ++++++ store/tikv/interface.go | 3 --- store/tikv/kv.go | 6 ------ 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index 133595af3d51f..62d54052a3f8a 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -340,3 +340,8 @@ func (s *tikvStore) CurrentVersion(txnScope string) (kv.Version, error) { ver, err := s.KVStore.CurrentTimestamp(txnScope) return kv.NewVersion(ver), err } + +// ShowStatus returns the specified status of the storage +func (s *tikvStore) ShowStatus(ctx context.Context, key string) (interface{}, error) { + return nil, kv.ErrNotImplemented +} diff --git a/store/mockstore/mockstorage/storage.go b/store/mockstore/mockstorage/storage.go index 6a47d3b523421..63875acaad222 100644 --- a/store/mockstore/mockstorage/storage.go +++ b/store/mockstore/mockstorage/storage.go @@ -14,6 +14,7 @@ package mockstorage import ( + "context" "crypto/tls" "github.com/pingcap/tidb/kv" @@ -76,6 +77,11 @@ func (s *mockStorage) Begin() (kv.Transaction, error) { return newTiKVTxn(txn, err) } +// ShowStatus returns the specified status of the storage +func (s *mockStorage) ShowStatus(ctx context.Context, key string) (interface{}, error) { + return nil, kv.ErrNotImplemented +} + // BeginWithOption begins a transaction with given option func (s *mockStorage) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { txnScope := option.TxnScope diff --git a/store/tikv/interface.go b/store/tikv/interface.go index 507cce87535f0..edc0bd06c0400 100644 --- a/store/tikv/interface.go +++ b/store/tikv/interface.go @@ -14,7 +14,6 @@ package tikv import ( - "context" "time" "github.com/pingcap/tidb/store/tikv/oracle" @@ -60,6 +59,4 @@ type Storage interface { GetOracle() oracle.Oracle // SupportDeleteRange gets the storage support delete range or not. SupportDeleteRange() (supported bool) - // ShowStatus returns the specified status of the storage - ShowStatus(ctx context.Context, key string) (interface{}, error) } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index e848c54fb8a16..cdbafe080ace1 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" - tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" @@ -349,11 +348,6 @@ func (s *KVStore) GetPDClient() pd.Client { return s.pdClient } -// ShowStatus returns the specified status of the storage -func (s *KVStore) ShowStatus(ctx context.Context, key string) (interface{}, error) { - return nil, tidbkv.ErrNotImplemented -} - // SupportDeleteRange gets the storage support delete range or not. func (s *KVStore) SupportDeleteRange() (supported bool) { return !s.mock From aaa3b748b34a7814ccff9bb7a0fe49fb53dcbf4f Mon Sep 17 00:00:00 2001 From: Shirly Date: Wed, 28 Apr 2021 19:55:56 +0800 Subject: [PATCH 11/36] store/tikv: use tikv.error.ErrInvalidTxn instead of kv.ErrInvalidTxn (#24347) --- store/driver/txn/error.go | 4 ++++ store/tikv/error/error.go | 2 ++ store/tikv/tests/2pc_fail_test.go | 4 ++-- store/tikv/tests/async_commit_fail_test.go | 3 +-- store/tikv/txn.go | 4 ++-- 5 files changed, 11 insertions(+), 6 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 31f1fd15445ab..9f92c9b1cfd46 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -155,6 +155,10 @@ func toTiDBErr(err error) error { if tikverr.IsErrNotFound(err) { return kv.ErrNotExist } + + if errors.ErrorEqual(err, tikverr.ErrInvalidTxn) { + return kv.ErrInvalidTxn + } return errors.Trace(err) } diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index ffa2baf96358a..987d80e591867 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -27,6 +27,8 @@ var ( ErrTiDBShuttingDown = errors.New("tidb server shutting down") // ErrNotExist means the related data not exist. ErrNotExist = errors.New("not exist") + // ErrInvalidTxn is the error when commits or rollbacks in an invalid transaction. + ErrInvalidTxn = errors.New("invalid transaction") ) // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. diff --git a/store/tikv/tests/2pc_fail_test.go b/store/tikv/tests/2pc_fail_test.go index ed8820a302fe0..d8b35e4b28c53 100644 --- a/store/tikv/tests/2pc_fail_test.go +++ b/store/tikv/tests/2pc_fail_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/kv" + tikverr "github.com/pingcap/tidb/store/tikv/error" ) // TestFailCommitPrimaryRpcErrors tests rpc errors are handled properly when @@ -40,7 +40,7 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRpcErrors(c *C) { // We don't need to call "Rollback" after "Commit" fails. err = t1.Rollback() - c.Assert(err, Equals, kv.ErrInvalidTxn) + c.Assert(err, Equals, tikverr.ErrInvalidTxn) } // TestFailCommitPrimaryRegionError tests RegionError is handled properly when diff --git a/store/tikv/tests/async_commit_fail_test.go b/store/tikv/tests/async_commit_fail_test.go index 5e519afb51c99..a791f16c54e86 100644 --- a/store/tikv/tests/async_commit_fail_test.go +++ b/store/tikv/tests/async_commit_fail_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/util" @@ -65,7 +64,7 @@ func (s *testAsyncCommitFailSuite) TestFailAsyncCommitPrewriteRpcErrors(c *C) { // We don't need to call "Rollback" after "Commit" fails. err = t1.Rollback() - c.Assert(err, Equals, kv.ErrInvalidTxn) + c.Assert(err, Equals, tikverr.ErrInvalidTxn) // Create a new transaction to check. The previous transaction should actually commit. t2 := s.beginAsyncCommit(c) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index e78eaba81febe..3ef4ad545e24a 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -218,7 +218,7 @@ func (txn *KVTxn) Commit(ctx context.Context) error { defer trace.StartRegion(ctx, "CommitTxn").End() if !txn.valid { - return tidbkv.ErrInvalidTxn + return tikverr.ErrInvalidTxn } defer txn.close() @@ -314,7 +314,7 @@ func (txn *KVTxn) close() { // Rollback undoes the transaction operations to KV store. func (txn *KVTxn) Rollback() error { if !txn.valid { - return tidbkv.ErrInvalidTxn + return tikverr.ErrInvalidTxn } start := time.Now() // Clean up pessimistic lock. From 3f8e8beb211615a5affe4cf6934e6bf3ea15fc35 Mon Sep 17 00:00:00 2001 From: Shirly Date: Wed, 28 Apr 2021 20:45:57 +0800 Subject: [PATCH 12/36] store/tikv: use tikv.error.ErrEntryTooLarge instead of kv error (#24351) --- store/driver/txn/error.go | 4 ++++ store/tikv/error/error.go | 12 ++++++++++++ store/tikv/unionstore/memdb.go | 5 ++++- 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 9f92c9b1cfd46..24b04d07a07ea 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -156,6 +156,10 @@ func toTiDBErr(err error) error { return kv.ErrNotExist } + if e, ok := err.(*tikverr.ErrEntryTooLarge); ok { + return kv.ErrEntryTooLarge.GenWithStackByArgs(e.Limit, e.Size) + } + if errors.ErrorEqual(err, tikverr.ErrInvalidTxn) { return kv.ErrInvalidTxn } diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 987d80e591867..3446025eade45 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -14,6 +14,8 @@ package error import ( + "fmt" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/pdpb" @@ -134,3 +136,13 @@ type ErrRetryable struct { func (k *ErrRetryable) Error() string { return k.Retryable } + +// ErrEntryTooLarge is the error when a key value entry is too large. +type ErrEntryTooLarge struct { + Limit uint64 + Size uint64 +} + +func (e *ErrEntryTooLarge) Error() string { + return fmt.Sprintf("entry size too large, size: %v,limit: %v.", e.Size, e.Limit) +} diff --git a/store/tikv/unionstore/memdb.go b/store/tikv/unionstore/memdb.go index a008a7cc6e0ab..5eeb2e08b7023 100644 --- a/store/tikv/unionstore/memdb.go +++ b/store/tikv/unionstore/memdb.go @@ -281,7 +281,10 @@ func (db *MemDB) set(key []byte, value []byte, ops ...kv.FlagsOp) error { if value != nil { if size := uint64(len(key) + len(value)); size > db.entrySizeLimit { - return tidbkv.ErrEntryTooLarge.GenWithStackByArgs(db.entrySizeLimit, size) + return &tikverr.ErrEntryTooLarge{ + Limit: db.entrySizeLimit, + Size: size, + } } } From 349221142bfbb80af58044ea1aa60b545c2d4e78 Mon Sep 17 00:00:00 2001 From: Shirly Date: Wed, 28 Apr 2021 21:05:56 +0800 Subject: [PATCH 13/36] store/tikv: use tikv.error.ErrCannotSetNilValue instead of kv error (#24350) --- store/driver/txn/error.go | 4 ++++ store/tikv/error/error.go | 2 ++ store/tikv/unionstore/memdb.go | 4 ++-- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 24b04d07a07ea..65f95c4dada8e 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -156,6 +156,10 @@ func toTiDBErr(err error) error { return kv.ErrNotExist } + if errors.ErrorEqual(err, tikverr.ErrCannotSetNilValue) { + return kv.ErrCannotSetNilValue + } + if e, ok := err.(*tikverr.ErrEntryTooLarge); ok { return kv.ErrEntryTooLarge.GenWithStackByArgs(e.Limit, e.Size) } diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 3446025eade45..f961738629e42 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -29,6 +29,8 @@ var ( ErrTiDBShuttingDown = errors.New("tidb server shutting down") // ErrNotExist means the related data not exist. ErrNotExist = errors.New("not exist") + // ErrCannotSetNilValue is the error when sets an empty value. + ErrCannotSetNilValue = errors.New("can not set nil value") // ErrInvalidTxn is the error when commits or rollbacks in an invalid transaction. ErrInvalidTxn = errors.New("invalid transaction") ) diff --git a/store/tikv/unionstore/memdb.go b/store/tikv/unionstore/memdb.go index 5eeb2e08b7023..fa2f8dad77d5e 100644 --- a/store/tikv/unionstore/memdb.go +++ b/store/tikv/unionstore/memdb.go @@ -217,7 +217,7 @@ func (db *MemDB) UpdateFlags(key []byte, ops ...kv.FlagsOp) { // v must NOT be nil or empty, otherwise it returns ErrCannotSetNilValue. func (db *MemDB) Set(key []byte, value []byte) error { if len(value) == 0 { - return tidbkv.ErrCannotSetNilValue + return tikverr.ErrCannotSetNilValue } return db.set(key, value) } @@ -225,7 +225,7 @@ func (db *MemDB) Set(key []byte, value []byte) error { // SetWithFlags put key-value into the last active staging buffer with the given KeyFlags. func (db *MemDB) SetWithFlags(key []byte, value []byte, ops ...kv.FlagsOp) error { if len(value) == 0 { - return tidbkv.ErrCannotSetNilValue + return tikverr.ErrCannotSetNilValue } return db.set(key, value, ops...) } From 91ca4eafdc4df45ca21b392ffd0f7eaff9caa306 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 28 Apr 2021 21:21:56 +0800 Subject: [PATCH 14/36] store/tikv: remove Pessimistic option (#24332) --- store/driver/txn/txn_driver.go | 2 ++ store/tikv/tests/2pc_test.go | 44 +++++++++++++------------- store/tikv/tests/async_commit_test.go | 2 +- store/tikv/tests/ticlient_slow_test.go | 2 +- store/tikv/txn.go | 13 ++++++-- 5 files changed, 36 insertions(+), 27 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 8992866877d4d..66413e6bdef28 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -132,6 +132,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { case tikvstore.IsolationLevel: level := getTiKVIsolationLevel(val.(kv.IsoLevel)) txn.KVTxn.GetSnapshot().SetIsolationLevel(level) + case tikvstore.Pessimistic: + txn.SetPessimistic(val.(bool)) default: txn.KVTxn.SetOption(opt, val) } diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 60af78db111ff..e70131a235474 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -617,7 +617,7 @@ func (s *testCommitterSuite) TestRejectCommitTS(c *C) { func (s *testCommitterSuite) TestPessimisticPrewriteRequest(c *C) { // This test checks that the isPessimisticLock field is set in the request even when no keys are pessimistic lock. txn := s.begin(c) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) err := txn.Set([]byte("t1"), []byte("v1")) c.Assert(err, IsNil) committer, err := txn.NewCommitter(0) @@ -636,7 +636,7 @@ func (s *testCommitterSuite) TestUnsetPrimaryKey(c *C) { c.Assert(txn.Commit(context.Background()), IsNil) txn = s.begin(c) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) _, _ = txn.GetUnionStore().Get(context.TODO(), key) c.Assert(txn.GetMemBuffer().SetWithFlags(key, key, kv.SetPresumeKeyNotExists), IsNil) lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()} @@ -651,7 +651,7 @@ func (s *testCommitterSuite) TestUnsetPrimaryKey(c *C) { func (s *testCommitterSuite) TestPessimisticLockedKeysDedup(c *C) { txn := s.begin(c) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) lockCtx := &kv.LockCtx{ForUpdateTS: 100, WaitStartTime: time.Now()} err := txn.LockKeys(context.Background(), lockCtx, []byte("abc"), []byte("def")) c.Assert(err, IsNil) @@ -664,7 +664,7 @@ func (s *testCommitterSuite) TestPessimisticLockedKeysDedup(c *C) { func (s *testCommitterSuite) TestPessimisticTTL(c *C) { key := []byte("key") txn := s.begin(c) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) time.Sleep(time.Millisecond * 100) lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()} err := txn.LockKeys(context.Background(), lockCtx, key) @@ -710,7 +710,7 @@ func (s *testCommitterSuite) TestPessimisticLockReturnValues(c *C) { c.Assert(txn.Set(key2, key2), IsNil) c.Assert(txn.Commit(context.Background()), IsNil) txn = s.begin(c) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()} lockCtx.ReturnValues = true lockCtx.Values = map[string]kv.ReturnedValue{} @@ -725,7 +725,7 @@ func (s *testCommitterSuite) TestElapsedTTL(c *C) { key := []byte("key") txn := s.begin(c) txn.SetStartTS(oracle.ComposeTS(oracle.GetPhysical(time.Now().Add(time.Second*10)), 1)) - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) time.Sleep(time.Millisecond * 100) lockCtx := &kv.LockCtx{ ForUpdateTS: oracle.ComposeTS(oracle.ExtractPhysical(txn.StartTS())+100, 1), @@ -746,7 +746,7 @@ func (s *testCommitterSuite) TestDeleteYourWriteCauseGhostPrimary(c *C) { // insert k1, k2, k3 and delete k1 txn1 := s.begin(c) - txn1.DelOption(kv.Pessimistic) + txn1.SetPessimistic(false) s.store.ClearTxnLatches() txn1.Get(context.Background(), k1) txn1.GetMemBuffer().SetWithFlags(k1, []byte{0}, kv.SetPresumeKeyNotExists) @@ -771,7 +771,7 @@ func (s *testCommitterSuite) TestDeleteYourWriteCauseGhostPrimary(c *C) { // start txn2 to read k3(prewrite success and primary should be committed) txn2 := s.begin(c) - txn2.DelOption(kv.Pessimistic) + txn2.SetPessimistic(false) s.store.ClearTxnLatches() v, err := txn2.Get(context.Background(), k3) c.Assert(err, IsNil) // should resolve lock and read txn1 k3 result instead of rollback it. @@ -788,7 +788,7 @@ func (s *testCommitterSuite) TestDeleteAllYourWrites(c *C) { // insert k1, k2, k3 and delete k1, k2, k3 txn1 := s.begin(c) - txn1.DelOption(kv.Pessimistic) + txn1.SetPessimistic(false) s.store.ClearTxnLatches() txn1.GetMemBuffer().SetWithFlags(k1, []byte{0}, kv.SetPresumeKeyNotExists) txn1.Delete(k1) @@ -808,7 +808,7 @@ func (s *testCommitterSuite) TestDeleteAllYourWritesWithSFU(c *C) { // insert k1, k2, k2 and delete k1 txn1 := s.begin(c) - txn1.DelOption(kv.Pessimistic) + txn1.SetPessimistic(false) s.store.ClearTxnLatches() txn1.GetMemBuffer().SetWithFlags(k1, []byte{0}, kv.SetPresumeKeyNotExists) txn1.Delete(k1) @@ -832,7 +832,7 @@ func (s *testCommitterSuite) TestDeleteAllYourWritesWithSFU(c *C) { <-ac // start txn2 to read k3 txn2 := s.begin(c) - txn2.DelOption(kv.Pessimistic) + txn2.SetPessimistic(false) s.store.ClearTxnLatches() err = txn2.Set(k3, []byte{33}) c.Assert(err, IsNil) @@ -860,7 +860,7 @@ func (s *testCommitterSuite) TestAcquireFalseTimeoutLock(c *C) { k2 := []byte("k2") txn1 := s.begin(c) - txn1.SetOption(kv.Pessimistic, true) + txn1.SetPessimistic(true) // lock the primary key lockCtx := &kv.LockCtx{ForUpdateTS: txn1.StartTS(), WaitStartTime: time.Now()} err := txn1.LockKeys(context.Background(), lockCtx, k1) @@ -875,7 +875,7 @@ func (s *testCommitterSuite) TestAcquireFalseTimeoutLock(c *C) { // wait until secondary key exceeds its own TTL time.Sleep(time.Duration(atomic.LoadUint64(&tikv.ManagedLockTTL)) * time.Millisecond) txn2 := s.begin(c) - txn2.SetOption(kv.Pessimistic, true) + txn2.SetPessimistic(true) // test no wait lockCtx = &kv.LockCtx{ForUpdateTS: txn2.StartTS(), LockWaitTime: tikv.LockNoWait, WaitStartTime: time.Now()} @@ -921,7 +921,7 @@ func (s *testCommitterSuite) TestPkNotFound(c *C) { k3 := []byte("k3") txn1 := s.begin(c) - txn1.SetOption(kv.Pessimistic, true) + txn1.SetPessimistic(true) // lock the primary key. lockCtx := &kv.LockCtx{ForUpdateTS: txn1.StartTS(), WaitStartTime: time.Now()} err := txn1.LockKeys(ctx, lockCtx, k1) @@ -957,7 +957,7 @@ func (s *testCommitterSuite) TestPkNotFound(c *C) { c.Assert(err, IsNil) c.Assert(status.Action(), Equals, kvrpcpb.Action_LockNotExistDoNothing) txn2 := s.begin(c) - txn2.SetOption(kv.Pessimistic, true) + txn2.SetPessimistic(true) lockCtx = &kv.LockCtx{ForUpdateTS: txn2.StartTS(), WaitStartTime: time.Now()} err = txn2.LockKeys(ctx, lockCtx, k2) c.Assert(err, IsNil) @@ -981,7 +981,7 @@ func (s *testCommitterSuite) TestPkNotFound(c *C) { // After disable fail point, the rollbackIfNotExist flag will be set, and the resolve should succeed. In this // case, the returned action of TxnStatus should be LockNotExistDoNothing, and lock on k3 could be resolved. txn3 := s.begin(c) - txn3.SetOption(kv.Pessimistic, true) + txn3.SetPessimistic(true) lockCtx = &kv.LockCtx{ForUpdateTS: txn3.StartTS(), WaitStartTime: time.Now(), LockWaitTime: tikv.LockNoWait} err = txn3.LockKeys(ctx, lockCtx, k3) c.Assert(err, IsNil) @@ -997,7 +997,7 @@ func (s *testCommitterSuite) TestPessimisticLockPrimary(c *C) { k2 := []byte("b") txn1 := s.begin(c) - txn1.SetOption(kv.Pessimistic, true) + txn1.SetPessimistic(true) // txn1 lock k1 lockCtx := &kv.LockCtx{ForUpdateTS: txn1.StartTS(), WaitStartTime: time.Now()} err := txn1.LockKeys(context.Background(), lockCtx, k1) @@ -1008,7 +1008,7 @@ func (s *testCommitterSuite) TestPessimisticLockPrimary(c *C) { doneCh := make(chan error) go func() { txn2 := s.begin(c) - txn2.SetOption(kv.Pessimistic, true) + txn2.SetPessimistic(true) lockCtx2 := &kv.LockCtx{ForUpdateTS: txn2.StartTS(), WaitStartTime: time.Now(), LockWaitTime: 200} waitErr := txn2.LockKeys(context.Background(), lockCtx2, k1, k2) doneCh <- waitErr @@ -1017,7 +1017,7 @@ func (s *testCommitterSuite) TestPessimisticLockPrimary(c *C) { // txn3 should locks k2 successfully using no wait txn3 := s.begin(c) - txn3.SetOption(kv.Pessimistic, true) + txn3.SetPessimistic(true) lockCtx3 := &kv.LockCtx{ForUpdateTS: txn3.StartTS(), WaitStartTime: time.Now(), LockWaitTime: tikv.LockNoWait} c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/txnNotFoundRetTTL", "return"), IsNil) err = txn3.LockKeys(context.Background(), lockCtx3, k2) @@ -1108,7 +1108,7 @@ func (s *testCommitterSuite) TestPushPessimisticLock(c *C) { ctx := context.Background() txn1 := s.begin(c) - txn1.SetOption(kv.Pessimistic, true) + txn1.SetPessimistic(true) lockCtx := &kv.LockCtx{ForUpdateTS: txn1.StartTS(), WaitStartTime: time.Now()} err := txn1.LockKeys(context.Background(), lockCtx, k1, k2) c.Assert(err, IsNil) @@ -1160,7 +1160,7 @@ func (s *testCommitterSuite) TestResolveMixed(c *C) { // make the optimistic and pessimistic lock left with primary lock not found txn1 := s.begin(c) - txn1.SetOption(kv.Pessimistic, true) + txn1.SetPessimistic(true) // lock the primary key lockCtx := &kv.LockCtx{ForUpdateTS: txn1.StartTS(), WaitStartTime: time.Now()} err := txn1.LockKeys(context.Background(), lockCtx, pk) @@ -1202,7 +1202,7 @@ func (s *testCommitterSuite) TestResolveMixed(c *C) { // txn2 tries to lock the pessimisticLockKey, the lock should has been resolved in clean whole region resolve txn2 := s.begin(c) - txn2.SetOption(kv.Pessimistic, true) + txn2.SetPessimistic(true) lockCtx = &kv.LockCtx{ForUpdateTS: txn2.StartTS(), WaitStartTime: time.Now(), LockWaitTime: tikv.LockNoWait} err = txn2.LockKeys(context.Background(), lockCtx, pessimisticLockKey) c.Assert(err, IsNil) diff --git a/store/tikv/tests/async_commit_test.go b/store/tikv/tests/async_commit_test.go index 96a35085f46b8..99e82add2e93d 100644 --- a/store/tikv/tests/async_commit_test.go +++ b/store/tikv/tests/async_commit_test.go @@ -350,7 +350,7 @@ func (s *testAsyncCommitSuite) TestRepeatableRead(c *C) { sessionID++ ctx := context.WithValue(context.Background(), util.SessionID, sessionID) txn1 := s.beginAsyncCommit(c) - txn1.SetOption(kv.Pessimistic, isPessimistic) + txn1.SetPessimistic(isPessimistic) s.mustGetFromTxn(c, txn1, []byte("k1"), []byte("v1")) txn1.Set([]byte("k1"), []byte("v2")) diff --git a/store/tikv/tests/ticlient_slow_test.go b/store/tikv/tests/ticlient_slow_test.go index b37245e72245c..61f6748874d14 100644 --- a/store/tikv/tests/ticlient_slow_test.go +++ b/store/tikv/tests/ticlient_slow_test.go @@ -64,7 +64,7 @@ func (s *testTiclientSuite) TestSplitRegionIn2PC(c *C) { checkKeyRegion(bo, startKey, endKey, Equals) txn := s.beginTxn(c) if m == "pessimistic" { - txn.SetOption(kv.Pessimistic, true) + txn.SetPessimistic(true) lockCtx := &kv.LockCtx{} lockCtx.ForUpdateTS = txn.StartTS() keys := make([][]byte, 0, preSplitThresholdInTest) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 3ef4ad545e24a..d1eae7c6e2b05 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -73,8 +73,10 @@ type KVTxn struct { schemaAmender SchemaAmender // commitCallback is called after current transaction gets committed commitCallback func(info string, err error) - binlog BinlogExecutor - kvFilter KVFilter + + binlog BinlogExecutor + isPessimistic bool + kvFilter KVFilter } func newTiKVTxn(store *KVStore, txnScope string) (*KVTxn, error) { @@ -198,6 +200,11 @@ func (txn *KVTxn) DelOption(opt int) { txn.us.DelOption(opt) } +// SetPessimistic indicates if the transaction should use pessimictic lock. +func (txn *KVTxn) SetPessimistic(b bool) { + txn.isPessimistic = b +} + // SetKVFilter sets the filter to ignore key-values in memory buffer. func (txn *KVTxn) SetKVFilter(filter KVFilter) { txn.kvFilter = filter @@ -205,7 +212,7 @@ func (txn *KVTxn) SetKVFilter(filter KVFilter) { // IsPessimistic returns true if it is pessimistic. func (txn *KVTxn) IsPessimistic() bool { - return txn.us.GetOption(kv.Pessimistic) != nil + return txn.isPessimistic } // Commit commits the transaction operations to KV store. From 75be70cd4cd33eacb95df5c567158ec60dcd4bd2 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 29 Apr 2021 02:39:56 +0800 Subject: [PATCH 15/36] store/tikv: remove use of SchemaLease transaction option in store/tikv (#24331) --- store/driver/txn/txn_driver.go | 2 ++ store/tikv/2pc.go | 8 ++++---- store/tikv/txn.go | 12 +++++++++--- 3 files changed, 15 insertions(+), 7 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 66413e6bdef28..9a5731c14973e 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -129,6 +129,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn: txn.KVTxn, binInfo: val.(*binloginfo.BinlogInfo), // val cannot be other type. }) + case tikvstore.SchemaChecker: + txn.SetSchemaLeaseChecker(val.(tikv.SchemaLeaseChecker)) case tikvstore.IsolationLevel: level := getTiKVIsolationLevel(val.(kv.IsoLevel)) txn.KVTxn.GetSnapshot().SetIsolationLevel(level) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index dafd0847af797..b01e18eebd5d1 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -1252,7 +1252,8 @@ type SchemaVer interface { SchemaMetaVersion() int64 } -type schemaLeaseChecker interface { +// SchemaLeaseChecker is used to validate schema version is not changed during transaction execution. +type SchemaLeaseChecker interface { // CheckBySchemaVer checks if the schema has changed for the transaction related tables between the startSchemaVer // and the schema version at txnTS, all the related schema changes will be returned. CheckBySchemaVer(txnTS uint64, startSchemaVer SchemaVer) (*RelatedSchemaChange, error) @@ -1398,8 +1399,7 @@ func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64 err := errors.Errorf("mock check schema valid failure") failpoint.Return(nil, false, err) }) - checker, ok := c.txn.us.GetOption(kv.SchemaChecker).(schemaLeaseChecker) - if !ok { + if c.txn.schemaLeaseChecker == nil { if c.sessionID > 0 { logutil.Logger(ctx).Warn("schemaLeaseChecker is not set for this transaction", zap.Uint64("sessionID", c.sessionID), @@ -1408,7 +1408,7 @@ func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64 } return nil, false, nil } - relatedChanges, err := checker.CheckBySchemaVer(checkTS, startInfoSchema) + relatedChanges, err := c.txn.schemaLeaseChecker.CheckBySchemaVer(checkTS, startInfoSchema) if err != nil { if tryAmend && relatedChanges != nil && relatedChanges.Amendable && c.txn.schemaAmender != nil { memAmended, amendErr := c.tryAmendTxn(ctx, startInfoSchema, relatedChanges) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index d1eae7c6e2b05..67477c510cee4 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -74,9 +74,10 @@ type KVTxn struct { // commitCallback is called after current transaction gets committed commitCallback func(info string, err error) - binlog BinlogExecutor - isPessimistic bool - kvFilter KVFilter + binlog BinlogExecutor + schemaLeaseChecker SchemaLeaseChecker + isPessimistic bool + kvFilter KVFilter } func newTiKVTxn(store *KVStore, txnScope string) (*KVTxn, error) { @@ -200,6 +201,11 @@ 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 +} + // SetPessimistic indicates if the transaction should use pessimictic lock. func (txn *KVTxn) SetPessimistic(b bool) { txn.isPessimistic = b From 4a8e671f89979706d30d54e2f26de26447766aed Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 29 Apr 2021 11:13:57 +0800 Subject: [PATCH 16/36] store/tikv: remove use of KeyOnly transaction option in store/tikv (#24334) --- store/tikv/snapshot.go | 7 +++++-- store/tikv/tests/lock_test.go | 3 +-- store/tikv/tests/scan_test.go | 4 ++-- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 571a6b0d3da5a..1e54955f5fe4e 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -557,8 +557,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.notFillCache = val.(bool) case kv.SyncLog: s.syncLog = val.(bool) - case kv.KeyOnly: - s.keyOnly = val.(bool) case kv.SnapshotTS: s.setSnapshotTS(val.(uint64)) case kv.ReplicaRead: @@ -602,6 +600,11 @@ func (s *KVSnapshot) DelOption(opt int) { } } +// SetKeyOnly indicates if tikv can return only keys. +func (s *KVSnapshot) SetKeyOnly(b bool) { + s.keyOnly = b +} + // SetIsolationLevel sets the isolation level used to scan data from tikv. func (s *KVSnapshot) SetIsolationLevel(level IsoLevel) { s.isolationLevel = level diff --git a/store/tikv/tests/lock_test.go b/store/tikv/tests/lock_test.go index d6e20157bc7ac..3c6c652d96041 100644 --- a/store/tikv/tests/lock_test.go +++ b/store/tikv/tests/lock_test.go @@ -26,7 +26,6 @@ import ( "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" ) @@ -142,7 +141,7 @@ func (s *testLockSuite) TestScanLockResolveWithSeekKeyOnly(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.KeyOnly, true) + txn.GetSnapshot().SetKeyOnly(true) iter, err := txn.Iter([]byte("a"), nil) c.Assert(err, IsNil) for ch := byte('a'); ch <= byte('z'); ch++ { diff --git a/store/tikv/tests/scan_test.go b/store/tikv/tests/scan_test.go index 91e15747d3894..c64c6c3c52500 100644 --- a/store/tikv/tests/scan_test.go +++ b/store/tikv/tests/scan_test.go @@ -146,7 +146,7 @@ func (s *testScanSuite) TestScan(c *C) { check(c, scan, upperBound, false) txn3 := s.beginTxn(c) - txn3.SetOption(kv.KeyOnly, true) + txn3.GetSnapshot().SetKeyOnly(true) // Test scan without upper bound scan, err = txn3.Iter(s.recordPrefix, nil) c.Assert(err, IsNil) @@ -157,7 +157,7 @@ func (s *testScanSuite) TestScan(c *C) { check(c, scan, upperBound, true) // Restore KeyOnly to false - txn3.SetOption(kv.KeyOnly, false) + txn3.GetSnapshot().SetKeyOnly(false) scan, err = txn3.Iter(s.recordPrefix, nil) c.Assert(err, IsNil) check(c, scan, rowNum, true) From 86b6429db96e5380a49f705473866c76ff45dad3 Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 29 Apr 2021 11:39:57 +0800 Subject: [PATCH 17/36] store/tikv: use tikv.error.ErrTxnTooLarge instead of kv.ErrTxnTooLarge (#24353) --- store/driver/txn/error.go | 4 ++++ store/tikv/error/error.go | 9 +++++++++ store/tikv/unionstore/memdb.go | 3 +-- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 65f95c4dada8e..e2e12cfde62e6 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -156,6 +156,10 @@ func toTiDBErr(err error) error { return kv.ErrNotExist } + if e, ok := err.(*tikverr.ErrTxnTooLarge); ok { + return kv.ErrTxnTooLarge.GenWithStackByArgs(e.Size) + } + if errors.ErrorEqual(err, tikverr.ErrCannotSetNilValue) { return kv.ErrCannotSetNilValue } diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index f961738629e42..4c6015ba4eba5 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -139,6 +139,15 @@ func (k *ErrRetryable) Error() string { return k.Retryable } +// ErrTxnTooLarge is the error when transaction is too large, lock time reached the maximum value. +type ErrTxnTooLarge struct { + Size int +} + +func (e *ErrTxnTooLarge) Error() string { + return fmt.Sprintf("txn too large, size: %v.", e.Size) +} + // ErrEntryTooLarge is the error when a key value entry is too large. type ErrEntryTooLarge struct { Limit uint64 diff --git a/store/tikv/unionstore/memdb.go b/store/tikv/unionstore/memdb.go index fa2f8dad77d5e..7bfc11971e4e2 100644 --- a/store/tikv/unionstore/memdb.go +++ b/store/tikv/unionstore/memdb.go @@ -20,7 +20,6 @@ import ( "sync" "unsafe" - tidbkv "github.com/pingcap/tidb/kv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" ) @@ -310,7 +309,7 @@ func (db *MemDB) set(key []byte, value []byte, ops ...kv.FlagsOp) error { db.setValue(x, value) if uint64(db.Size()) > db.bufferSizeLimit { - return tidbkv.ErrTxnTooLarge.GenWithStackByArgs(db.Size()) + return &tikverr.ErrTxnTooLarge{Size: db.Size()} } return nil } From 267648fb1f7fc4216ba408c6718194cf03e2fc95 Mon Sep 17 00:00:00 2001 From: Eric Shen Date: Thu, 29 Apr 2021 12:35:57 +0800 Subject: [PATCH 18/36] changelog: fix broken link (#24358) --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index bf4f96510c353..df77268b59263 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,5 @@ # TiDB Changelog -All notable changes to this project will be documented in this file. See also [Release Notes](https://github.com/pingcap/docs/blob/master/releases/rn.md), [TiKV Changelog](https://github.com/tikv/tikv/blob/master/CHANGELOG.md) and [PD Changelog](https://github.com/tikv/pd/blob/master/CHANGELOG.md). +All notable changes to this project will be documented in this file. See also [Release Notes](https://github.com/pingcap/docs/blob/master/releases/release-notes.md), [TiKV Changelog](https://github.com/tikv/tikv/blob/master/CHANGELOG.md) and [PD Changelog](https://github.com/tikv/pd/blob/master/CHANGELOG.md). ## [3.0.4] 2019-10-08 ## New features From 0ec8f2d9fbfb3d078e270fff953aeb9b92fe6b42 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 29 Apr 2021 13:59:57 +0800 Subject: [PATCH 19/36] store/tikv: remove use of Priority transaction options in store/tikv (#24360) --- ddl/backfilling.go | 2 +- ddl/reorg.go | 3 +-- distsql/request_builder.go | 11 +++++------ executor/adapter.go | 10 +++++----- executor/analyze.go | 4 ++-- kv/kv.go | 7 +++++++ kv/mock_test.go | 2 +- meta/meta.go | 10 +++++----- session/session.go | 2 +- sessionctx/variable/session.go | 10 +++++----- store/copr/batch_coprocessor.go | 2 +- store/copr/coprocessor.go | 14 +++++++++++++- store/driver/txn/snapshot.go | 13 +++++++++++++ store/driver/txn/txn_driver.go | 2 ++ store/tikv/2pc.go | 21 +-------------------- store/tikv/kv/option.go | 1 + store/tikv/scan.go | 4 ++-- store/tikv/snapshot.go | 30 ++++++++++++++++++++++++------ store/tikv/tests/store_test.go | 9 ++++----- store/tikv/txn.go | 7 +++++++ 20 files changed, 101 insertions(+), 63 deletions(-) diff --git a/ddl/backfilling.go b/ddl/backfilling.go index cd8935572f45c..56512eec6ab65 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -159,7 +159,7 @@ func newBackfillWorker(sessCtx sessionctx.Context, worker *worker, id int, t tab sessCtx: sessCtx, taskCh: make(chan *reorgBackfillTask, 1), resultCh: make(chan *backfillResult, 1), - priority: tikvstore.PriorityLow, + priority: kv.PriorityLow, } } diff --git a/ddl/reorg.go b/ddl/reorg.go index dd24b2934dc4b..fbe42573dbbf7 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -425,7 +424,7 @@ func (dc *ddlCtx) buildDescTableScan(ctx context.Context, startTS uint64, tbl ta SetConcurrency(1).SetDesc(true) builder.Request.NotFillCache = true - builder.Request.Priority = tikvstore.PriorityLow + builder.Request.Priority = kv.PriorityLow kvReq, err := builder.Build() if err != nil { diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 9dfd4124304b4..ce577b993d009 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -146,7 +145,7 @@ func (builder *RequestBuilder) SetAnalyzeRequest(ana *tipb.AnalyzeReq) *RequestB builder.Request.Data, builder.err = ana.Marshal() builder.Request.NotFillCache = true builder.Request.IsolationLevel = kv.RC - builder.Request.Priority = tikvstore.PriorityLow + builder.Request.Priority = kv.PriorityLow } return builder @@ -210,13 +209,13 @@ func (builder *RequestBuilder) getIsolationLevel() kv.IsoLevel { func (builder *RequestBuilder) getKVPriority(sv *variable.SessionVars) int { switch sv.StmtCtx.Priority { case mysql.NoPriority, mysql.DelayedPriority: - return tikvstore.PriorityNormal + return kv.PriorityNormal case mysql.LowPriority: - return tikvstore.PriorityLow + return kv.PriorityLow case mysql.HighPriority: - return tikvstore.PriorityHigh + return kv.PriorityHigh } - return tikvstore.PriorityNormal + return kv.PriorityNormal } // SetFromSessionVars sets the following fields for "kv.Request" from session variables: diff --git a/executor/adapter.go b/executor/adapter.go index 5a65a31e49c7d..5e5b7990f61d9 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -213,7 +213,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec if err != nil { return nil, err } - a.Ctx.GetSessionVars().StmtCtx.Priority = tikvstore.PriorityHigh + a.Ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityHigh // try to reuse point get executor if a.PsStmt.Executor != nil { @@ -730,15 +730,15 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { if stmtPri := stmtCtx.Priority; stmtPri == mysql.NoPriority { switch { case useMaxTS: - stmtCtx.Priority = tikvstore.PriorityHigh + stmtCtx.Priority = kv.PriorityHigh case a.LowerPriority: - stmtCtx.Priority = tikvstore.PriorityLow + stmtCtx.Priority = kv.PriorityLow } } } } if _, ok := a.Plan.(*plannercore.Analyze); ok && ctx.GetSessionVars().InRestrictedSQL { - ctx.GetSessionVars().StmtCtx.Priority = tikvstore.PriorityLow + ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityLow } b := newExecutorBuilder(ctx, a.InfoSchema) @@ -758,7 +758,7 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { a.isPreparedStmt = true a.Plan = executorExec.plan if executorExec.lowerPriority { - ctx.GetSessionVars().StmtCtx.Priority = tikvstore.PriorityLow + ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityLow } e = executorExec.stmtExec } diff --git a/executor/analyze.go b/executor/analyze.go index f595931d0c036..8a2e665587aa4 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -964,7 +964,7 @@ func (e *AnalyzeFastExec) activateTxnForRowCount() (rollbackFn func() error, err return nil, errors.Trace(err) } } - txn.SetOption(tikvstore.Priority, tikvstore.PriorityLow) + txn.SetOption(tikvstore.Priority, kv.PriorityLow) txn.SetOption(tikvstore.IsolationLevel, kv.RC) txn.SetOption(tikvstore.NotFillCache, true) return rollbackFn, nil @@ -1186,7 +1186,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { snapshot := e.ctx.GetStore().GetSnapshot(kv.MaxVersion) snapshot.SetOption(tikvstore.NotFillCache, true) snapshot.SetOption(tikvstore.IsolationLevel, kv.RC) - snapshot.SetOption(tikvstore.Priority, tikvstore.PriorityLow) + snapshot.SetOption(tikvstore.Priority, kv.PriorityLow) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) } diff --git a/kv/kv.go b/kv/kv.go index 4f020bbc9b3f9..1b1e1a5f46a4a 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -439,6 +439,13 @@ type SplittableStore interface { CheckRegionInScattering(regionID uint64) (bool, error) } +// Priority value for transaction priority. +const ( + PriorityNormal = iota + PriorityLow + PriorityHigh +) + // IsoLevel is the transaction's isolation level. type IsoLevel int diff --git a/kv/mock_test.go b/kv/mock_test.go index 6b8cec5ae9f66..45e45d5941251 100644 --- a/kv/mock_test.go +++ b/kv/mock_test.go @@ -35,7 +35,7 @@ func (s testMockSuite) TestInterface(c *C) { snapshot := storage.GetSnapshot(version) _, err = snapshot.BatchGet(context.Background(), []Key{Key("abc"), Key("def")}) c.Check(err, IsNil) - snapshot.SetOption(tikvstore.Priority, tikvstore.PriorityNormal) + snapshot.SetOption(tikvstore.Priority, PriorityNormal) transaction, err := storage.Begin() c.Check(err, IsNil) diff --git a/meta/meta.go b/meta/meta.go index 06bafa9dcd546..e5a0c97658b43 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -94,7 +94,7 @@ type Meta struct { // NewMeta creates a Meta in transaction txn. // If the current Meta needs to handle a job, jobListKey is the type of the job's list. func NewMeta(txn kv.Transaction, jobListKeys ...JobListKeyType) *Meta { - txn.SetOption(tikvstore.Priority, tikvstore.PriorityHigh) + txn.SetOption(tikvstore.Priority, kv.PriorityHigh) txn.SetOption(tikvstore.SyncLog, true) t := structure.NewStructure(txn, txn, mMetaPrefix) listKey := DefaultJobListKey @@ -636,13 +636,13 @@ func (m *Meta) getDDLJob(key []byte, index int64) (*model.Job, error) { job := &model.Job{ // For compatibility, if the job is enqueued by old version TiDB and Priority field is omitted, - // set the default priority to tikvstore.PriorityLow. - Priority: tikvstore.PriorityLow, + // set the default priority to kv.PriorityLow. + Priority: kv.PriorityLow, } err = job.Decode(value) // Check if the job.Priority is valid. - if job.Priority < tikvstore.PriorityNormal || job.Priority > tikvstore.PriorityHigh { - job.Priority = tikvstore.PriorityLow + if job.Priority < kv.PriorityNormal || job.Priority > kv.PriorityHigh { + job.Priority = kv.PriorityLow } return job, errors.Trace(err) } diff --git a/session/session.go b/session/session.go index 2e78d6167d73c..52d59514d0223 100644 --- a/session/session.go +++ b/session/session.go @@ -1714,7 +1714,7 @@ func (s *session) cachedPlanExec(ctx context.Context, s.txn.changeToInvalid() case *plannercore.Update: s.PrepareTSFuture(ctx) - stmtCtx.Priority = tikvstore.PriorityHigh + stmtCtx.Priority = kv.PriorityHigh resultSet, err = runStmt(ctx, s, stmt) case nil: // cache is invalid diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 6e75bd7ca010c..536f446b16d68 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -962,7 +962,7 @@ func NewSessionVars() *SessionVars { OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, RetryLimit: DefTiDBRetryLimit, DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, - DDLReorgPriority: tikvstore.PriorityLow, + DDLReorgPriority: kv.PriorityLow, allowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, preferRangeScan: DefOptPreferRangeScan, CorrelationThreshold: DefOptCorrelationThreshold, @@ -1308,13 +1308,13 @@ func (s *SessionVars) setDDLReorgPriority(val string) { val = strings.ToLower(val) switch val { case "priority_low": - s.DDLReorgPriority = tikvstore.PriorityLow + s.DDLReorgPriority = kv.PriorityLow case "priority_normal": - s.DDLReorgPriority = tikvstore.PriorityNormal + s.DDLReorgPriority = kv.PriorityNormal case "priority_high": - s.DDLReorgPriority = tikvstore.PriorityHigh + s.DDLReorgPriority = kv.PriorityHigh default: - s.DDLReorgPriority = tikvstore.PriorityLow + s.DDLReorgPriority = kv.PriorityLow } } diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index af6fb47676d1a..cec3e49644363 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -346,7 +346,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *tikv.Backoffe req := tikvrpc.NewRequest(task.cmdType, &copReq, kvrpcpb.Context{ IsolationLevel: isolationLevelToPB(b.req.IsolationLevel), - Priority: tikv.PriorityToPB(b.req.Priority), + Priority: priorityToPB(b.req.Priority), NotFillCache: b.req.NotFillCache, RecordTimeStat: true, RecordScanStat: true, diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index dbb9dc807d5e8..19a93c217d04c 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -698,7 +698,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *tikv.Backoffer, task *copTas req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, worker.req.ReplicaRead, &worker.replicaReadSeed, kvrpcpb.Context{ IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel), - Priority: tikv.PriorityToPB(worker.req.Priority), + Priority: priorityToPB(worker.req.Priority), NotFillCache: worker.req.NotFillCache, RecordTimeStat: true, RecordScanStat: true, @@ -1192,6 +1192,18 @@ func (e *rateLimitAction) isEnabled() bool { return atomic.LoadUint32(&e.enabled) > 0 } +// priorityToPB converts priority type to wire type. +func priorityToPB(pri int) kvrpcpb.CommandPri { + switch pri { + case kv.PriorityLow: + return kvrpcpb.CommandPri_Low + case kv.PriorityHigh: + return kvrpcpb.CommandPri_High + default: + return kvrpcpb.CommandPri_Normal + } +} + func isolationLevelToPB(level kv.IsoLevel) kvrpcpb.IsolationLevel { switch level { case kv.RC: diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 5fa8f67f90905..28ecbd567e786 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -67,6 +67,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { case tikvstore.IsolationLevel: level := getTiKVIsolationLevel(val.(kv.IsoLevel)) s.KVSnapshot.SetIsolationLevel(level) + case tikvstore.Priority: + s.KVSnapshot.SetPriority(getTiKVPriority(val.(int))) default: s.KVSnapshot.SetOption(opt, val) } @@ -87,3 +89,14 @@ func getTiKVIsolationLevel(level kv.IsoLevel) tikv.IsoLevel { return tikv.SI } } + +func getTiKVPriority(pri int) tikv.Priority { + switch pri { + case kv.PriorityHigh: + return tikv.PriorityHigh + case kv.PriorityLow: + return tikv.PriorityLow + default: + return tikv.PriorityNormal + } +} diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 9a5731c14973e..ddec266419fd9 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -134,6 +134,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { case tikvstore.IsolationLevel: level := getTiKVIsolationLevel(val.(kv.IsoLevel)) txn.KVTxn.GetSnapshot().SetIsolationLevel(level) + case tikvstore.Priority: + txn.KVTxn.SetPriority(getTiKVPriority(val.(int))) case tikvstore.Pessimistic: txn.SetPessimistic(val.(bool)) default: diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index b01e18eebd5d1..e2172d17bd806 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -425,7 +425,7 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error { metrics.TiKVTxnWriteSizeHistogram.Observe(float64(commitDetail.WriteSize)) c.hasNoNeedCommitKeys = checkCnt > 0 c.lockTTL = txnLockTTL(txn.startTime, size) - c.priority = getTxnPriority(txn) + c.priority = txn.priority.ToPB() c.syncLog = getTxnSyncLog(txn) c.setDetail(commitDetail) return nil @@ -1649,13 +1649,6 @@ func (batchExe *batchExecutor) process(batches []batchMutations) error { return err } -func getTxnPriority(txn *KVTxn) pb.CommandPri { - if pri := txn.us.GetOption(kv.Priority); pri != nil { - return PriorityToPB(pri.(int)) - } - return pb.CommandPri_Normal -} - func getTxnSyncLog(txn *KVTxn) bool { if syncOption := txn.us.GetOption(kv.SyncLog); syncOption != nil { return syncOption.(bool) @@ -1663,18 +1656,6 @@ func getTxnSyncLog(txn *KVTxn) bool { return false } -// PriorityToPB converts priority type to wire type. -func PriorityToPB(pri int) pb.CommandPri { - switch pri { - case kv.PriorityLow: - return pb.CommandPri_Low - case kv.PriorityHigh: - return pb.CommandPri_High - default: - return pb.CommandPri_Normal - } -} - func (c *twoPhaseCommitter) setDetail(d *util.CommitDetails) { atomic.StorePointer(&c.detail, unsafe.Pointer(d)) } diff --git a/store/tikv/kv/option.go b/store/tikv/kv/option.go index a87ddcb7214a4..bac9316d41773 100644 --- a/store/tikv/kv/option.go +++ b/store/tikv/kv/option.go @@ -62,6 +62,7 @@ const ( ) // Priority value for transaction priority. +// TODO: remove after BR update. const ( PriorityNormal = iota PriorityLow diff --git a/store/tikv/scan.go b/store/tikv/scan.go index d2d151f59627f..19a14b3f73819 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -189,7 +189,7 @@ func (s *Scanner) getData(bo *Backoffer) error { } sreq := &pb.ScanRequest{ Context: &pb.Context{ - Priority: s.snapshot.priority, + Priority: s.snapshot.priority.ToPB(), NotFillCache: s.snapshot.notFillCache, IsolationLevel: s.snapshot.isolationLevel.ToPB(), }, @@ -207,7 +207,7 @@ func (s *Scanner) getData(bo *Backoffer) error { } s.snapshot.mu.RLock() req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdScan, sreq, s.snapshot.mu.replicaRead, &s.snapshot.replicaReadSeed, pb.Context{ - Priority: s.snapshot.priority, + Priority: s.snapshot.priority.ToPB(), NotFillCache: s.snapshot.notFillCache, TaskId: s.snapshot.mu.taskID, }) diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 1e54955f5fe4e..b7159c59931c5 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -45,6 +45,21 @@ const ( maxTimestamp = math.MaxUint64 ) +// Priority is the priority for tikv to execute a command. +type Priority kvrpcpb.CommandPri + +// Priority value for transaction priority. +const ( + PriorityNormal = Priority(kvrpcpb.CommandPri_Normal) + PriorityLow = Priority(kvrpcpb.CommandPri_Low) + PriorityHigh = Priority(kvrpcpb.CommandPri_High) +) + +// ToPB converts priority to wire type. +func (p Priority) ToPB() kvrpcpb.CommandPri { + return kvrpcpb.CommandPri(p) +} + // IsoLevel is the transaction's isolation level. type IsoLevel kvrpcpb.IsolationLevel @@ -65,7 +80,7 @@ type KVSnapshot struct { store *KVStore version uint64 isolationLevel IsoLevel - priority pb.CommandPri + priority Priority notFillCache bool syncLog bool keyOnly bool @@ -106,7 +121,7 @@ func newTiKVSnapshot(store *KVStore, ts uint64, replicaReadSeed uint32) *KVSnaps return &KVSnapshot{ store: store, version: ts, - priority: pb.CommandPri_Normal, + priority: PriorityNormal, vars: kv.DefaultVars, replicaReadSeed: replicaReadSeed, resolvedLocks: util.NewTSSet(5), @@ -293,7 +308,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec Keys: pending, Version: s.version, }, s.mu.replicaRead, &s.replicaReadSeed, pb.Context{ - Priority: s.priority, + Priority: s.priority.ToPB(), NotFillCache: s.notFillCache, TaskId: s.mu.taskID, }) @@ -443,7 +458,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, Key: k, Version: s.version, }, s.mu.replicaRead, &s.replicaReadSeed, pb.Context{ - Priority: s.priority, + Priority: s.priority.ToPB(), NotFillCache: s.notFillCache, TaskId: s.mu.taskID, }) @@ -551,8 +566,6 @@ func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) { // value of this option. Only ReplicaRead is supported for snapshot func (s *KVSnapshot) SetOption(opt int, val interface{}) { switch opt { - case kv.Priority: - s.priority = PriorityToPB(val.(int)) case kv.NotFillCache: s.notFillCache = val.(bool) case kv.SyncLog: @@ -610,6 +623,11 @@ func (s *KVSnapshot) SetIsolationLevel(level IsoLevel) { s.isolationLevel = level } +// SetPriority sets the priority for tikv to execute commands. +func (s *KVSnapshot) SetPriority(pri Priority) { + s.priority = pri +} + // SnapCacheHitCount gets the snapshot cache hit count. Only for test. func (s *KVSnapshot) SnapCacheHitCount() int { return int(atomic.LoadInt64(&s.mu.hitCnt)) diff --git a/store/tikv/tests/store_test.go b/store/tikv/tests/store_test.go index d10b09defd2ad..659dc6ea8f226 100644 --- a/store/tikv/tests/store_test.go +++ b/store/tikv/tests/store_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/oracle/oracles" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -124,7 +123,7 @@ func (s *testStoreSuite) TestRequestPriority(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) client.priority = pb.CommandPri_High - txn.SetOption(kv.Priority, kv.PriorityHigh) + txn.SetPriority(tikv.PriorityHigh) err = txn.Set([]byte("key"), []byte("value")) c.Assert(err, IsNil) err = txn.Commit(context.Background()) @@ -134,20 +133,20 @@ func (s *testStoreSuite) TestRequestPriority(c *C) { txn, err = s.store.Begin() c.Assert(err, IsNil) client.priority = pb.CommandPri_Low - txn.SetOption(kv.Priority, kv.PriorityLow) + txn.SetPriority(tikv.PriorityLow) _, err = txn.Get(context.TODO(), []byte("key")) c.Assert(err, IsNil) // A counter example. client.priority = pb.CommandPri_Low - txn.SetOption(kv.Priority, kv.PriorityNormal) + txn.SetPriority(tikv.PriorityNormal) _, err = txn.Get(context.TODO(), []byte("key")) // err is translated to "try again later" by backoffer, so doesn't check error value here. c.Assert(err, NotNil) // Cover Seek request. client.priority = pb.CommandPri_High - txn.SetOption(kv.Priority, kv.PriorityHigh) + txn.SetPriority(tikv.PriorityHigh) iter, err := txn.Iter([]byte("key"), nil) c.Assert(err, IsNil) for iter.Valid() { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 67477c510cee4..1377d02d28581 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -76,6 +76,7 @@ type KVTxn struct { binlog BinlogExecutor schemaLeaseChecker SchemaLeaseChecker + priority Priority isPessimistic bool kvFilter KVFilter } @@ -211,6 +212,12 @@ func (txn *KVTxn) SetPessimistic(b bool) { txn.isPessimistic = b } +// SetPriority sets the priority for both write and read. +func (txn *KVTxn) SetPriority(pri Priority) { + txn.priority = pri + txn.GetSnapshot().SetPriority(pri) +} + // SetKVFilter sets the filter to ignore key-values in memory buffer. func (txn *KVTxn) SetKVFilter(filter KVFilter) { txn.kvFilter = filter From 64938b80ee9538075c7a034a6072c87b74813248 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 29 Apr 2021 14:17:57 +0800 Subject: [PATCH 20/36] store/tikv: remove use of SyncLog transaction option in store/tikv (#24333) --- meta/meta.go | 2 +- store/driver/txn/txn_driver.go | 2 ++ store/tikv/2pc.go | 9 +-------- store/tikv/snapshot.go | 3 --- store/tikv/txn.go | 6 ++++++ 5 files changed, 10 insertions(+), 12 deletions(-) diff --git a/meta/meta.go b/meta/meta.go index e5a0c97658b43..2682ed5b47d1e 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -95,7 +95,7 @@ type Meta struct { // If the current Meta needs to handle a job, jobListKey is the type of the job's list. func NewMeta(txn kv.Transaction, jobListKeys ...JobListKeyType) *Meta { txn.SetOption(tikvstore.Priority, kv.PriorityHigh) - txn.SetOption(tikvstore.SyncLog, true) + txn.SetOption(tikvstore.SyncLog, struct{}{}) t := structure.NewStructure(txn, txn, mMetaPrefix) listKey := DefaultJobListKey if len(jobListKeys) != 0 { diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index ddec266419fd9..9984473d926e9 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -134,6 +134,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { case tikvstore.IsolationLevel: level := getTiKVIsolationLevel(val.(kv.IsoLevel)) txn.KVTxn.GetSnapshot().SetIsolationLevel(level) + case tikvstore.SyncLog: + txn.EnableForceSyncLog() case tikvstore.Priority: txn.KVTxn.SetPriority(getTiKVPriority(val.(int))) case tikvstore.Pessimistic: diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index e2172d17bd806..dbe74dd844bf0 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -426,7 +426,7 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error { c.hasNoNeedCommitKeys = checkCnt > 0 c.lockTTL = txnLockTTL(txn.startTime, size) c.priority = txn.priority.ToPB() - c.syncLog = getTxnSyncLog(txn) + c.syncLog = txn.syncLog c.setDetail(commitDetail) return nil } @@ -1649,13 +1649,6 @@ func (batchExe *batchExecutor) process(batches []batchMutations) error { return err } -func getTxnSyncLog(txn *KVTxn) bool { - if syncOption := txn.us.GetOption(kv.SyncLog); syncOption != nil { - return syncOption.(bool) - } - return false -} - func (c *twoPhaseCommitter) setDetail(d *util.CommitDetails) { atomic.StorePointer(&c.detail, unsafe.Pointer(d)) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index b7159c59931c5..21063ec122dfd 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -82,7 +82,6 @@ type KVSnapshot struct { isolationLevel IsoLevel priority Priority notFillCache bool - syncLog bool keyOnly bool vars *kv.Variables replicaReadSeed uint32 @@ -568,8 +567,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { switch opt { case kv.NotFillCache: s.notFillCache = val.(bool) - case kv.SyncLog: - s.syncLog = val.(bool) case kv.SnapshotTS: s.setSnapshotTS(val.(uint64)) case kv.ReplicaRead: diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 1377d02d28581..155ce74ae2aee 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -76,6 +76,7 @@ type KVTxn struct { binlog BinlogExecutor schemaLeaseChecker SchemaLeaseChecker + syncLog bool priority Priority isPessimistic bool kvFilter KVFilter @@ -207,6 +208,11 @@ func (txn *KVTxn) SetSchemaLeaseChecker(checker SchemaLeaseChecker) { txn.schemaLeaseChecker = checker } +// EnableForceSyncLog indicates tikv to always sync log for the transaction. +func (txn *KVTxn) EnableForceSyncLog() { + txn.syncLog = true +} + // SetPessimistic indicates if the transaction should use pessimictic lock. func (txn *KVTxn) SetPessimistic(b bool) { txn.isPessimistic = b From ec3558b7bfbd45f9312fa2c23754a05f9b327af8 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 29 Apr 2021 14:35:57 +0800 Subject: [PATCH 21/36] store/tikv: remove use of NotFillCache transaction option in store/tikv (#24361) --- store/driver/txn/snapshot.go | 2 ++ store/driver/txn/txn_driver.go | 6 ++++-- store/tikv/snapshot.go | 8 ++++++-- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 28ecbd567e786..34ad18b11b209 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -69,6 +69,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetIsolationLevel(level) case tikvstore.Priority: s.KVSnapshot.SetPriority(getTiKVPriority(val.(int))) + case tikvstore.NotFillCache: + s.KVSnapshot.SetNotFillCache(val.(bool)) default: s.KVSnapshot.SetOption(opt, val) } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 9984473d926e9..4ab9b0f437945 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -134,10 +134,12 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { case tikvstore.IsolationLevel: level := getTiKVIsolationLevel(val.(kv.IsoLevel)) txn.KVTxn.GetSnapshot().SetIsolationLevel(level) - case tikvstore.SyncLog: - txn.EnableForceSyncLog() case tikvstore.Priority: txn.KVTxn.SetPriority(getTiKVPriority(val.(int))) + case tikvstore.NotFillCache: + txn.KVTxn.GetSnapshot().SetNotFillCache(val.(bool)) + case tikvstore.SyncLog: + txn.EnableForceSyncLog() case tikvstore.Pessimistic: txn.SetPessimistic(val.(bool)) default: diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 21063ec122dfd..750b20b9045f9 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -565,8 +565,6 @@ func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) { // value of this option. Only ReplicaRead is supported for snapshot func (s *KVSnapshot) SetOption(opt int, val interface{}) { switch opt { - case kv.NotFillCache: - s.notFillCache = val.(bool) case kv.SnapshotTS: s.setSnapshotTS(val.(uint64)) case kv.ReplicaRead: @@ -610,6 +608,12 @@ func (s *KVSnapshot) DelOption(opt int) { } } +// SetNotFillCache indicates whether tikv should skip filling cache when +// loading data. +func (s *KVSnapshot) SetNotFillCache(b bool) { + s.notFillCache = b +} + // SetKeyOnly indicates if tikv can return only keys. func (s *KVSnapshot) SetKeyOnly(b bool) { s.keyOnly = b From 257ce7d93de5f5dfeb2b56b0d136d844907bcdf2 Mon Sep 17 00:00:00 2001 From: rebelice Date: Thu, 29 Apr 2021 14:51:57 +0800 Subject: [PATCH 22/36] planner: filter conflict read_from_storage hints (#24313) --- planner/core/integration_test.go | 31 +++++++++++++++++++ planner/core/logical_plan_builder.go | 7 ++--- .../core/testdata/integration_suite_in.json | 3 +- .../core/testdata/integration_suite_out.json | 13 -------- 4 files changed, 35 insertions(+), 19 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index a6faf4b0cda08..25719198b66e4 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3524,3 +3524,34 @@ func (s *testIntegrationSuite) TestIssue24095(c *C) { tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) } } + +func (s *testIntegrationSuite) TestConflictReadFromStorage(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(`create table t ( + a int, b int, c varchar(20), + primary key(a), key(b), key(c) + ) partition by range columns(a) ( + partition p0 values less than(6), + partition p1 values less than(11), + partition p2 values less than(16));`) + tk.MustExec(`insert into t values (1,1,"1"), (2,2,"2"), (8,8,"8"), (11,11,"11"), (15,15,"15")`) + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Se) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + c.Assert(exists, IsTrue) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustQuery(`explain select /*+ read_from_storage(tikv[t partition(p0)], tiflash[t partition(p1, p2)]) */ * from t`) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Storage hints are conflict, you can only specify one storage type of table test.t")) + tk.MustQuery(`explain select /*+ read_from_storage(tikv[t], tiflash[t]) */ * from t`) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 Storage hints are conflict, you can only specify one storage type of table test.t")) +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 37bef8ffbc38d..0bf3d3fd3a7e4 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -610,10 +610,9 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { } } if hintTbl := hintInfo.ifPreferTiFlash(alias); hintTbl != nil { - // 1. `ds.tableInfo.Partition == nil`, which means the hint takes effect in the whole table. - // 2. `ds.preferStoreType != 0`, which means there's a hint hit the both TiKV value and TiFlash value for table. - // If it's satisfied the above two conditions, then we can make sure there are some hints conflicted. - if ds.preferStoreType != 0 && ds.tableInfo.Partition == nil { + // `ds.preferStoreType != 0`, which means there's a hint hit the both TiKV value and TiFlash value for table. + // We can't support read a table from two different storages, even partition table. + if ds.preferStoreType != 0 { errMsg := fmt.Sprintf("Storage hints are conflict, you can only specify one storage type of table %s.%s", alias.dbName.L, alias.tblName.L) warning := ErrInternal.GenWithStack(errMsg) diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 7a734a9b9a4cc..bf549da9431e2 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -168,8 +168,7 @@ "select /*+ hash_join(t1, t2 partition(p0)) */ * from t t1 join t t2 on t1.a = t2.a", "select /*+ use_index_merge(t partition(p0)) */ * from t where t.b = 1 or t.c = \"8\"", "select /*+ use_index_merge(t partition(p0, p1) primary, b) */ * from t where t.a = 1 or t.b = 2", - "select /*+ use_index(t partition(p0) b) */ * from t partition(p0, p1)", - "select /*+ read_from_storage(tikv[t partition(p0)], tiflash[t partition(p1, p2)]) */ * from t" + "select /*+ use_index(t partition(p0) b) */ * from t partition(p0, p1)" ] }, { diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 47ce50c6ab717..a454ff4f31891 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -879,19 +879,6 @@ " └─TableFullScan 10000.00 cop[tiflash] table:t, partition:p1 keep order:false, stats:pseudo" ], "Warn": null - }, - { - "SQL": "select /*+ read_from_storage(tikv[t partition(p0)], tiflash[t partition(p1, p2)]) */ * from t", - "Plan": [ - "PartitionUnion 30000.00 root ", - "├─TableReader 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo", - "├─TableReader 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tiflash] table:t, partition:p1 keep order:false, stats:pseudo", - "└─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tiflash] table:t, partition:p2 keep order:false, stats:pseudo" - ], - "Warn": null } ] }, From 1a69af6aac8e4051ce3777aec958c90261e005ba Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 29 Apr 2021 15:35:56 +0800 Subject: [PATCH 23/36] =?UTF-8?q?store/tikv:=20use=20tikv.error.ErrWriteCo?= =?UTF-8?q?nflictInLatch=20instead=20of=20kv.ErrW=E2=80=A6=20(#24348)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- store/driver/txn/error.go | 5 +++++ store/tikv/config/config.go | 5 +++++ store/tikv/error/error.go | 9 +++++++++ store/tikv/tests/2pc_test.go | 4 ++-- store/tikv/txn.go | 3 +-- 5 files changed, 22 insertions(+), 4 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index e2e12cfde62e6..2c2ef4bc60d3b 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -156,6 +156,10 @@ func toTiDBErr(err error) error { return kv.ErrNotExist } + if e, ok := err.(*tikverr.ErrWriteConflictInLatch); ok { + return kv.ErrWriteConflictInTiDB.FastGenByArgs(e.StartTS) + } + if e, ok := err.(*tikverr.ErrTxnTooLarge); ok { return kv.ErrTxnTooLarge.GenWithStackByArgs(e.Size) } @@ -171,6 +175,7 @@ func toTiDBErr(err error) error { if errors.ErrorEqual(err, tikverr.ErrInvalidTxn) { return kv.ErrInvalidTxn } + return errors.Trace(err) } diff --git a/store/tikv/config/config.go b/store/tikv/config/config.go index a8337cdcedf8c..6fe0c018498ea 100644 --- a/store/tikv/config/config.go +++ b/store/tikv/config/config.go @@ -36,6 +36,11 @@ const ( DefStoresRefreshInterval = 60 ) +func init() { + conf := DefaultConfig() + StoreGlobalConfig(&conf) +} + // Config contains configuration options. type Config struct { CommitterConcurrency int diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 4c6015ba4eba5..db5be4ebc3666 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -130,6 +130,15 @@ func NewErrWriteConfictWithArgs(startTs, conflictTs, conflictCommitTs uint64, ke return &ErrWriteConflict{WriteConflict: &conflict} } +// ErrWriteConflictInLatch is the error when the commit meets an write conflict error when local latch is enabled. +type ErrWriteConflictInLatch struct { + StartTS uint64 +} + +func (e *ErrWriteConflictInLatch) Error() string { + return fmt.Sprintf("write conflict in latch,startTS: %v", e.StartTS) +} + // ErrRetryable wraps *kvrpcpb.Retryable to implement the error interface. type ErrRetryable struct { Retryable string diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index e70131a235474..6d7d7e89d1a8e 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" - tidbkv "github.com/pingcap/tidb/kv" drivertxn "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" @@ -307,7 +306,8 @@ func (s *testCommitterSuite) TestContextCancelRetryable(c *C) { c.Assert(err, IsNil) err = txn2.Commit(context.Background()) c.Assert(err, NotNil) - c.Assert(tidbkv.ErrWriteConflictInTiDB.Equal(err), IsTrue, Commentf("err: %s", err)) + _, ok := err.(*tikverr.ErrWriteConflictInLatch) + c.Assert(ok, IsTrue, Commentf("err: %s", err)) } func (s *testCommitterSuite) TestContextCancelCausingUndetermined(c *C) { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 155ce74ae2aee..4a73bb30b4d77 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" - tidbkv "github.com/pingcap/tidb/kv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -320,7 +319,7 @@ func (txn *KVTxn) Commit(ctx context.Context) error { } defer txn.store.txnLatches.UnLock(lock) if lock.IsStale() { - return tidbkv.ErrWriteConflictInTiDB.FastGenByArgs(txn.startTS) + return &tikverr.ErrWriteConflictInLatch{StartTS: txn.startTS} } err = committer.execute(ctx) if val == nil || sessionID > 0 { From 68a22eda627e53625013b313ba416795b5209dcf Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 29 Apr 2021 16:21:57 +0800 Subject: [PATCH 24/36] store/tikv: remove use of SnapshotTS transaction option in store/tikv (#24362) --- store/driver/txn/snapshot.go | 2 ++ store/driver/txn/txn_driver.go | 2 ++ store/tikv/snapshot.go | 5 ++--- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 34ad18b11b209..3d41ef8cfa1ae 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -71,6 +71,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetPriority(getTiKVPriority(val.(int))) case tikvstore.NotFillCache: s.KVSnapshot.SetNotFillCache(val.(bool)) + case tikvstore.SnapshotTS: + s.KVSnapshot.SetSnapshotTS(val.(uint64)) default: s.KVSnapshot.SetOption(opt, val) } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 4ab9b0f437945..0a78c8ed935d3 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -142,6 +142,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.EnableForceSyncLog() case tikvstore.Pessimistic: txn.SetPessimistic(val.(bool)) + case tikvstore.SnapshotTS: + txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) default: txn.KVTxn.SetOption(opt, val) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 750b20b9045f9..0b09a408d7dc9 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -127,7 +127,8 @@ func newTiKVSnapshot(store *KVStore, ts uint64, replicaReadSeed uint32) *KVSnaps } } -func (s *KVSnapshot) setSnapshotTS(ts uint64) { +// SetSnapshotTS resets the timestamp for reads. +func (s *KVSnapshot) SetSnapshotTS(ts uint64) { // Sanity check for snapshot version. if ts >= math.MaxInt64 && ts != math.MaxUint64 { err := errors.Errorf("try to get snapshot with a large ts %d", ts) @@ -565,8 +566,6 @@ func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) { // value of this option. Only ReplicaRead is supported for snapshot func (s *KVSnapshot) SetOption(opt int, val interface{}) { switch opt { - case kv.SnapshotTS: - s.setSnapshotTS(val.(uint64)) case kv.ReplicaRead: s.mu.Lock() s.mu.replicaRead = val.(kv.ReplicaReadType) From 371e10ed1b8479b097f6da000b87275cdd68a607 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 29 Apr 2021 16:59:57 +0800 Subject: [PATCH 25/36] store/tikv: remove use of CommitHook transaction option in store/tikv (#24363) --- store/driver/txn/txn_driver.go | 2 ++ store/tikv/txn.go | 8 ++++++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 0a78c8ed935d3..968d0f8701233 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -144,6 +144,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetPessimistic(val.(bool)) case tikvstore.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) + case tikvstore.CommitHook: + txn.SetCommitCallback(val.(func(string, error))) default: txn.KVTxn.SetOption(opt, val) } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 4a73bb30b4d77..de0d607f5f55a 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -187,8 +187,6 @@ func (txn *KVTxn) SetOption(opt int, val interface{}) { txn.txnInfoSchema = val.(SchemaVer) case kv.SchemaAmender: txn.schemaAmender = val.(SchemaAmender) - case kv.CommitHook: - txn.commitCallback = val.(func(info string, err error)) } } @@ -223,6 +221,12 @@ func (txn *KVTxn) SetPriority(pri Priority) { txn.GetSnapshot().SetPriority(pri) } +// SetCommitCallback sets up a function that will be called when the transaction +// is finished. +func (txn *KVTxn) SetCommitCallback(f func(string, error)) { + txn.commitCallback = f +} + // SetKVFilter sets the filter to ignore key-values in memory buffer. func (txn *KVTxn) SetKVFilter(filter KVFilter) { txn.kvFilter = filter From f1521632e2c895940070577fdc9e9504a48ed803 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Thu, 29 Apr 2021 18:11:57 +0800 Subject: [PATCH 26/36] executor: fix a concurrent-access problem caused by accessing a single parser object in session concurrently (#24338) --- executor/distsql_test.go | 2 +- executor/index_lookup_join_test.go | 2 +- session/bootstrap_test.go | 3 ++- session/schema_amender_test.go | 3 ++- session/session.go | 19 +++++++++++-------- 5 files changed, 17 insertions(+), 12 deletions(-) diff --git a/executor/distsql_test.go b/executor/distsql_test.go index 2a3de537c748a..eca6e1d016b40 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -360,7 +360,7 @@ func (s *testSuite3) TestPartitionTableIndexJoinIndexLookUp(c *C) { tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec(`create table t (a int, b int, key(a)) partition by hash(a) partitions 4`) tk.MustExec("create table tnormal (a int, b int, key(a), key(b))") - nRows := 64 + nRows := 512 values := make([]string, 0, nRows) for i := 0; i < nRows; i++ { values = append(values, fmt.Sprintf("(%v, %v)", rand.Intn(nRows), rand.Intn(nRows))) diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index 6b42c0f63cf6f..9515907b8eba6 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -344,7 +344,7 @@ func (s *testSuite5) TestPartitionTableIndexJoinAndIndexReader(c *C) { tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec(`create table t (a int, b int, key(a)) partition by hash(a) partitions 4`) tk.MustExec("create table tnormal (a int, b int, key(a), key(b))") - nRows := 64 + nRows := 512 values := make([]string, 0, nRows) for i := 0; i < nRows; i++ { values = append(values, fmt.Sprintf("(%v, %v)", rand.Intn(nRows), rand.Intn(nRows))) diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index e7e8398845d88..ffa9ba22e5723 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -16,6 +16,7 @@ package session import ( "context" "fmt" + "sync" . "github.com/pingcap/check" "github.com/pingcap/parser" @@ -118,7 +119,7 @@ func globalVarsCount() int64 { func (s *testBootstrapSuite) bootstrapWithOnlyDDLWork(store kv.Storage, c *C) { ss := &session{ store: store, - parser: parser.New(), + parserPool: &sync.Pool{New: func() interface{} { return parser.New() }}, sessionVars: variable.NewSessionVars(), } ss.txn.init() diff --git a/session/schema_amender_test.go b/session/schema_amender_test.go index 91ff6a3e29cc4..ca05f4a74dbff 100644 --- a/session/schema_amender_test.go +++ b/session/schema_amender_test.go @@ -18,6 +18,7 @@ import ( "context" "sort" "strconv" + "sync" . "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/kvrpcpb" @@ -255,7 +256,7 @@ func (s *testSchemaAmenderSuite) TestAmendCollectAndGenMutations(c *C) { defer store.Close() se := &session{ store: store, - parser: parser.New(), + parserPool: &sync.Pool{New: func() interface{} { return parser.New() }}, sessionVars: variable.NewSessionVars(), } startStates := []model.SchemaState{model.StateNone, model.StateDeleteOnly, model.StateWriteOnly, model.StateWriteReorganization} diff --git a/session/session.go b/session/session.go index 52d59514d0223..85aabd16a12e8 100644 --- a/session/session.go +++ b/session/session.go @@ -194,7 +194,7 @@ type session struct { store kv.Storage - parser *parser.Parser + parserPool *sync.Pool preparedPlanCache *kvcache.SimpleLRUCache @@ -1089,9 +1089,12 @@ func (s *session) ParseSQL(ctx context.Context, sql, charset, collation string) defer span1.Finish() } defer trace.StartRegion(ctx, "ParseSQL").End() - s.parser.SetSQLMode(s.sessionVars.SQLMode) - s.parser.SetParserConfig(s.sessionVars.BuildParserConfig()) - return s.parser.Parse(sql, charset, collation) + + p := s.parserPool.Get().(*parser.Parser) + defer s.parserPool.Put(p) + p.SetSQLMode(s.sessionVars.SQLMode) + p.SetParserConfig(s.sessionVars.BuildParserConfig()) + return p.Parse(sql, charset, collation) } func (s *session) SetProcessInfo(sql string, t time.Time, command byte, maxExecutionTime uint64) { @@ -2102,7 +2105,7 @@ func CreateSessionWithOpt(store kv.Storage, opt *Opt) (Session, error) { } privilege.BindPrivilegeManager(s, pm) - sessionBindHandle := bindinfo.NewSessionBindHandle(s.parser) + sessionBindHandle := bindinfo.NewSessionBindHandle(parser.New()) s.SetValue(bindinfo.SessionBindInfoKeyType, sessionBindHandle) // Add stats collector, and it will be freed by background stats worker // which periodically updates stats using the collected data. @@ -2341,7 +2344,7 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) { } s := &session{ store: store, - parser: parser.New(), + parserPool: &sync.Pool{New: func() interface{} { return parser.New() }}, sessionVars: variable.NewSessionVars(), ddlOwnerChecker: dom.DDL().OwnerManager(), client: store.GetClient(), @@ -2363,7 +2366,7 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) { s.sessionVars.BinlogClient = binloginfo.GetPumpsClient() s.txn.init() - sessionBindHandle := bindinfo.NewSessionBindHandle(s.parser) + sessionBindHandle := bindinfo.NewSessionBindHandle(parser.New()) s.SetValue(bindinfo.SessionBindInfoKeyType, sessionBindHandle) return s, nil } @@ -2375,7 +2378,7 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) { func CreateSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, error) { s := &session{ store: store, - parser: parser.New(), + parserPool: &sync.Pool{New: func() interface{} { return parser.New() }}, sessionVars: variable.NewSessionVars(), client: store.GetClient(), mppClient: store.GetMPPClient(), From 6b0b74e5bef94979a54867813925a464e7c2caea Mon Sep 17 00:00:00 2001 From: rebelice Date: Thu, 29 Apr 2021 19:31:57 +0800 Subject: [PATCH 27/36] statistics: fix three bugs about MergePartTopN2GlobalTopN (#24370) --- statistics/cmsketch.go | 13 +++++++++---- statistics/handle/handle.go | 2 +- statistics/handle/handle_test.go | 18 ++++++++++++++++++ statistics/histogram.go | 2 ++ 4 files changed, 30 insertions(+), 5 deletions(-) diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 6587340c4a05c..a512b21a484a5 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -638,7 +638,7 @@ func NewTopN(n int) *TopN { // 1. `*TopN` is the final global-level topN. // 2. `[]TopNMeta` is the left topN value from the partition-level TopNs, but is not placed to global-level TopN. We should put them back to histogram latter. // 3. `[]*Histogram` are the partition-level histograms which just delete some values when we merge the global-level topN. -func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, topNs []*TopN, n uint32, hists []*Histogram, isIndex bool) (*TopN, []TopNMeta, []*Histogram, error) { +func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, version int, topNs []*TopN, n uint32, hists []*Histogram, isIndex bool) (*TopN, []TopNMeta, []*Histogram, error) { if checkEmptyTopNs(topNs) { return nil, nil, hists, nil } @@ -674,7 +674,7 @@ func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, topNs []*TopN, n uin // 1. Check the topN first. // 2. If the topN doesn't contain the value corresponding to encodedVal. We should check the histogram. for j := 0; j < partNum; j++ { - if j == i || topNs[j].findTopN(val.Encoded) != -1 { + if (j == i && version >= 2) || topNs[j].findTopN(val.Encoded) != -1 { continue } // Get the encodedVal from the hists[j] @@ -706,7 +706,7 @@ func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, topNs []*TopN, n uin if count != 0 { counter[encodedVal] += count // Remove the value corresponding to encodedVal from the histogram. - removeVals[j] = append(removeVals[j], TopNMeta{Encoded: val.Encoded, Count: uint64(count)}) + removeVals[j] = append(removeVals[j], TopNMeta{Encoded: datum.GetBytes(), Count: uint64(count)}) } } } @@ -714,7 +714,12 @@ func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, topNs []*TopN, n uin // Remove the value from the Hists. for i := 0; i < partNum; i++ { if len(removeVals[i]) > 0 { - hists[i].RemoveVals(removeVals[i]) + tmp := removeVals[i] + sort.Slice(tmp, func(i, j int) bool { + cmpResult := bytes.Compare(tmp[i].Encoded, tmp[j].Encoded) + return cmpResult < 0 + }) + hists[i].RemoveVals(tmp) } } numTop := len(counter) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 32c256afe8834..8ea4f2fe15aa5 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -435,7 +435,7 @@ func (h *Handle) mergePartitionStats2GlobalStats(sc sessionctx.Context, opts map // Because after merging TopN, some numbers will be left. // These remaining topN numbers will be used as a separate bucket for later histogram merging. var popedTopN []statistics.TopNMeta - globalStats.TopN[i], popedTopN, allHg[i], err = statistics.MergePartTopN2GlobalTopN(sc.GetSessionVars().StmtCtx, allTopN[i], uint32(opts[ast.AnalyzeOptNumTopN]), allHg[i], isIndex == 1) + globalStats.TopN[i], popedTopN, allHg[i], err = statistics.MergePartTopN2GlobalTopN(sc.GetSessionVars().StmtCtx, sc.GetSessionVars().AnalyzeVersion, allTopN[i], uint32(opts[ast.AnalyzeOptNumTopN]), allHg[i], isIndex == 1) if err != nil { return } diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 0a55f03722b75..16561c78ecc2b 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2784,3 +2784,21 @@ func (s *testStatsSuite) TestStatsCacheUpdateSkip(c *C) { statsTbl2 := h.GetTableStats(tableInfo) c.Assert(statsTbl1, Equals, statsTbl2) } + +func (s *testSerialStatsSuite) TestIssues24349(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("set @@tidb_partition_prune_mode='dynamic'") + testKit.MustExec("set @@tidb_analyze_version=2") + defer testKit.MustExec("set @@tidb_analyze_version=1") + defer testKit.MustExec("set @@tidb_partition_prune_mode='static'") + testKit.MustExec("create table t (a int, b int) partition by hash(a) partitions 3") + testKit.MustExec("insert into t values (0, 3), (0, 3), (0, 3), (0, 2), (1, 1), (1, 2), (1, 2), (1, 2), (1, 3), (1, 4), (2, 1), (2, 1)") + testKit.MustExec("analyze table t with 1 topn, 3 buckets") + testKit.MustQuery("show stats_buckets where partition_name='global'").Check(testkit.Rows( + "test t global a 0 0 2 2 0 2 0", + "test t global b 0 0 3 1 1 2 0", + "test t global b 0 1 10 1 4 4 0", + )) +} diff --git a/statistics/histogram.go b/statistics/histogram.go index e58abddc16867..cdf0ee9f965ff 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -344,6 +344,7 @@ func (hg *Histogram) BucketToString(bktID, idxCols int) string { } // RemoveVals remove the given values from the histogram. +// This function contains an **ASSUMPTION**: valCntPairs is sorted in ascending order. func (hg *Histogram) RemoveVals(valCntPairs []TopNMeta) { totalSubCnt := int64(0) var cmpResult int @@ -352,6 +353,7 @@ func (hg *Histogram) RemoveVals(valCntPairs []TopNMeta) { // If the current val smaller than current bucket's lower bound, skip it. cmpResult = bytes.Compare(hg.Bounds.Column(0).GetRaw(bktIdx*2), valCntPairs[pairIdx].Encoded) if cmpResult > 0 { + pairIdx++ continue } // If the current val bigger than current bucket's upper bound, break. From af6cece281c5d396f0cbca9924e1b89b3751528a Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 29 Apr 2021 21:49:58 +0800 Subject: [PATCH 28/36] *: support read and write operations for the global temporary table (#24196) --- ddl/column_type_change_test.go | 2 +- ddl/ddl_api.go | 8 ++++++++ executor/seqtest/seq_executor_test.go | 2 +- go.mod | 2 +- go.sum | 5 +++-- planner/core/preprocess.go | 4 ++-- session/session.go | 22 ++++++++++++++++++++++ session/session_test.go | 21 +++++++++++++++++++++ sessionctx/variable/session.go | 2 ++ table/tables/tables.go | 21 +++++++++++++++++++++ 10 files changed, 82 insertions(+), 7 deletions(-) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 402704401eff6..68ee059f47305 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -233,7 +233,7 @@ func (s *testColumnTypeChangeSuite) TestRollbackColumnTypeChangeBetweenInteger(c SQL := "alter table t modify column c2 int not null" _, err := tk.Exec(SQL) c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:1]MockRollingBackInCallBack-none") + c.Assert(err.Error(), Equals, "[ddl:1]MockRollingBackInCallBack-queueing") assertRollBackedColUnchanged(c, tk) // Mock roll back at model.StateDeleteOnly. diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 6520bc77dbba9..06e43e905d616 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1751,6 +1751,14 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh if err != nil { return nil, errors.Trace(err) } + switch s.TemporaryKeyword { + case ast.TemporaryGlobal: + tbInfo.TempTableType = model.TempTableGlobal + case ast.TemporaryLocal: + tbInfo.TempTableType = model.TempTableLocal + case ast.TemporaryNone: + tbInfo.TempTableType = model.TempTableNone + } if err = setTableAutoRandomBits(ctx, tbInfo, colDefs); err != nil { return nil, errors.Trace(err) diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 7b38a6f6b673e..061e09dcc1315 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -967,7 +967,7 @@ func (s *seqTestSuite) TestBatchInsertDelete(c *C) { atomic.StoreUint64(&kv.TxnTotalSizeLimit, originLimit) }() // Set the limitation to a small value, make it easier to reach the limitation. - atomic.StoreUint64(&kv.TxnTotalSizeLimit, 5000) + atomic.StoreUint64(&kv.TxnTotalSizeLimit, 5500) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/go.mod b/go.mod index 3fc648111c65d..4ad29e29e6273 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20210402093459-65aa336ccbbf github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 - github.com/pingcap/parser v0.0.0-20210421190254-588138d35e55 + github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 diff --git a/go.sum b/go.sum index a4450303c0621..f3396ccd03898 100644 --- a/go.sum +++ b/go.sum @@ -443,8 +443,8 @@ github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 h1:ERrF0fTuIOnwfGbt71Ji3DKbOEaP189tjym50u8gpC8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20210421190254-588138d35e55 h1:J/NfwCFFPCv7h44ft+2pS3KiMyvOkHprPM5NhDJEoHc= -github.com/pingcap/parser v0.0.0-20210421190254-588138d35e55/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= +github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde h1:CcGOCE3kr8aYBy6rRcWWldidL1X5smQxV79nlnzOk+o= +github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 h1:A9KL9R+lWSVPH8IqUuH1QSTRJ5FGoY1bT2IcfPKsWD8= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= @@ -500,6 +500,7 @@ 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= diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index d84fede3e5b00..a3719fe4c4b0b 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -564,7 +564,7 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { return } enableNoopFuncs := p.ctx.GetSessionVars().EnableNoopFuncs - if stmt.IsTemporary && !enableNoopFuncs { + if stmt.TemporaryKeyword == ast.TemporaryLocal && !enableNoopFuncs { p.err = expression.ErrFunctionsNoopImpl.GenWithStackByArgs("CREATE TEMPORARY TABLE") return } @@ -676,7 +676,7 @@ func (p *preprocessor) checkDropSequenceGrammar(stmt *ast.DropSequenceStmt) { func (p *preprocessor) checkDropTableGrammar(stmt *ast.DropTableStmt) { p.checkDropTableNames(stmt.Tables) enableNoopFuncs := p.ctx.GetSessionVars().EnableNoopFuncs - if stmt.IsTemporary && !enableNoopFuncs { + if stmt.TemporaryKeyword == ast.TemporaryLocal && !enableNoopFuncs { p.err = expression.ErrFunctionsNoopImpl.GenWithStackByArgs("DROP TEMPORARY TABLE") return } diff --git a/session/session.go b/session/session.go index 85aabd16a12e8..67354cd15e494 100644 --- a/session/session.go +++ b/session/session.go @@ -68,6 +68,7 @@ import ( tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/telemetry" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" @@ -509,6 +510,27 @@ func (s *session) doCommit(ctx context.Context) error { s.GetSessionVars().TxnCtx.IsExplicit && s.GetSessionVars().GuaranteeLinearizability) } + // Filter out the temporary table key-values. + if tables := s.sessionVars.TxnCtx.GlobalTemporaryTables; tables != nil { + memBuffer := s.txn.GetMemBuffer() + for tid := range tables { + seekKey := tablecodec.EncodeTablePrefix(tid) + endKey := tablecodec.EncodeTablePrefix(tid + 1) + iter, err := memBuffer.Iter(seekKey, endKey) + if err != nil { + return err + } + for iter.Valid() && iter.Key().HasPrefix(seekKey) { + if err = memBuffer.Delete(iter.Key()); err != nil { + return errors.Trace(err) + } + if err = iter.Next(); err != nil { + return errors.Trace(err) + } + } + } + } + return s.txn.Commit(tikvutil.SetSessionID(ctx, s.GetSessionVars().ConnectionID)) } diff --git a/session/session_test.go b/session/session_test.go index 91dd04abded29..a3a283eb09848 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4243,3 +4243,24 @@ func (s *testSessionSerialSuite) TestParseWithParams(c *C) { c.Assert(err, IsNil) c.Assert(sb.String(), Equals, "SELECT 3") } + +func (s *testSessionSuite3) TestGlobalTemporaryTable(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create global temporary table g_tmp (a int primary key, b int, c int, index i_b(b)) on commit delete rows") + tk.MustExec("begin") + tk.MustExec("insert into g_tmp values (3, 3, 3)") + tk.MustExec("insert into g_tmp values (4, 7, 9)") + + // Cover table scan. + tk.MustQuery("select * from g_tmp").Check(testkit.Rows("3 3 3", "4 7 9")) + // Cover index reader. + tk.MustQuery("select b from g_tmp where b > 3").Check(testkit.Rows("7")) + // Cover index lookup. + tk.MustQuery("select c from g_tmp where b = 3").Check(testkit.Rows("3")) + // Cover point get. + tk.MustQuery("select * from g_tmp where a = 3").Check(testkit.Rows("3 3 3")) + tk.MustExec("commit") + + // The global temporary table data is discard after the transaction commit. + tk.MustQuery("select * from g_tmp").Check(testkit.Rows()) +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 536f446b16d68..a154d3d451b06 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -172,6 +172,8 @@ type TransactionContext struct { // TableDeltaMap lock to prevent potential data race tdmLock sync.Mutex + + GlobalTemporaryTables map[int64]struct{} } // GetShard returns the shard prefix for the next `count` rowids. diff --git a/table/tables/tables.go b/table/tables/tables.go index e725130834583..d1ae7d804fb58 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -322,6 +322,10 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context, sh := memBuffer.Staging() defer memBuffer.Cleanup(sh) + if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { + addTemporaryTableID(sctx, meta.ID) + } + var colIDs, binlogColIDs []int64 var row, binlogOldRow, binlogNewRow []types.Datum numColsCap := len(newData) + 1 // +1 for the extra handle column that we may need to append. @@ -584,6 +588,14 @@ func TryGetCommonPkColumns(tbl table.Table) []*table.Column { return pkCols } +func addTemporaryTableID(sctx sessionctx.Context, id int64) { + txnCtx := sctx.GetSessionVars().TxnCtx + if txnCtx.GlobalTemporaryTables == nil { + txnCtx.GlobalTemporaryTables = make(map[int64]struct{}) + } + txnCtx.GlobalTemporaryTables[id] = struct{}{} +} + // AddRecord implements table.Table AddRecord interface. func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) { txn, err := sctx.Txn(true) @@ -596,6 +608,10 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts . fn.ApplyOn(&opt) } + if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { + addTemporaryTableID(sctx, meta.ID) + } + var ctx context.Context if opt.Ctx != nil { ctx = opt.Ctx @@ -992,6 +1008,11 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type if err != nil { return err } + + if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { + addTemporaryTableID(ctx, meta.ID) + } + // The table has non-public column and this column is doing the operation of "modify/change column". if len(t.Columns) > len(r) && t.Columns[len(r)].ChangeStateInfo != nil { r = append(r, r[t.Columns[len(r)].ChangeStateInfo.DependencyColumnOffset]) From c5ca2ea7f91786f682c448aebfd07513af20c40e Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Thu, 29 Apr 2021 10:31:57 -0600 Subject: [PATCH 29/36] *: add security enhanced mode part 2 (#24279) --- docs/design/2021-03-09-dynamic-privileges.md | 4 +- .../2021-03-09-security-enhanced-mode.md | 18 +++---- executor/infoschema_reader.go | 36 +++++++++++-- executor/show.go | 8 +++ executor/slow_query.go | 6 +-- infoschema/cluster.go | 11 +++- privilege/privileges/privileges.go | 3 +- privilege/privileges/privileges_test.go | 52 +++++++++++++++++++ util/sem/sem.go | 9 ++++ util/sem/sem_test.go | 7 +++ 10 files changed, 134 insertions(+), 20 deletions(-) diff --git a/docs/design/2021-03-09-dynamic-privileges.md b/docs/design/2021-03-09-dynamic-privileges.md index 48b324914ce80..7ad0d59d2c54e 100644 --- a/docs/design/2021-03-09-dynamic-privileges.md +++ b/docs/design/2021-03-09-dynamic-privileges.md @@ -1,7 +1,7 @@ # Proposal: - Author(s): [morgo](https://github.com/morgo) -- Last updated: April 12, 2021 +- Last updated: April 25, 2021 - Discussion at: N/A ## Table of Contents @@ -239,7 +239,7 @@ No change | Privilege Name | Description | Notes | | --------------- | --------------- | --------------- | | `RESTRICTED_SYSTEM_VARIABLES_ADMIN` | Allows changing a restricted `GLOBAL` system variable. | Currently in SEM all high risk variables are unloaded. TBD, it might be required in future that they are only visible/settable to those with this privilege and not SUPER. | -| `RESTRICTED_STATUS_VARIABLES_ADMIN` | Allows observing restricted status variables. | i.e. `SHOW GLOBAL STATUS` by default hides some statistics when `SEM` is enabled. | +| `RESTRICTED_STATUS_ADMIN` | Allows observing restricted status variables. | i.e. `SHOW GLOBAL STATUS` by default hides some statistics when `SEM` is enabled. | | `RESTRICTED_CONNECTION_ADMIN` | A special privilege to say that their connections, etc. can’t be killed by SUPER users AND they can kill connections by all other users. Affects `KILL`, `KILL TIDB` commands. | It is intended for the CloudAdmin user in DBaaS. | | `RESTRICTED_USER_ADMIN` | A special privilege to say that their access can’t be changed by `SUPER` users. Statements `DROP USER`, `SET PASSWORD`, `ALTER USER`, `REVOKE` are all limited. | It is intended for the CloudAdmin user in DbaaS. | | `RESTRICTED_TABLES_ADMIN` | A special privilege which means that the SEM hidden table semantic doesn’t apply. | It is intended for the CloudAdmin user in DbaaS. | diff --git a/docs/design/2021-03-09-security-enhanced-mode.md b/docs/design/2021-03-09-security-enhanced-mode.md index 98d30f3624557..e939fec67c154 100644 --- a/docs/design/2021-03-09-security-enhanced-mode.md +++ b/docs/design/2021-03-09-security-enhanced-mode.md @@ -1,7 +1,7 @@ # Proposal: - Author(s): [morgo](https://github.com/morgo) -- Last updated: April 12, 2021 +- Last updated: April 25, 2021 - Discussion at: N/A ## Table of Contents @@ -49,7 +49,7 @@ A boolean option called `EnableEnhancedSecurity` (default `FALSE`) will be added ### System Variables -The following system variables will be hidden: +The following system variables will be hidden unless the user has the `RESTRICTED_SYSTEM_VARIABLES_ADMIN` privilege: * variable.TiDBDDLSlowOprThreshold, * variable.TiDBAllowRemoveAutoInc, @@ -78,13 +78,13 @@ The following system variables will be reset to defaults: ### Status Variables -The following status variables will be hidden: +The following status variables will be hidden unless the user has the `RESTRICTED_STATUS_ADMIN` privilege: * tidb_gc_leader_desc ### Information Schema Tables -The following tables will be hidden: +The following tables will be hidden unless the user has the `RESTRICTED_TABLES_ADMIN` privilege: * cluster_config * cluster_hardware @@ -99,7 +99,7 @@ The following tables will be hidden: * metrics_tables * tidb_hot_regions -The following tables will be modified to hide columns: +The following tables will be modified to hide columns unless the user has the `RESTRICTED_TABLES_ADMIN` privilege: * tikv_store_status * The address, capacity, available, start_ts and uptime columns will return NULL. @@ -110,7 +110,7 @@ The following tables will be modified to hide columns: ### Performance Schema Tables -The following tables will be hidden: +The following tables will be hidden unless the user has the `RESTRICTED_TABLES_ADMIN` privilege: * pd_profile_allocs * pd_profile_block @@ -128,7 +128,7 @@ The following tables will be hidden: ### System (mysql) Tables -The following tables will be hidden: +The following tables will be hidden unless the user has the `RESTRICTED_TABLES_ADMIN` privilege: * expr_pushdown_blacklist * gc_delete_range @@ -137,11 +137,11 @@ The following tables will be hidden: * tidb * global_variables -The remaining system tables will be limited to read-only operations. +The remaining system tables will be limited to read-only operations and can not create new tables. ### Metrics Schema -All tables will be hidden, including the schema itself. +All tables will be hidden, including the schema itself unless the user has the `RESTRICTED_TABLES_ADMIN` privilege. ### Commands diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 33db7c9fdd18b..368d8838a777e 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -53,6 +53,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/pdapi" + "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" @@ -132,7 +133,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex case infoschema.TableSessionVar: err = e.setDataFromSessionVar(sctx) case infoschema.TableTiDBServersInfo: - err = e.setDataForServersInfo() + err = e.setDataForServersInfo(sctx) case infoschema.TableTiFlashReplica: e.dataForTableTiFlashReplica(sctx, dbs) case infoschema.TableTiKVStoreStatus: @@ -978,6 +979,18 @@ func (e *memtableRetriever) dataForTiKVStoreStatus(ctx sessionctx.Context) (err lastHeartbeatTs := types.NewTime(types.FromGoTime(storeStat.Status.LastHeartbeatTs), mysql.TypeDatetime, types.DefaultFsp) row[17].SetMysqlTime(lastHeartbeatTs) row[18].SetString(storeStat.Status.Uptime, mysql.DefaultCollationName) + if sem.IsEnabled() { + // Patch out IP addresses etc if the user does not have the RESTRICTED_TABLES_ADMIN privilege + checker := privilege.GetPrivilegeManager(ctx) + if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) { + row[1].SetString(strconv.FormatInt(storeStat.Store.ID, 10), mysql.DefaultCollationName) + row[1].SetNull() + row[6].SetNull() + row[7].SetNull() + row[16].SetNull() + row[18].SetNull() + } + } e.rows = append(e.rows, row) } return nil @@ -1120,6 +1133,15 @@ func (e *memtableRetriever) dataForTiDBClusterInfo(ctx sessionctx.Context) error upTimeStr, server.ServerID, ) + if sem.IsEnabled() { + checker := privilege.GetPrivilegeManager(ctx) + if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) { + row[1].SetString(strconv.FormatUint(server.ServerID, 10), mysql.DefaultCollationName) + row[2].SetNull() + row[5].SetNull() + row[6].SetNull() + } + } rows = append(rows, row) } e.rows = rows @@ -1143,7 +1165,7 @@ func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx sessionctx.Context, sc func (e *memtableRetriever) setDataForClusterProcessList(ctx sessionctx.Context) error { e.setDataForProcessList(ctx) - rows, err := infoschema.AppendHostInfoToRows(e.rows) + rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) if err != nil { return err } @@ -1729,7 +1751,7 @@ func (e *memtableRetriever) setDataForPseudoProfiling(sctx sessionctx.Context) { } } -func (e *memtableRetriever) setDataForServersInfo() error { +func (e *memtableRetriever) setDataForServersInfo(ctx sessionctx.Context) error { serversInfo, err := infosync.GetAllServerInfo(context.Background()) if err != nil { return err @@ -1747,6 +1769,12 @@ func (e *memtableRetriever) setDataForServersInfo() error { info.BinlogStatus, // BINLOG_STATUS stringutil.BuildStringFromLabels(info.Labels), // LABELS ) + if sem.IsEnabled() { + checker := privilege.GetPrivilegeManager(ctx) + if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) { + row[1].SetNull() // clear IP + } + } rows = append(rows, row) } e.rows = rows @@ -1844,7 +1872,7 @@ func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, switch tableName { case infoschema.ClusterTableStatementsSummary, infoschema.ClusterTableStatementsSummaryHistory: - rows, err := infoschema.AppendHostInfoToRows(e.rows) + rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) if err != nil { return err } diff --git a/executor/show.go b/executor/show.go index 0194305f8b913..725c0e0e28ec2 100644 --- a/executor/show.go +++ b/executor/show.go @@ -61,6 +61,7 @@ import ( "github.com/pingcap/tidb/util/format" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/hint" + "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stringutil" @@ -707,10 +708,17 @@ func (e *ShowExec) fetchShowStatus() error { if err != nil { return errors.Trace(err) } + checker := privilege.GetPrivilegeManager(e.ctx) for status, v := range statusVars { if e.GlobalScope && v.Scope == variable.ScopeSession { continue } + // Skip invisible status vars if permission fails. + if sem.IsEnabled() && sem.IsInvisibleStatusVar(status) { + if checker == nil || !checker.RequestDynamicVerification(sessionVars.ActiveRoles, "RESTRICTED_STATUS_ADMIN", false) { + continue + } + } switch v.Value.(type) { case []interface{}, nil: v.Value = fmt.Sprintf("%v", v.Value) diff --git a/executor/slow_query.go b/executor/slow_query.go index c6a5f8872f0e3..9e32cb175e4b8 100755 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -74,7 +74,7 @@ func (e *slowQueryRetriever) retrieve(ctx context.Context, sctx sessionctx.Conte } e.initializeAsyncParsing(ctx, sctx) } - rows, retrieved, err := e.dataForSlowLog(ctx) + rows, retrieved, err := e.dataForSlowLog(ctx, sctx) if err != nil { return nil, err } @@ -193,7 +193,7 @@ func (e *slowQueryRetriever) parseDataForSlowLog(ctx context.Context, sctx sessi e.parseSlowLog(ctx, sctx, reader, ParseSlowLogBatchSize) } -func (e *slowQueryRetriever) dataForSlowLog(ctx context.Context) ([][]types.Datum, bool, error) { +func (e *slowQueryRetriever) dataForSlowLog(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, bool, error) { var ( task slowLogTask ok bool @@ -216,7 +216,7 @@ func (e *slowQueryRetriever) dataForSlowLog(ctx context.Context) ([][]types.Datu continue } if e.table.Name.L == strings.ToLower(infoschema.ClusterTableSlowLog) { - rows, err := infoschema.AppendHostInfoToRows(rows) + rows, err := infoschema.AppendHostInfoToRows(sctx, rows) return rows, false, err } return rows, false, nil diff --git a/infoschema/cluster.go b/infoschema/cluster.go index 2bb3998934815..f113e90a0f587 100644 --- a/infoschema/cluster.go +++ b/infoschema/cluster.go @@ -19,8 +19,11 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/privilege" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/sem" ) // Cluster table list, attention: @@ -78,12 +81,18 @@ func isClusterTableByName(dbName, tableName string) bool { } // AppendHostInfoToRows appends host info to the rows. -func AppendHostInfoToRows(rows [][]types.Datum) ([][]types.Datum, error) { +func AppendHostInfoToRows(ctx sessionctx.Context, rows [][]types.Datum) ([][]types.Datum, error) { serverInfo, err := infosync.GetServerInfo() if err != nil { return nil, err } addr := serverInfo.IP + ":" + strconv.FormatUint(uint64(serverInfo.StatusPort), 10) + if sem.IsEnabled() { + checker := privilege.GetPrivilegeManager(ctx) + if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) { + addr = serverInfo.ID + } + } for i := range rows { row := make([]types.Datum, 0, len(rows[i])+1) row = append(row, types.NewStringDatum(addr)) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index f976e65117138..5b7917e802aac 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -43,7 +43,8 @@ var dynamicPrivs = []string{ "SYSTEM_VARIABLES_ADMIN", "ROLE_ADMIN", "CONNECTION_ADMIN", - "RESTRICTED_TABLES_ADMIN", + "RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled + "RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled. } var dynamicPrivLock sync.Mutex diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 46deaab7ad550..816fe5a59d0bd 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1339,5 +1339,57 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeRestrictedTables(c *C) { mustExec(c, cloudAdminSe, "USE metrics_schema") mustExec(c, cloudAdminSe, "SELECT * FROM metrics_schema.uptime") mustExec(c, cloudAdminSe, "CREATE TABLE mysql.abcd (a int)") +} + +func (s *testPrivilegeSuite) TestSecurityEnhancedModeInfoschema(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE USER uroot1, uroot2, uroot3") + tk.MustExec("GRANT SUPER ON *.* to uroot1 WITH GRANT OPTION") // super not process + tk.MustExec("SET tidb_enable_dynamic_privileges=1") + tk.MustExec("GRANT SUPER, PROCESS, RESTRICTED_TABLES_ADMIN ON *.* to uroot2 WITH GRANT OPTION") + tk.Se.Auth(&auth.UserIdentity{ + Username: "uroot1", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + sem.Enable() + defer sem.Disable() + + // Even though we have super, we still can't read protected information from tidb_servers_info, cluster_* tables + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.tidb_servers_info WHERE ip IS NOT NULL`).Check(testkit.Rows("0")) + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.cluster_info WHERE status_address IS NOT NULL`).Check(testkit.Rows("0")) + // 36 = a UUID. Normally it is an IP address. + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.CLUSTER_STATEMENTS_SUMMARY WHERE length(instance) != 36`).Check(testkit.Rows("0")) + + // That is unless we have the RESTRICTED_TABLES_ADMIN privilege + tk.Se.Auth(&auth.UserIdentity{ + Username: "uroot2", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + + // flip from is NOT NULL etc + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.tidb_servers_info WHERE ip IS NULL`).Check(testkit.Rows("0")) + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.cluster_info WHERE status_address IS NULL`).Check(testkit.Rows("0")) + tk.MustQuery(`SELECT COUNT(*) FROM information_schema.CLUSTER_STATEMENTS_SUMMARY WHERE length(instance) = 36`).Check(testkit.Rows("0")) +} + +func (s *testPrivilegeSuite) TestSecurityEnhancedModeStatusVars(c *C) { + // Without TiKV the status var list does not include tidb_gc_leader_desc + // So we can only test that the dynamic privilege is grantable. + // We will have to use an integration test to run SHOW STATUS LIKE 'tidb_gc_leader_desc' + // and verify if it appears. + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE USER unostatus, ustatus") + tk.MustExec("SET tidb_enable_dynamic_privileges=1") + tk.MustExec("GRANT RESTRICTED_STATUS_ADMIN ON *.* to ustatus") + tk.Se.Auth(&auth.UserIdentity{ + Username: "unostatus", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) } diff --git a/util/sem/sem.go b/util/sem/sem.go index 0dde650b9d765..8c3d2b456d991 100644 --- a/util/sem/sem.go +++ b/util/sem/sem.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/logutil" ) const ( @@ -56,6 +57,7 @@ const ( tidbProfileMemory = "tidb_profile_memory" tidbProfileMutex = "tidb_profile_mutex" tikvProfileCPU = "tikv_profile_cpu" + tidbGCLeaderDesc = "tidb_gc_leader_desc" restrictedPriv = "RESTRICTED_" ) @@ -68,6 +70,8 @@ var ( func Enable() { atomic.StoreInt32(&semEnabled, 1) variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.On) + // write to log so users understand why some operations are weird. + logutil.BgLogger().Info("tidb-server is operating with security enhanced mode (SEM) enabled") } // Disable disables SEM. This is intended to be used by the test-suite. @@ -116,6 +120,11 @@ func IsInvisibleTable(dbLowerName, tblLowerName string) bool { return false } +// IsInvisibleStatusVar returns true if the status var needs to be hidden +func IsInvisibleStatusVar(varName string) bool { + return varName == tidbGCLeaderDesc +} + // IsRestrictedPrivilege returns true if the privilege shuld not be satisfied by SUPER // As most dynamic privileges are. func IsRestrictedPrivilege(privNameInUpper string) bool { diff --git a/util/sem/sem_test.go b/util/sem/sem_test.go index d22f9e52b5e57..c303d2195c7f4 100644 --- a/util/sem/sem_test.go +++ b/util/sem/sem_test.go @@ -67,3 +67,10 @@ func (s *testSecurity) TestIsRestrictedPrivilege(c *C) { c.Assert(IsRestrictedPrivilege("BACKUP_ADMIN"), IsFalse) c.Assert(IsRestrictedPrivilege("aa"), IsFalse) } + +func (s *testSecurity) TestIsInvisibleStatusVar(c *C) { + c.Assert(IsInvisibleStatusVar(tidbGCLeaderDesc), IsTrue) + c.Assert(IsInvisibleStatusVar("server_id"), IsFalse) + c.Assert(IsInvisibleStatusVar("ddl_schema_version"), IsFalse) + c.Assert(IsInvisibleStatusVar("Ssl_version"), IsFalse) +} From c506155130eee8e55eed44a8707666e25ab7014d Mon Sep 17 00:00:00 2001 From: Shirly Date: Fri, 30 Apr 2021 09:29:49 +0800 Subject: [PATCH 30/36] store/tikv: make tikv.ErrTiKVServerTimeout as a normal error (#24367) --- store/driver/txn/error.go | 12 ++++++++++++ store/tikv/error/errcode.go | 1 - store/tikv/error/error.go | 3 ++- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 2c2ef4bc60d3b..33d30d635bc7c 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -25,15 +25,23 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/dbterror" "go.uber.org/zap" ) +// tikv error instance +var ( + // ErrTiKVServerTimeout is the error when tikv server is timeout. + ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) +) + func genKeyExistsError(name string, value string, err error) error { if err != nil { logutil.BgLogger().Info("extractKeyExistsErr meets error", zap.Error(err)) @@ -176,6 +184,10 @@ func toTiDBErr(err error) error { return kv.ErrInvalidTxn } + if errors.ErrorEqual(err, tikverr.ErrTiKVServerTimeout) { + return ErrTiKVServerTimeout + } + return errors.Trace(err) } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index e84c4577c4058..71d97e76088b8 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -26,7 +26,6 @@ const ( // TiKV/PD/TiFlash errors. CodePDServerTimeout = 9001 - CodeTiKVServerTimeout = 9002 CodeTiKVServerBusy = 9003 CodeResolveLockTimeout = 9004 CodeRegionUnavailable = 9005 diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index db5be4ebc3666..417e805f5ef3d 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -33,6 +33,8 @@ var ( ErrCannotSetNilValue = errors.New("can not set nil value") // ErrInvalidTxn is the error when commits or rollbacks in an invalid transaction. ErrInvalidTxn = errors.New("invalid transaction") + // ErrTiKVServerTimeout is the error when tikv server is timeout. + ErrTiKVServerTimeout = errors.New("tikv server timeout") ) // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. @@ -40,7 +42,6 @@ const MismatchClusterID = "mismatch cluster id" // error instances. var ( - ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiKVServerTimeout) ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(CodeResolveLockTimeout) ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(CodePDServerTimeout) From 8ff80f12b72072d82344d4738f89fb09ed7311cf Mon Sep 17 00:00:00 2001 From: disksing Date: Fri, 30 Apr 2021 10:47:50 +0800 Subject: [PATCH 31/36] store/tikv: remove use of InfoSchema transaction option in store/tikv (#24364) --- store/driver/txn/txn_driver.go | 2 ++ store/tikv/2pc.go | 6 +++--- store/tikv/txn.go | 11 +++++++---- 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 968d0f8701233..cca5aaac085ce 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -144,6 +144,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetPessimistic(val.(bool)) case tikvstore.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) + case tikvstore.InfoSchema: + txn.SetSchemaVer(val.(tikv.SchemaVer)) case tikvstore.CommitHook: txn.SetCommitCallback(val.(func(string, error))) default: diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index dbe74dd844bf0..f32c4a99283ce 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -1114,12 +1114,12 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { if !c.isAsyncCommit() { tryAmend := c.isPessimistic && c.sessionID > 0 && c.txn.schemaAmender != nil if !tryAmend { - _, _, err = c.checkSchemaValid(ctx, commitTS, c.txn.txnInfoSchema, false) + _, _, err = c.checkSchemaValid(ctx, commitTS, c.txn.schemaVer, false) if err != nil { return errors.Trace(err) } } else { - relatedSchemaChange, memAmended, err := c.checkSchemaValid(ctx, commitTS, c.txn.txnInfoSchema, true) + relatedSchemaChange, memAmended, err := c.checkSchemaValid(ctx, commitTS, c.txn.schemaVer, true) if err != nil { return errors.Trace(err) } @@ -1431,7 +1431,7 @@ func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64 func (c *twoPhaseCommitter) calculateMaxCommitTS(ctx context.Context) error { // Amend txn with current time first, then we can make sure we have another SafeWindow time to commit currentTS := oracle.EncodeTSO(int64(time.Since(c.txn.startTime)/time.Millisecond)) + c.startTS - _, _, err := c.checkSchemaValid(ctx, currentTS, c.txn.txnInfoSchema, true) + _, _, err := c.checkSchemaValid(ctx, currentTS, c.txn.schemaVer, true) if err != nil { logutil.Logger(ctx).Info("Schema changed for async commit txn", zap.Error(err), diff --git a/store/tikv/txn.go b/store/tikv/txn.go index de0d607f5f55a..00dfcee7b44e4 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -66,8 +66,8 @@ type KVTxn struct { valid bool - // txnInfoSchema is the infoSchema fetched at startTS. - txnInfoSchema SchemaVer + // schemaVer is the infoSchema fetched at startTS. + schemaVer SchemaVer // SchemaAmender is used amend pessimistic txn commit mutations for schema change schemaAmender SchemaAmender // commitCallback is called after current transaction gets committed @@ -183,8 +183,6 @@ func (txn *KVTxn) SetOption(opt int, val interface{}) { txn.us.SetOption(opt, val) txn.snapshot.SetOption(opt, val) switch opt { - case kv.InfoSchema: - txn.txnInfoSchema = val.(SchemaVer) case kv.SchemaAmender: txn.schemaAmender = val.(SchemaAmender) } @@ -215,6 +213,11 @@ func (txn *KVTxn) SetPessimistic(b bool) { txn.isPessimistic = b } +// SetSchemaVer updates schema version to validate transaction. +func (txn *KVTxn) SetSchemaVer(schemaVer SchemaVer) { + txn.schemaVer = schemaVer +} + // SetPriority sets the priority for both write and read. func (txn *KVTxn) SetPriority(pri Priority) { txn.priority = pri From a9a5e057e9ab2cf330a849657751ecb8c712cc73 Mon Sep 17 00:00:00 2001 From: disksing Date: Fri, 30 Apr 2021 11:15:50 +0800 Subject: [PATCH 32/36] store/tikv: remove use of TxnScope transaction option in store/tikv (#24365) --- executor/point_get.go | 2 +- session/session.go | 2 +- store/driver/txn/txn_driver.go | 11 +++++++++++ store/tikv/2pc.go | 10 ++++------ store/tikv/commit.go | 3 +-- store/tikv/snapshot.go | 3 --- store/tikv/tests/1pc_test.go | 4 ++-- store/tikv/tests/async_commit_test.go | 4 ++-- store/tikv/txn.go | 20 +++++++++++++++----- 9 files changed, 37 insertions(+), 22 deletions(-) diff --git a/executor/point_get.go b/executor/point_get.go index b4ec8d13bfe72..241f52d421344 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -391,7 +391,7 @@ func (e *PointGetExecutor) get(ctx context.Context, key kv.Key) ([]byte, error) } func (e *PointGetExecutor) verifyTxnScope() error { - txnScope := e.txn.GetUnionStore().GetOption(tikvstore.TxnScope).(string) + txnScope := e.txn.GetOption(tikvstore.TxnScope).(string) if txnScope == "" || txnScope == oracle.GlobalTxnScope { return nil } diff --git a/session/session.go b/session/session.go index 67354cd15e494..8ca49b6a9e79d 100644 --- a/session/session.go +++ b/session/session.go @@ -2734,7 +2734,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionctx.StalenessTxnOption) error { if s.txn.Valid() { txnID := s.txn.StartTS() - txnScope := s.txn.GetUnionStore().GetOption(tikvstore.TxnScope).(string) + txnScope := s.txn.GetOption(tikvstore.TxnScope).(string) err := s.CommitTxn(ctx) if err != nil { return err diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index cca5aaac085ce..d368962adb67e 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -148,11 +148,22 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetSchemaVer(val.(tikv.SchemaVer)) case tikvstore.CommitHook: txn.SetCommitCallback(val.(func(string, error))) + case tikvstore.TxnScope: + txn.SetScope(val.(string)) default: txn.KVTxn.SetOption(opt, val) } } +func (txn *tikvTxn) GetOption(opt int) interface{} { + switch opt { + case tikvstore.TxnScope: + return txn.KVTxn.GetScope() + default: + return txn.KVTxn.GetOption(opt) + } +} + // SetVars sets variables to the transaction. func (txn *tikvTxn) SetVars(vars interface{}) { if vs, ok := vars.(*tikv.Variables); ok { diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index f32c4a99283ce..cae15bc7aab4e 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -821,8 +821,7 @@ func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, tt // checkAsyncCommit checks if async commit protocol is available for current transaction commit, true is returned if possible. func (c *twoPhaseCommitter) checkAsyncCommit() bool { // Disable async commit in local transactions - txnScopeOption := c.txn.us.GetOption(kv.TxnScope) - if txnScopeOption == nil || txnScopeOption.(string) != oracle.GlobalTxnScope { + if c.txn.GetScope() != oracle.GlobalTxnScope { return false } @@ -849,8 +848,7 @@ func (c *twoPhaseCommitter) checkAsyncCommit() bool { // checkOnePC checks if 1PC protocol is available for current transaction. func (c *twoPhaseCommitter) checkOnePC() bool { // Disable 1PC in local transactions - txnScopeOption := c.txn.us.GetOption(kv.TxnScope) - if txnScopeOption == nil || txnScopeOption.(string) != oracle.GlobalTxnScope { + if c.txn.GetScope() != oracle.GlobalTxnScope { return false } @@ -1099,7 +1097,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { } else { start = time.Now() logutil.Event(ctx, "start get commit ts") - commitTS, err = c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetUnionStore().GetOption(kv.TxnScope).(string)) + commitTS, err = c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) if err != nil { logutil.Logger(ctx).Warn("2PC get commitTS failed", zap.Error(err), @@ -1368,7 +1366,7 @@ func (c *twoPhaseCommitter) tryAmendTxn(ctx context.Context, startInfoSchema Sch func (c *twoPhaseCommitter) getCommitTS(ctx context.Context, commitDetail *util.CommitDetails) (uint64, error) { start := time.Now() logutil.Event(ctx, "start get commit ts") - commitTS, err := c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetUnionStore().GetOption(kv.TxnScope).(string)) + commitTS, err := c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) if err != nil { logutil.Logger(ctx).Warn("2PC get commitTS failed", zap.Error(err), diff --git a/store/tikv/commit.go b/store/tikv/commit.go index 73406ff19f4c1..ce9df6a927355 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -100,7 +99,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch } // Update commit ts and retry. - commitTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetUnionStore().GetOption(kv.TxnScope).(string)) + commitTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) if err != nil { logutil.Logger(bo.ctx).Warn("2PC get commitTS failed", zap.Error(err), diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 0b09a408d7dc9..2b9926c7a2b9a 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -107,7 +107,6 @@ type KVSnapshot struct { matchStoreLabels []*metapb.StoreLabel } sampleStep uint32 - txnScope string } // newTiKVSnapshot creates a snapshot of an TiKV store. @@ -588,8 +587,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.mu.Lock() s.mu.matchStoreLabels = val.([]*metapb.StoreLabel) s.mu.Unlock() - case kv.TxnScope: - s.txnScope = val.(string) } } diff --git a/store/tikv/tests/1pc_test.go b/store/tikv/tests/1pc_test.go index 509dab684628a..a7778294f1d3e 100644 --- a/store/tikv/tests/1pc_test.go +++ b/store/tikv/tests/1pc_test.go @@ -254,7 +254,7 @@ func (s *testOnePCSuite) Test1PCWithMultiDC(c *C) { localTxn := s.begin1PC(c) err := localTxn.Set([]byte("a"), []byte("a1")) - localTxn.SetOption(kv.TxnScope, "bj") + localTxn.SetScope("bj") c.Assert(err, IsNil) ctx := context.WithValue(context.Background(), util.SessionID, uint64(1)) err = localTxn.Commit(ctx) @@ -263,7 +263,7 @@ func (s *testOnePCSuite) Test1PCWithMultiDC(c *C) { globalTxn := s.begin1PC(c) err = globalTxn.Set([]byte("b"), []byte("b1")) - globalTxn.SetOption(kv.TxnScope, oracle.GlobalTxnScope) + globalTxn.SetScope(oracle.GlobalTxnScope) c.Assert(err, IsNil) err = globalTxn.Commit(ctx) c.Assert(err, IsNil) diff --git a/store/tikv/tests/async_commit_test.go b/store/tikv/tests/async_commit_test.go index 99e82add2e93d..ea171920e52bf 100644 --- a/store/tikv/tests/async_commit_test.go +++ b/store/tikv/tests/async_commit_test.go @@ -409,7 +409,7 @@ func (s *testAsyncCommitSuite) TestAsyncCommitWithMultiDC(c *C) { localTxn := s.beginAsyncCommit(c) err := localTxn.Set([]byte("a"), []byte("a1")) - localTxn.SetOption(kv.TxnScope, "bj") + localTxn.SetScope("bj") c.Assert(err, IsNil) ctx := context.WithValue(context.Background(), util.SessionID, uint64(1)) err = localTxn.Commit(ctx) @@ -418,7 +418,7 @@ func (s *testAsyncCommitSuite) TestAsyncCommitWithMultiDC(c *C) { globalTxn := s.beginAsyncCommit(c) err = globalTxn.Set([]byte("b"), []byte("b1")) - globalTxn.SetOption(kv.TxnScope, oracle.GlobalTxnScope) + globalTxn.SetScope(oracle.GlobalTxnScope) c.Assert(err, IsNil) err = globalTxn.Commit(ctx) c.Assert(err, IsNil) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 00dfcee7b44e4..b064758cc9b88 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -78,6 +78,7 @@ type KVTxn struct { syncLog bool priority Priority isPessimistic bool + scope string kvFilter KVFilter } @@ -93,7 +94,7 @@ func newTiKVTxn(store *KVStore, txnScope string) (*KVTxn, error) { // newTiKVTxnWithStartTS creates a txn with startTS. func newTiKVTxnWithStartTS(store *KVStore, txnScope string, startTS uint64, replicaReadSeed uint32) (*KVTxn, error) { snapshot := newTiKVSnapshot(store, startTS, replicaReadSeed) - newTiKVTxn := &KVTxn{ + return &KVTxn{ snapshot: snapshot, us: unionstore.NewUnionStore(snapshot), store: store, @@ -101,9 +102,8 @@ func newTiKVTxnWithStartTS(store *KVStore, txnScope string, startTS uint64, repl startTime: time.Now(), valid: true, vars: kv.DefaultVars, - } - newTiKVTxn.SetOption(kv.TxnScope, txnScope) - return newTiKVTxn, nil + scope: txnScope, + }, nil } func newTiKVTxnWithExactStaleness(store *KVStore, txnScope string, prevSec uint64) (*KVTxn, error) { @@ -230,6 +230,11 @@ func (txn *KVTxn) SetCommitCallback(f func(string, error)) { txn.commitCallback = f } +// SetScope sets the geographical scope of the transaction. +func (txn *KVTxn) SetScope(scope string) { + txn.scope = scope +} + // SetKVFilter sets the filter to ignore key-values in memory buffer. func (txn *KVTxn) SetKVFilter(filter KVFilter) { txn.kvFilter = filter @@ -240,6 +245,11 @@ func (txn *KVTxn) IsPessimistic() bool { return txn.isPessimistic } +// GetScope returns the geographical scope of the transaction. +func (txn *KVTxn) GetScope() string { + return txn.scope +} + // Commit commits the transaction operations to KV store. func (txn *KVTxn) Commit(ctx context.Context) error { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { @@ -409,7 +419,7 @@ func (txn *KVTxn) onCommitted(err error) { } info := TxnInfo{ - TxnScope: txn.GetUnionStore().GetOption(kv.TxnScope).(string), + TxnScope: txn.GetScope(), StartTS: txn.startTS, CommitTS: txn.commitTS, TxnCommitMode: commitMode, From 1342f3a054b747c85c9303781cf93d82f899a33f Mon Sep 17 00:00:00 2001 From: disksing Date: Fri, 30 Apr 2021 11:47:49 +0800 Subject: [PATCH 33/36] store/tikv: remove use of Enable1PC transaction option in store/tikv (#24366) --- store/driver/txn/txn_driver.go | 2 ++ store/tikv/2pc.go | 3 +-- store/tikv/tests/1pc_test.go | 3 +-- store/tikv/tests/snapshot_fail_test.go | 2 +- store/tikv/txn.go | 6 ++++++ 5 files changed, 11 insertions(+), 5 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index d368962adb67e..3b5435d596c77 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -148,6 +148,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetSchemaVer(val.(tikv.SchemaVer)) case tikvstore.CommitHook: txn.SetCommitCallback(val.(func(string, error))) + case tikvstore.Enable1PC: + txn.SetEnable1PC(val.(bool)) case tikvstore.TxnScope: txn.SetScope(val.(string)) default: diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index cae15bc7aab4e..8703b1861c65d 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -852,8 +852,7 @@ func (c *twoPhaseCommitter) checkOnePC() bool { return false } - enable1PCOption := c.txn.us.GetOption(kv.Enable1PC) - return c.sessionID > 0 && !c.shouldWriteBinlog() && enable1PCOption != nil && enable1PCOption.(bool) + return c.sessionID > 0 && !c.shouldWriteBinlog() && c.txn.enable1PC } func (c *twoPhaseCommitter) needLinearizability() bool { diff --git a/store/tikv/tests/1pc_test.go b/store/tikv/tests/1pc_test.go index a7778294f1d3e..6ff57700f4e3c 100644 --- a/store/tikv/tests/1pc_test.go +++ b/store/tikv/tests/1pc_test.go @@ -18,7 +18,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/util" @@ -27,7 +26,7 @@ import ( func (s *testAsyncCommitCommon) begin1PC(c *C) tikv.TxnProbe { txn, err := s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.Enable1PC, true) + txn.SetEnable1PC(true) return tikv.TxnProbe{KVTxn: txn} } diff --git a/store/tikv/tests/snapshot_fail_test.go b/store/tikv/tests/snapshot_fail_test.go index 572f6cae884e2..1360841bd743a 100644 --- a/store/tikv/tests/snapshot_fail_test.go +++ b/store/tikv/tests/snapshot_fail_test.go @@ -211,7 +211,7 @@ func (s *testSnapshotFailSuite) TestRetryPointGetResolveTS(c *C) { err = txn.Set([]byte("k2"), []byte("v2")) c.Assert(err, IsNil) txn.SetOption(kv.EnableAsyncCommit, false) - txn.SetOption(kv.Enable1PC, false) + txn.SetEnable1PC(false) txn.SetOption(kv.GuaranteeLinearizability, false) // Prewrite the lock without committing it diff --git a/store/tikv/txn.go b/store/tikv/txn.go index b064758cc9b88..4e462653c415c 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -78,6 +78,7 @@ type KVTxn struct { syncLog bool priority Priority isPessimistic bool + enable1PC bool scope string kvFilter KVFilter } @@ -230,6 +231,11 @@ func (txn *KVTxn) SetCommitCallback(f func(string, error)) { txn.commitCallback = f } +// SetEnable1PC indicates if the transaction will try to use 1 phase commit. +func (txn *KVTxn) SetEnable1PC(b bool) { + txn.enable1PC = b +} + // SetScope sets the geographical scope of the transaction. func (txn *KVTxn) SetScope(scope string) { txn.scope = scope From b77338d012ff2801b13f3c63361f591e9eec4160 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Fri, 30 Apr 2021 13:29:50 +0800 Subject: [PATCH 34/36] store/tikv: not setting special ts when resolving async locks (#22723) --- go.sum | 1 - store/tikv/lock_resolver.go | 8 +-- store/tikv/test_probe.go | 7 +++ store/tikv/tests/async_commit_test.go | 89 +++++++++++++++++++++++++++ 4 files changed, 98 insertions(+), 7 deletions(-) diff --git a/go.sum b/go.sum index f3396ccd03898..d4863b5d8b369 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= diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index 8ee1cd411c13f..05feee6d31adb 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -485,12 +485,8 @@ func (lr *LockResolver) getTxnStatusFromLock(bo *Backoffer, l *Lock, callerStart var currentTS uint64 var err error var status TxnStatus - if l.UseAsyncCommit && !forceSyncCommit { - // Async commit doesn't need the current ts since it uses the minCommitTS. - currentTS = 0 - // Set to 0 so as not to push forward min commit ts. - callerStartTS = 0 - } else if l.TTL == 0 { + + if l.TTL == 0 { // NOTE: l.TTL = 0 is a special protocol!!! // When the pessimistic txn prewrite meets locks of a txn, it should resolve the lock **unconditionally**. // In this case, TiKV use lock TTL = 0 to notify TiDB, and TiDB should resolve the lock! diff --git a/store/tikv/test_probe.go b/store/tikv/test_probe.go index e2b072ef3c7f1..a6a4f8d826655 100644 --- a/store/tikv/test_probe.go +++ b/store/tikv/test_probe.go @@ -14,6 +14,7 @@ package tikv import ( + "bytes" "context" "sync/atomic" "time" @@ -304,6 +305,12 @@ func (c CommitterProbe) BuildPrewriteRequest(regionID, regionConf, regionVersion var batch batchMutations batch.mutations = mutations batch.region = RegionVerID{regionID, regionConf, regionVersion} + for _, key := range mutations.GetKeys() { + if bytes.Equal(key, c.primary()) { + batch.isPrimary = true + break + } + } return c.buildPrewriteRequest(batch, txnSize) } diff --git a/store/tikv/tests/async_commit_test.go b/store/tikv/tests/async_commit_test.go index ea171920e52bf..0f4985fa7ab86 100644 --- a/store/tikv/tests/async_commit_test.go +++ b/store/tikv/tests/async_commit_test.go @@ -425,6 +425,95 @@ func (s *testAsyncCommitSuite) TestAsyncCommitWithMultiDC(c *C) { c.Assert(globalTxn.IsAsyncCommit(), IsTrue) } +func (s *testAsyncCommitSuite) TestResolveTxnFallbackFromAsyncCommit(c *C) { + keys := [][]byte{[]byte("k0"), []byte("k1")} + values := [][]byte{[]byte("v00"), []byte("v10")} + initTest := func() tikv.CommitterProbe { + t0 := s.begin(c) + err := t0.Set(keys[0], values[0]) + c.Assert(err, IsNil) + err = t0.Set(keys[1], values[1]) + c.Assert(err, IsNil) + err = t0.Commit(context.Background()) + c.Assert(err, IsNil) + + t1 := s.beginAsyncCommit(c) + err = t1.Set(keys[0], []byte("v01")) + c.Assert(err, IsNil) + err = t1.Set(keys[1], []byte("v11")) + c.Assert(err, IsNil) + + committer, err := t1.NewCommitter(1) + c.Assert(err, IsNil) + committer.SetLockTTL(1) + committer.SetUseAsyncCommit() + return committer + } + prewriteKey := func(committer tikv.CommitterProbe, idx int, fallback bool) { + bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil) + loc, err := s.store.GetRegionCache().LocateKey(bo, keys[idx]) + c.Assert(err, IsNil) + req := committer.BuildPrewriteRequest(loc.Region.GetID(), loc.Region.GetConfVer(), loc.Region.GetVer(), + committer.GetMutations().Slice(idx, idx+1), 1) + if fallback { + req.Req.(*kvrpcpb.PrewriteRequest).MaxCommitTs = 1 + } + resp, err := s.store.SendReq(bo, req, loc.Region, 5000) + c.Assert(err, IsNil) + c.Assert(resp.Resp, NotNil) + } + readKey := func(idx int) { + t2 := s.begin(c) + ctx, cancel := context.WithTimeout(context.Background(), time.Second) + defer cancel() + val, err := t2.Get(ctx, keys[idx]) + c.Assert(err, IsNil) + c.Assert(val, DeepEquals, values[idx]) + } + + // Case 1: Fallback primary, read primary + committer := initTest() + prewriteKey(committer, 0, true) + prewriteKey(committer, 1, false) + readKey(0) + readKey(1) + + // Case 2: Fallback primary, read secondary + committer = initTest() + prewriteKey(committer, 0, true) + prewriteKey(committer, 1, false) + readKey(1) + readKey(0) + + // Case 3: Fallback secondary, read primary + committer = initTest() + prewriteKey(committer, 0, false) + prewriteKey(committer, 1, true) + readKey(0) + readKey(1) + + // Case 4: Fallback secondary, read secondary + committer = initTest() + prewriteKey(committer, 0, false) + prewriteKey(committer, 1, true) + readKey(1) + readKey(0) + + // Case 5: Fallback both, read primary + committer = initTest() + prewriteKey(committer, 0, true) + prewriteKey(committer, 1, true) + readKey(0) + readKey(1) + + // Case 6: Fallback both, read secondary + committer = initTest() + prewriteKey(committer, 0, true) + prewriteKey(committer, 1, true) + readKey(1) + readKey(0) +} + type mockResolveClient struct { inner tikv.Client onResolveLock func(*kvrpcpb.ResolveLockRequest) (*tikvrpc.Response, error) From d16d25d5d2e72e09dde9ad918c96f0ec3c9085ec Mon Sep 17 00:00:00 2001 From: Shirly Date: Fri, 30 Apr 2021 15:21:50 +0800 Subject: [PATCH 35/36] =?UTF-8?q?store/tikv:=20make=20tikv.ErrResolveLockT?= =?UTF-8?q?imeout=20as=20a=20normal=20error=20instead=E2=80=A6=20(#24386)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- store/driver/txn/error.go | 6 +++++- store/tikv/error/errcode.go | 9 ++++----- store/tikv/error/error.go | 3 ++- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 33d30d635bc7c..bd1ecc2c9aa2b 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -39,7 +39,8 @@ import ( // tikv error instance var ( // ErrTiKVServerTimeout is the error when tikv server is timeout. - ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) + ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) + ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) ) func genKeyExistsError(name string, value string, err error) error { @@ -188,6 +189,9 @@ func toTiDBErr(err error) error { return ErrTiKVServerTimeout } + if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { + return ErrResolveLockTimeout + } return errors.Trace(err) } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 71d97e76088b8..192d2163b3b62 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -25,11 +25,10 @@ const ( CodeLockAcquireFailAndNoWaitSet = 3572 // TiKV/PD/TiFlash errors. - CodePDServerTimeout = 9001 - CodeTiKVServerBusy = 9003 - CodeResolveLockTimeout = 9004 - CodeRegionUnavailable = 9005 - CodeGCTooEarly = 9006 + CodePDServerTimeout = 9001 + CodeTiKVServerBusy = 9003 + CodeRegionUnavailable = 9005 + CodeGCTooEarly = 9006 CodeTiKVStoreLimit = 9008 diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 417e805f5ef3d..f623c9f0f7491 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -35,6 +35,8 @@ var ( ErrInvalidTxn = errors.New("invalid transaction") // ErrTiKVServerTimeout is the error when tikv server is timeout. ErrTiKVServerTimeout = errors.New("tikv server timeout") + // ErrResolveLockTimeout is the error that resolve lock timeout. + ErrResolveLockTimeout = errors.New("resolve lock timeout") ) // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. @@ -43,7 +45,6 @@ const MismatchClusterID = "mismatch cluster id" // error instances. var ( ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(CodeTiFlashServerTimeout) - ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(CodeResolveLockTimeout) ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(CodePDServerTimeout) ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(CodeRegionUnavailable) ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(CodeTiKVServerBusy) From 27cacd8caf64b7382b918fd02c82f61b95ba54e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BD=99=E6=9E=97=E9=A2=96?= Date: Fri, 30 Apr 2021 17:01:50 +0800 Subject: [PATCH 36/36] ddl: fix the bug of maximum length limit of identifier #24288 (#24337) --- ddl/db_test.go | 31 +++++++++++++++++++++++++++++++ ddl/ddl_api.go | 13 +++++++------ 2 files changed, 38 insertions(+), 6 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 7970b14989aaf..e865de39d3248 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -77,6 +77,7 @@ var _ = Suite(&testDBSuite5{&testDBSuite{}}) var _ = Suite(&testDBSuite6{&testDBSuite{}}) var _ = Suite(&testDBSuite7{&testDBSuite{}}) var _ = SerialSuites(&testSerialDBSuite{&testDBSuite{}}) +var _ = Suite(&testDBSuite8{&testDBSuite{}}) const defaultBatchSize = 1024 const defaultReorgBatchSize = 256 @@ -145,6 +146,7 @@ type testDBSuite5 struct{ *testDBSuite } type testDBSuite6 struct{ *testDBSuite } type testDBSuite7 struct{ *testDBSuite } type testSerialDBSuite struct{ *testDBSuite } +type testDBSuite8 struct{ *testDBSuite } func testAddIndexWithPK(tk *testkit.TestKit) { tk.MustExec("drop table if exists test_add_index_with_pk") @@ -6700,3 +6702,32 @@ func (s *testSerialDBSuite) TestJsonUnmarshalErrWhenPanicInCancellingPath(c *C) _, err := tk.Exec("alter table test_add_index_after_add_col add unique index cc(c);") c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '0' for key 'cc'") } + +// For Close issue #24288 +// see https://github.com/pingcap/tidb/issues/24288 +func (s *testDBSuite8) TestDdlMaxLimitOfIdentifier(c *C) { + tk := testkit.NewTestKit(c, s.store) + + // create/drop database test + longDbName := strings.Repeat("库", mysql.MaxDatabaseNameLength-1) + tk.MustExec(fmt.Sprintf("create database %s", longDbName)) + defer func() { + tk.MustExec(fmt.Sprintf("drop database %s", longDbName)) + }() + tk.MustExec(fmt.Sprintf("use %s", longDbName)) + + // create/drop table,index test + longTblName := strings.Repeat("表", mysql.MaxTableNameLength-1) + longColName := strings.Repeat("三", mysql.MaxColumnNameLength-1) + longIdxName := strings.Repeat("索", mysql.MaxIndexIdentifierLen-1) + tk.MustExec(fmt.Sprintf("create table %s(f1 int primary key,f2 int, %s varchar(50))", longTblName, longColName)) + tk.MustExec(fmt.Sprintf("create index %s on %s(%s)", longIdxName, longTblName, longColName)) + defer func() { + tk.MustExec(fmt.Sprintf("drop index %s on %s", longIdxName, longTblName)) + tk.MustExec(fmt.Sprintf("drop table %s", longTblName)) + }() + + // alter table + tk.MustExec(fmt.Sprintf("alter table %s change f2 %s int", longTblName, strings.Repeat("二", mysql.MaxColumnNameLength-1))) + +} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 06e43e905d616..67b51f7a3750e 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -25,6 +25,7 @@ import ( "strings" "sync/atomic" "time" + "unicode/utf8" "github.com/cznic/mathutil" "github.com/go-yaml/yaml" @@ -225,21 +226,21 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) } func checkTooLongSchema(schema model.CIStr) error { - if len(schema.L) > mysql.MaxDatabaseNameLength { + if utf8.RuneCountInString(schema.L) > mysql.MaxDatabaseNameLength { return ErrTooLongIdent.GenWithStackByArgs(schema) } return nil } func checkTooLongTable(table model.CIStr) error { - if len(table.L) > mysql.MaxTableNameLength { + if utf8.RuneCountInString(table.L) > mysql.MaxTableNameLength { return ErrTooLongIdent.GenWithStackByArgs(table) } return nil } func checkTooLongIndex(index model.CIStr) error { - if len(index.L) > mysql.MaxIndexIdentifierLen { + if utf8.RuneCountInString(index.L) > mysql.MaxIndexIdentifierLen { return ErrTooLongIdent.GenWithStackByArgs(index) } return nil @@ -1107,7 +1108,7 @@ func checkGeneratedColumn(colDefs []*ast.ColumnDef) error { func checkTooLongColumn(cols []*model.ColumnInfo) error { for _, col := range cols { colName := col.Name.O - if len(colName) > mysql.MaxColumnNameLength { + if utf8.RuneCountInString(colName) > mysql.MaxColumnNameLength { return ErrTooLongIdent.GenWithStackByArgs(colName) } } @@ -2722,7 +2723,7 @@ func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model if err = checkColumnAttributes(colName, specNewColumn.Tp); err != nil { return nil, errors.Trace(err) } - if len(colName) > mysql.MaxColumnNameLength { + if utf8.RuneCountInString(colName) > mysql.MaxColumnNameLength { return nil, ErrTooLongIdent.GenWithStackByArgs(colName) } @@ -4988,7 +4989,7 @@ func buildHiddenColumnInfo(ctx sessionctx.Context, indexPartSpecifications []*as } idxPart.Length = types.UnspecifiedLength // The index part is an expression, prepare a hidden column for it. - if len(idxPart.Column.Name.L) > mysql.MaxColumnNameLength { + if utf8.RuneCountInString(idxPart.Column.Name.L) > mysql.MaxColumnNameLength { // TODO: Refine the error message. return nil, ErrTooLongIdent.GenWithStackByArgs("hidden column") }