From 9122e319e907048663825b9fa97d549c5c130864 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Sat, 7 Mar 2020 18:48:19 +0800 Subject: [PATCH 01/61] support batch cop for tiflash --- go.mod | 2 +- go.sum | 4 + kv/kv.go | 2 + sessionctx/variable/tidb_vars.go | 4 + store/tikv/batch_coprocessor.go | 438 +++++++++++++++++++++++++++++++ store/tikv/coprocessor.go | 3 + store/tikv/region_request.go | 46 ++++ store/tikv/tikvrpc/tikvrpc.go | 28 ++ 8 files changed, 526 insertions(+), 1 deletion(-) create mode 100644 store/tikv/batch_coprocessor.go diff --git a/go.mod b/go.mod index ba2b4b8b50aac..7a7392bba389b 100644 --- a/go.mod +++ b/go.mod @@ -36,7 +36,7 @@ require ( github.com/pingcap/failpoint v0.0.0-20200210140405-f8f9fb234798 github.com/pingcap/fn v0.0.0-20191016082858-07623b84a47d github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20200221125103-35b65c96516e + github.com/pingcap/kvproto v0.0.0-20200306045313-90914f3920d7 github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd github.com/pingcap/parser v0.0.0-20200301092054-bfc519c0a57f github.com/pingcap/pd v1.1.0-beta.0.20200106144140-f5a7aa985497 diff --git a/go.sum b/go.sum index 6b478b0f53392..79bb90af2fec9 100644 --- a/go.sum +++ b/go.sum @@ -208,6 +208,10 @@ github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO github.com/pingcap/kvproto v0.0.0-20191213111810-93cb7c623c8b/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200221125103-35b65c96516e h1:z7j9uyuG/6I4god5h5NbsbMDSfhoOYAvVW6JxhwdHHw= github.com/pingcap/kvproto v0.0.0-20200221125103-35b65c96516e/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20200305060151-808c28b37e52 h1:9N63n+O5DQUrvJN7JCQPooDeRl73ZssaSL8jj4PYx4A= +github.com/pingcap/kvproto v0.0.0-20200305060151-808c28b37e52/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20200306045313-90914f3920d7 h1:KOSbqEWn0nFViynHFtavxSO7d/6Uk/gUOvTeD1eFc/M= +github.com/pingcap/kvproto v0.0.0-20200306045313-90914f3920d7/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfEgRJ4T9NGgGTxdHpJerent7rM= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= diff --git a/kv/kv.go b/kv/kv.go index ba2b9e3857de3..5f65088905769 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -293,6 +293,8 @@ type Request struct { Cacheable bool // SchemaVer is for any schema-ful storage to validate schema correctness if necessary. SchemaVar int64 + // CopTaskBatchSize is the max batch size for cop task + CopTaskBatchSize int } // ResultSubset represents a result subset from a single storage unit. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index e324a10d43a2e..f65931c231452 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -232,6 +232,9 @@ const ( // TiDBMaxChunkSize is used to control the max chunk size during query execution. TiDBMaxChunkSize = "tidb_max_chunk_size" + // TiDBCopTaskBatchSize is used to control the max batch size for cop task. + TiDBCopTaskBatchSize = "tidb_cop_task_batch_size" + // TiDBInitChunkSize is used to control the init chunk size during query execution. TiDBInitChunkSize = "tidb_init_chunk_size" @@ -421,6 +424,7 @@ const ( DefTiDBHashJoinConcurrency = 5 DefTiDBProjectionConcurrency = 4 DefTiDBOptimizerSelectivityLevel = 0 + DefCopTaskBatchSize = 10 DefTiDBTxnMode = "" DefTiDBRowFormatV1 = 1 DefTiDBRowFormatV2 = 2 diff --git a/store/tikv/batch_coprocessor.go b/store/tikv/batch_coprocessor.go new file mode 100644 index 0000000000000..6c472f425557e --- /dev/null +++ b/store/tikv/batch_coprocessor.go @@ -0,0 +1,438 @@ +package tikv + +import ( + "context" + "github.com/cznic/mathutil" + "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/coprocessor" + "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/tikv/tikvrpc" + "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/memory" + "go.uber.org/zap" + "io" + "sync" + "sync/atomic" + "time" +) + +type batchCopTask struct { + respChan chan *batchCopResponse + storeAddr string + cmdType tikvrpc.CmdType + + regionTaskMap map[uint64]*copTask + copTasks []copTaskAndRPCContext +} + +type batchCopResponse struct { + pbResp *coprocessor.BatchResponse + detail *execdetails.ExecDetails + startKey kv.Key + err error + respSize int64 + respTime time.Duration +} + + +// GetData implements the kv.ResultSubset GetData interface. +func (rs *batchCopResponse) GetData() []byte { + return rs.pbResp.Data +} + +// GetStartKey implements the kv.ResultSubset GetStartKey interface. +func (rs *batchCopResponse) GetStartKey() kv.Key { + return rs.startKey +} + +func (rs *batchCopResponse) GetExecDetails() *execdetails.ExecDetails { + return rs.detail +} + +// MemSize returns how many bytes of memory this response use +func (rs *batchCopResponse) MemSize() int64 { + if rs.respSize != 0 { + return rs.respSize + } + + // ignore rs.err + rs.respSize += int64(cap(rs.startKey)) + if rs.detail != nil { + rs.respSize += int64(sizeofExecDetails) + if rs.detail.CommitDetail != nil { + rs.respSize += int64(sizeofCommitDetails) + } + } + if rs.pbResp != nil { + // Using a approximate size since it's hard to get a accurate value. + rs.respSize += int64(rs.pbResp.Size()) + } + return rs.respSize +} + +func (rs *batchCopResponse) RespTime() time.Duration { + return rs.respTime +} + + +type copTaskAndRPCContext struct { + task *copTask + ctx *RPCContext +} + +func getFlashRPCContextWithRetry(bo *Backoffer, task *copTask, cache *RegionCache) (*RPCContext, error) { + for { + ctx, err := cache.GetTiFlashRPCContext(bo, task.region) + if err != nil { + return nil, err + } + if ctx != nil { + return ctx, nil + } + + if err = bo.Backoff(BoRegionMiss, errors.New("failed to get flash rpc context")); err != nil { + return nil, errors.Trace(err) + } + } +} + +func buildBatchCopTasks(bo *Backoffer, cache *RegionCache, ranges *copRanges, req *kv.Request) ([]*batchCopTask, error) { + if req.StoreType != kv.TiFlash { + return nil, errors.New("store type must be tiflash !") + } + + start := time.Now() + cmdType := tikvrpc.CmdBatchCop + rangesLen := ranges.len() + var tasks []*copTask + appendTask := func(regionWithRangeInfo *KeyLocation, ranges *copRanges) { + // TiKV will return gRPC error if the message is too large. So we need to limit the length of the ranges slice + // to make sure the message can be sent successfully. + rLen := ranges.len() + for i := 0; i < rLen; { + nextI := mathutil.Min(i+rangesPerTask, rLen) + tasks = append(tasks, &copTask{ + region: regionWithRangeInfo.Region, + ranges: ranges.slice(i, nextI), + // Channel buffer is 2 for handling region split. + // In a common case, two region split tasks will not be blocked. + respChan: make(chan *copResponse, 2), + cmdType: cmdType, + storeType: req.StoreType, + }) + i = nextI + } + } + + err := splitRanges(bo, cache, ranges, appendTask) + if err != nil { + return nil, errors.Trace(err) + } + + var batchTasks []*batchCopTask + + storeTaskMap := make(map[string]*batchCopTask) + + for _, task := range tasks { + rpcCtx, err := getFlashRPCContextWithRetry(bo, task, cache) + if err != nil { + return nil, err + } + if batchCop, ok := storeTaskMap[rpcCtx.Addr]; ok { + batchCop.copTasks = append(batchCop.copTasks, copTaskAndRPCContext{task: task, ctx: rpcCtx}) + batchCop.regionTaskMap[task.region.id] = task + } else { + batchTask := &batchCopTask { + respChan : make(chan * batchCopResponse, 10), + storeAddr: rpcCtx.Addr, + cmdType : task.cmdType, + regionTaskMap: make(map[uint64]*copTask), + } + batchTask.regionTaskMap[task.region.id] = task + storeTaskMap[rpcCtx.Addr] = batchTask + } + } + + for _, task := range storeTaskMap { + batchTasks = append(batchTasks, task) + } + + if elapsed := time.Since(start); elapsed > time.Millisecond*500 { + logutil.BgLogger().Warn("buildCopTasks takes too much time", + zap.Duration("elapsed", elapsed), + zap.Int("range len", rangesLen), + zap.Int("task len", len(tasks))) + } + tikvTxnRegionsNumHistogramWithCoprocessor.Observe(float64(len(tasks))) + return batchTasks, nil + +} + +func (c *CopClient) SendBatch(ctx context.Context, req *kv.Request, vars *kv.Variables) kv.Response { + ctx = context.WithValue(ctx, txnStartKey, req.StartTs) + bo := NewBackoffer(ctx, copBuildTaskMaxBackoff).WithVars(vars) + tasks, err := buildBatchCopTasks(bo, c.store.regionCache, &copRanges{mid: req.KeyRanges}, req) + if err != nil { + return copErrorResponse{err} + } + it := &batchCopIterator{ + store: c.store, + req: req, + concurrency: req.Concurrency, + finishCh: make(chan struct{}), + vars: vars, + memTracker: req.MemTracker, + replicaReadSeed: c.replicaReadSeed, + clientHelper: clientHelper{ + LockResolver: c.store.lockResolver, + RegionCache: c.store.regionCache, + Client: c.store.client, + minCommitTSPushed: &minCommitTSPushed{data: make(map[uint64]struct{}, 5)}, + }, + } + it.tasks = tasks + if it.concurrency > len(tasks) { + it.concurrency = len(tasks) + } + if it.concurrency < 1 { + // Make sure that there is at least one worker. + it.concurrency = 1 + } + if it.req.KeepOrder { + it.sendRate = newRateLimit(2 * it.concurrency) + } else { + it.respChan = make(chan *batchCopResponse, it.concurrency) + } + it.open(ctx) + return it +} + +type batchCopIterator struct { + clientHelper + + store *tikvStore + req *kv.Request + concurrency int + finishCh chan struct{} + + // If keepOrder, results are stored in copTask.respChan, read them out one by one. + tasks []*batchCopTask + curr int + // sendRate controls the sending rate of copIteratorTaskSender, if keepOrder, + // to prevent all tasks being done (aka. all of the responses are buffered) + sendRate *rateLimit + + // Otherwise, results are stored in respChan. + respChan chan *batchCopResponse + + vars *kv.Variables + + memTracker *memory.Tracker + + replicaReadSeed uint32 + + wg sync.WaitGroup + // closed represents when the Close is called. + // There are two cases we need to close the `finishCh` channel, one is when context is done, the other one is + // when the Close is called. we use atomic.CompareAndSwap `closed` to to make sure the channel is not closed twice. + closed uint32 + +} + +func (b * batchCopIterator) open(ctx context.Context) { + for _, task := range b.tasks { + bo := NewBackoffer(ctx, copNextMaxBackoff).WithVars(b.vars) + go b.handleTask(ctx, bo, task) + } + +} + +// Next returns next coprocessor result. +// NOTE: Use nil to indicate finish, so if the returned ResultSubset is not nil, reader should continue to call Next(). +func (b *batchCopIterator) Next(ctx context.Context) (kv.ResultSubset, error) { + var ( + resp *batchCopResponse + ok bool + closed bool + ) + // If data order matters, response should be returned in the same order as copTask slice. + // Otherwise all responses are returned from a single channel. + + // Get next fetched resp from chan + resp, ok, closed = b.recvFromRespCh(ctx) + if !ok || closed { + return nil, nil + } + + if resp.err != nil { + return nil, errors.Trace(resp.err) + } + + err := b.store.CheckVisibility(b.req.StartTs) + if err != nil { + return nil, errors.Trace(err) + } + return resp, nil +} + +func (b *batchCopIterator) recvFromRespCh(ctx context.Context) (resp *batchCopResponse, ok bool, exit bool) { + select { + case resp, ok = <-b.respChan: + case <-b.finishCh: + exit = true + case <-ctx.Done(): + // We select the ctx.Done() in the thread of `Next` instead of in the worker to avoid the cost of `WithCancel`. + if atomic.CompareAndSwapUint32(&b.closed, 0, 1) { + close(b.finishCh) + } + exit = true + } + return +} + +func (it *batchCopIterator) Close() error { + if atomic.CompareAndSwapUint32(&it.closed, 0, 1) { + close(it.finishCh) + } + it.wg.Wait() + return nil +} + + +func (b *batchCopIterator) handleTask(ctx context.Context, bo *Backoffer, task *batchCopTask) { + tasks := []*batchCopTask{task} + idx := 0 + for idx < len(tasks) { + ret, err := b.handleTaskOnce(ctx, bo, task) + if err != nil { + resp := &batchCopResponse{err : errors.Trace(err)} + b.sendToRespCh(resp) + } else { + tasks = append(tasks, ret...) + } + } +} + +func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo * Backoffer, task *batchCopTask)([]*batchCopTask, error){ + + sender := NewRegionBatchRequestSender(b.store.regionCache, b.store.client) + var regionInfos []*coprocessor.RegionInfo + for _, task := range task.copTasks { + regionInfos = append(regionInfos, &coprocessor.RegionInfo{ + RegionId: task.task.region.id, + RegionEpoch: &metapb.RegionEpoch{ + ConfVer: task.task.region.confVer, + Version: task.task.region.ver, + }, + Ranges: task.task.ranges.toPBRanges(), + }) + } + + copReq := coprocessor.BatchRequest{ + Tp: b.req.Tp, + StartTs: b.req.StartTs, + Data: b.req.Data, + SchemaVer: b.req.SchemaVar, + Regions: regionInfos, + } + + req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, b.req.ReplicaRead, b.replicaReadSeed, kvrpcpb.Context{ + IsolationLevel: pbIsolationLevel(b.req.IsolationLevel), + Priority: kvPriorityToCommandPri(b.req.Priority), + NotFillCache: b.req.NotFillCache, + HandleTime: true, + ScanDetail: true, + }) + req.StoreTp = kv.TiFlash + + resp, err := sender.sendReqToAddr(bo, task.copTasks, req, ReadTimeoutMedium) + if err != nil { + return nil, errors.Trace(err) + } + return b.handleBatchCopResponse(ctx, bo, resp.Resp.(*coprocessor.BatchResponse), task) +} +func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *Backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (totalRetTask []*batchCopTask, err error) { + for { + resp, err := response.Recv() + if err != nil { + if errors.Cause(err) == io.EOF { + return nil, nil + } + + if err1 := bo.Backoff(boTiKVRPC, errors.Errorf("recv stream response error: %v, task: %s", err, task)); err1 != nil { + return nil, errors.Trace(err) + } + + // No coprocessor.Response for network error, rebuild task based on the last success one. + if errors.Cause(err) == context.Canceled { + logutil.BgLogger().Info("stream recv timeout", zap.Error(err)) + } else { + logutil.BgLogger().Info("stream unknown error", zap.Error(err)) + } + } + + remainedTasks, err := b.handleBatchCopResponse(ctx, bo, resp, task) + if err != nil || len(remainedTasks) != 0 { + return remainedTasks, errors.Trace(err) + } + } +} + +func (b *batchCopIterator) handleBatchCopResponse(ctx context.Context, bo *Backoffer, response *coprocessor.BatchResponse, task *batchCopTask) (totalRetTask []*batchCopTask, err error) { + for _, status := range response.RegionStatus { + id := status.RegionId + if status.RegionError != nil { + + if err := bo.Backoff(BoRegionMiss, errors.New(status.RegionError.String())); err != nil { + return nil, errors.Trace(err) + } + + copTask := task.regionTaskMap[id] + + // We may meet RegionError at the first packet, but not during visiting the stream. + retTasks, err := buildBatchCopTasks(bo, b.store.regionCache, copTask.ranges, b.req) + if err != nil { + return nil, errors.Trace(err) + } + totalRetTask = append(totalRetTask, retTasks...) + } + if status.Locked != nil { + msBeforeExpired, err1 := b.ResolveLocks(bo, b.req.StartTs, []*Lock{NewLock(status.Locked)}) + if err1 != nil { + return nil, errors.Trace(err1) + } + if msBeforeExpired > 0 { + if err := bo.BackoffWithMaxSleep(boTxnLockFast, int(msBeforeExpired), errors.New(status.Locked.String())); err != nil { + return nil, errors.Trace(err) + } + } + totalRetTask = append(totalRetTask, task) + } + } + if otherErr := response.GetOtherError(); otherErr != "" { + err := errors.Errorf("other error: %s", otherErr) + logutil.BgLogger().Warn("other error", + zap.Uint64("txnStartTS", b.req.StartTs), + zap.String("storeAddr", task.storeAddr), + zap.Error(err)) + return nil, errors.Trace(err) + } + + b.sendToRespCh(&batchCopResponse{ + pbResp: response, + }) + + return totalRetTask, nil +} + +func (b *batchCopIterator) sendToRespCh(resp *batchCopResponse) (exit bool) { + select { + case b.respChan <- resp: + case <-b.finishCh: + exit = true + } + return +} diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 1aa5f3734d579..2cebb2dc60d9d 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -52,6 +52,9 @@ type CopClient struct { // Send builds the request and gets the coprocessor iterator response. func (c *CopClient) Send(ctx context.Context, req *kv.Request, vars *kv.Variables) kv.Response { + if req.StoreType == kv.TiFlash { + return c.SendBatch(ctx, req, vars) + } ctx = context.WithValue(ctx, txnStartKey, req.StartTs) bo := NewBackoffer(ctx, copBuildTaskMaxBackoff).WithVars(vars) tasks, err := buildCopTasks(bo, c.store.regionCache, &copRanges{mid: req.KeyRanges}, req) diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 5b55b49b05cd5..afd0f0954300e 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -15,6 +15,7 @@ package tikv import ( "context" + "github.com/pingcap/kvproto/pkg/coprocessor" "strconv" "sync/atomic" "time" @@ -62,6 +63,51 @@ type RegionRequestSender struct { failStoreIDs map[uint64]struct{} } +type RegionBatchRequestSender struct { + RegionRequestSender +} + +func NewRegionBatchRequestSender(cache *RegionCache, client Client) *RegionBatchRequestSender { + return &RegionBatchRequestSender{RegionRequestSender: RegionRequestSender{regionCache: cache, client: client}} +} + +func (ss *RegionBatchRequestSender) sendReqToAddr(bo *Backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, err error) { + // use the first ctx to send request + ctx := ctxs[0].ctx + if e := tikvrpc.SetContext(req, ctx.Meta, ctx.Peer); e != nil { + return nil, errors.Trace(e) + } + for { + resp, err = ss.client.SendRequest(bo.ctx, ctx.Addr, req, timout) + if err != nil { + ss.rpcError = err + // todo should all the region need to call onSendFail?? + e := ss.onSendFail(bo, ctx, err) + if e != nil { + return nil, errors.Trace(e) + } + // always return error on send fail + return nil, errors.Trace(err) + } + seed := req.ReplicaReadSeed + if batchResponse, isBatchResponse := resp.Resp.(*coprocessor.BatchResponse); isBatchResponse { + for idx, s := range batchResponse.RegionStatus { + if s.RegionError != nil { + // do not retry on region error + // todo check check all the region error + _, err = ss.onRegionError(bo, ctxs[idx].ctx, &seed, s.RegionError) + if err != nil { + return nil, errors.Trace(err) + } + } + } + } else { + return nil, errors.New("Should not happen, batch coprocessor request should receive batch coprocessor response") + } + return resp, nil + } +} + // NewRegionRequestSender creates a new sender. func NewRegionRequestSender(regionCache *RegionCache, client Client) *RegionRequestSender { return &RegionRequestSender{ diff --git a/store/tikv/tikvrpc/tikvrpc.go b/store/tikv/tikvrpc/tikvrpc.go index 7dbd0d98d5fb0..76b16ba7d94d7 100644 --- a/store/tikv/tikvrpc/tikvrpc.go +++ b/store/tikv/tikvrpc/tikvrpc.go @@ -68,6 +68,7 @@ const ( CmdCop CmdType = 512 + iota CmdCopStream + CmdBatchCop CmdMvccGetByKey CmdType = 1024 + iota CmdMvccGetByStartTs @@ -136,6 +137,8 @@ func (t CmdType) String() string { return "Cop" case CmdCopStream: return "CopStream" + case CmdBatchCop: + return "BatchCop" case CmdMvccGetByKey: return "MvccGetByKey" case CmdMvccGetByStartTs: @@ -309,6 +312,11 @@ func (req *Request) Cop() *coprocessor.Request { return req.req.(*coprocessor.Request) } +// BatchCop returns coprocessor request in request. +func (req *Request) BatchCop() *coprocessor.BatchRequest { + return req.req.(*coprocessor.BatchRequest) +} + // MvccGetByKey returns MvccGetByKeyRequest in request. func (req *Request) MvccGetByKey() *kvrpcpb.MvccGetByKeyRequest { return req.req.(*kvrpcpb.MvccGetByKeyRequest) @@ -495,6 +503,13 @@ type CopStreamResponse struct { Lease // Shared by this object and a background goroutine. } +type BatchCopStreamResponse struct { + tikvpb.Tikv_BatchCoprocessorClient + *coprocessor.BatchResponse // The first result of Recv() + Timeout time.Duration + Lease // Shared by this object and a background goroutine. +} + // SetContext set the Context field for the given req to the specified ctx. func SetContext(req *Request, region *metapb.Region, peer *metapb.Peer) error { ctx := &req.Context @@ -561,6 +576,8 @@ func SetContext(req *Request, region *metapb.Region, peer *metapb.Peer) error { req.Cop().Context = ctx case CmdCopStream: req.Cop().Context = ctx + case CmdBatchCop: + req.BatchCop().Context = ctx case CmdMvccGetByKey: req.MvccGetByKey().Context = ctx case CmdMvccGetByStartTs: @@ -683,6 +700,11 @@ func GenRegionErrorResp(req *Request, e *errorpb.Error) (*Response, error) { RegionError: e, }, } + case CmdBatchCop: + // todo support batch cop + p = &coprocessor.BatchResponse{ + OtherError: e.Message, + } case CmdMvccGetByKey: p = &kvrpcpb.MvccGetByKeyResponse{ RegionError: e, @@ -797,6 +819,12 @@ func CallRPC(ctx context.Context, client tikvpb.TikvClient, req *Request) (*Resp resp.Resp = &CopStreamResponse{ Tikv_CoprocessorStreamClient: streamClient, } + case CmdBatchCop: + var streamClient tikvpb.Tikv_BatchCoprocessorClient + streamClient, err = client.BatchCoprocessor(ctx, req.BatchCop()) + resp.Resp = &BatchCopStreamResponse{ + Tikv_BatchCoprocessorClient: streamClient, + } case CmdMvccGetByKey: resp.Resp, err = client.MvccGetByKey(ctx, req.MvccGetByKey()) case CmdMvccGetByStartTs: From 460d5a8428b0ba6eb3666a4f15b6427736c1a497 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 10 Mar 2020 15:00:50 +0800 Subject: [PATCH 02/61] support batch cop --- store/tikv/batch_coprocessor.go | 23 ++++++++++++++++------- store/tikv/client.go | 7 +++++++ store/tikv/coprocessor.go | 1 + store/tikv/region_cache.go | 1 + store/tikv/region_request.go | 19 +++---------------- store/tikv/tikvrpc/tikvrpc.go | 5 ++++- 6 files changed, 32 insertions(+), 24 deletions(-) diff --git a/store/tikv/batch_coprocessor.go b/store/tikv/batch_coprocessor.go index 6c472f425557e..a107880ef3ab5 100644 --- a/store/tikv/batch_coprocessor.go +++ b/store/tikv/batch_coprocessor.go @@ -49,7 +49,7 @@ func (rs *batchCopResponse) GetStartKey() kv.Key { } func (rs *batchCopResponse) GetExecDetails() *execdetails.ExecDetails { - return rs.detail + return &execdetails.ExecDetails{} } // MemSize returns how many bytes of memory this response use @@ -119,7 +119,7 @@ func buildBatchCopTasks(bo *Backoffer, cache *RegionCache, ranges *copRanges, re ranges: ranges.slice(i, nextI), // Channel buffer is 2 for handling region split. // In a common case, two region split tasks will not be blocked. - respChan: make(chan *copResponse, 2), + respChan: make(chan *copResponse, 1024), cmdType: cmdType, storeType: req.StoreType, }) @@ -150,6 +150,7 @@ func buildBatchCopTasks(bo *Backoffer, cache *RegionCache, ranges *copRanges, re storeAddr: rpcCtx.Addr, cmdType : task.cmdType, regionTaskMap: make(map[uint64]*copTask), + copTasks : []copTaskAndRPCContext{{task, rpcCtx}} , } batchTask.regionTaskMap[task.region.id] = task storeTaskMap[rpcCtx.Addr] = batchTask @@ -206,7 +207,7 @@ func (c *CopClient) SendBatch(ctx context.Context, req *kv.Request, vars *kv.Var } else { it.respChan = make(chan *batchCopResponse, it.concurrency) } - it.open(ctx) + go it.run(ctx) return it } @@ -242,12 +243,14 @@ type batchCopIterator struct { } -func (b * batchCopIterator) open(ctx context.Context) { +func (b * batchCopIterator) run(ctx context.Context) { for _, task := range b.tasks { + b.wg.Add(1) bo := NewBackoffer(ctx, copNextMaxBackoff).WithVars(b.vars) go b.handleTask(ctx, bo, task) } - + b.wg.Wait() + close(b.respChan) } // Next returns next coprocessor result. @@ -303,6 +306,7 @@ func (it *batchCopIterator) Close() error { func (b *batchCopIterator) handleTask(ctx context.Context, bo *Backoffer, task *batchCopTask) { + logutil.BgLogger().Debug("handle batch task") tasks := []*batchCopTask{task} idx := 0 for idx < len(tasks) { @@ -313,11 +317,14 @@ func (b *batchCopIterator) handleTask(ctx context.Context, bo *Backoffer, task * } else { tasks = append(tasks, ret...) } + idx ++ } + close(task.respChan) + b.wg.Done(); } func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo * Backoffer, task *batchCopTask)([]*batchCopTask, error){ - + logutil.BgLogger().Debug("handle batch task once") sender := NewRegionBatchRequestSender(b.store.regionCache, b.store.client) var regionInfos []*coprocessor.RegionInfo for _, task := range task.copTasks { @@ -348,11 +355,12 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo * Backoffer, t }) req.StoreTp = kv.TiFlash + logutil.BgLogger().Debug("send batch to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.copTasks))) resp, err := sender.sendReqToAddr(bo, task.copTasks, req, ReadTimeoutMedium) if err != nil { return nil, errors.Trace(err) } - return b.handleBatchCopResponse(ctx, bo, resp.Resp.(*coprocessor.BatchResponse), task) + return b.handleStreamedBatchCopResponse(ctx, bo, resp.Resp.(*tikvrpc.BatchCopStreamResponse), task) } func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *Backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (totalRetTask []*batchCopTask, err error) { for { @@ -372,6 +380,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b } else { logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } + return nil, errors.Trace(err) } remainedTasks, err := b.handleBatchCopResponse(ctx, bo, resp, task) diff --git a/store/tikv/client.go b/store/tikv/client.go index 05f5f6849796c..df94e347baab7 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -16,6 +16,7 @@ package tikv import ( "context" + "go.uber.org/zap" "io" "math" "strconv" @@ -326,6 +327,12 @@ func (c *rpcClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R client := tikvpb.NewTikvClient(clientConn) + if req.Type == tikvrpc.CmdBatchCop { + logutil.BgLogger().Debug("send query to ", zap.String("store addr", addr)) + return tikvrpc.CallRPC(ctx, client, req) + + } + if req.Type != tikvrpc.CmdCopStream { ctx1, cancel := context.WithTimeout(ctx, timeout) defer cancel() diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 2cebb2dc60d9d..8f62022c415d2 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -53,6 +53,7 @@ type CopClient struct { // Send builds the request and gets the coprocessor iterator response. func (c *CopClient) Send(ctx context.Context, req *kv.Request, vars *kv.Variables) kv.Response { if req.StoreType == kv.TiFlash { + logutil.BgLogger().Info("send batch requests") return c.SendBatch(ctx, req, vars) } ctx = context.WithValue(ctx, txnStartKey, req.StartTs) diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 7688ce62c4638..b1efba774cbee 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -389,6 +389,7 @@ func (c *RegionCache) GetTiFlashRPCContext(bo *Backoffer, id RegionVerID) (*RPCC cachedRegion := c.getCachedRegionWithRLock(id) if cachedRegion == nil { + logutil.BgLogger().Info("not found region", zap.Uint64("region id", id.id)) return nil, nil } if !cachedRegion.checkRegionCacheTTL(ts) { diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index afd0f0954300e..bde4063df90d7 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -15,7 +15,6 @@ package tikv import ( "context" - "github.com/pingcap/kvproto/pkg/coprocessor" "strconv" "sync/atomic" "time" @@ -78,7 +77,9 @@ func (ss *RegionBatchRequestSender) sendReqToAddr(bo *Backoffer, ctxs []copTaskA return nil, errors.Trace(e) } for { + logutil.BgLogger().Debug("begin send") resp, err = ss.client.SendRequest(bo.ctx, ctx.Addr, req, timout) + logutil.BgLogger().Debug("end send") if err != nil { ss.rpcError = err // todo should all the region need to call onSendFail?? @@ -89,21 +90,7 @@ func (ss *RegionBatchRequestSender) sendReqToAddr(bo *Backoffer, ctxs []copTaskA // always return error on send fail return nil, errors.Trace(err) } - seed := req.ReplicaReadSeed - if batchResponse, isBatchResponse := resp.Resp.(*coprocessor.BatchResponse); isBatchResponse { - for idx, s := range batchResponse.RegionStatus { - if s.RegionError != nil { - // do not retry on region error - // todo check check all the region error - _, err = ss.onRegionError(bo, ctxs[idx].ctx, &seed, s.RegionError) - if err != nil { - return nil, errors.Trace(err) - } - } - } - } else { - return nil, errors.New("Should not happen, batch coprocessor request should receive batch coprocessor response") - } + logutil.BgLogger().Debug("no error") return resp, nil } } diff --git a/store/tikv/tikvrpc/tikvrpc.go b/store/tikv/tikvrpc/tikvrpc.go index 76b16ba7d94d7..6b8f2c42169b7 100644 --- a/store/tikv/tikvrpc/tikvrpc.go +++ b/store/tikv/tikvrpc/tikvrpc.go @@ -16,6 +16,8 @@ package tikvrpc import ( "context" "fmt" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" "sync/atomic" "time" @@ -314,6 +316,7 @@ func (req *Request) Cop() *coprocessor.Request { // BatchCop returns coprocessor request in request. func (req *Request) BatchCop() *coprocessor.BatchRequest { + logutil.BgLogger().Debug("convert batch cop") return req.req.(*coprocessor.BatchRequest) } @@ -505,7 +508,6 @@ type CopStreamResponse struct { type BatchCopStreamResponse struct { tikvpb.Tikv_BatchCoprocessorClient - *coprocessor.BatchResponse // The first result of Recv() Timeout time.Duration Lease // Shared by this object and a background goroutine. } @@ -822,6 +824,7 @@ func CallRPC(ctx context.Context, client tikvpb.TikvClient, req *Request) (*Resp case CmdBatchCop: var streamClient tikvpb.Tikv_BatchCoprocessorClient streamClient, err = client.BatchCoprocessor(ctx, req.BatchCop()) + logutil.BgLogger().Info("send batch cop", zap.Error(err)) resp.Resp = &BatchCopStreamResponse{ Tikv_BatchCoprocessorClient: streamClient, } From 7a147b697907a467239382a95d2954bc79ecbbeb Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 3 Mar 2020 21:32:31 +0800 Subject: [PATCH 03/61] support join push down to tiflash --- go.mod | 2 ++ go.sum | 18 ++++++++++ planner/core/exhaust_physical_plans.go | 50 +++++++++++++++++++++++++- planner/core/find_best_task.go | 11 ++++++ planner/core/initialize.go | 9 +++++ planner/core/physical_plans.go | 5 +++ planner/core/plan_to_pb.go | 50 ++++++++++++++++++++++++++ planner/core/task.go | 32 ++++++++++++++++- planner/property/task_type.go | 4 +++ util/plancodec/id.go | 2 ++ 10 files changed, 181 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 12aa780e2ea1a..95dcb916c669d 100644 --- a/go.mod +++ b/go.mod @@ -1,5 +1,7 @@ module github.com/pingcap/tidb +replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200303121836-bac275f41413 + require ( github.com/BurntSushi/toml v0.3.1 github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d diff --git a/go.sum b/go.sum index 9f180daab92e7..7ac276cc6e03d 100644 --- a/go.sum +++ b/go.sum @@ -126,6 +126,24 @@ github.com/grpc-ecosystem/grpc-gateway v1.9.5 h1:UImYN5qQ8tuGpGE16ZmjvcTtTw24zw1 github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.12.1 h1:zCy2xE9ablevUOrUZc3Dl72Dt+ya2FNAvC2yLYMHzi4= github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqCisSPWTxCZ7sBRjU6iH9c= +github.com/hanfei1991/tipb v0.0.0-20200226132147-3fc41406b29e h1:3O9Lc15gzfOWE+CQgsC42rH80GNgwJjPe9GOaT/ZPlk= +github.com/hanfei1991/tipb v0.0.0-20200226132147-3fc41406b29e/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200303064518-fd5903ecf553 h1:WMXryw+8ikrQE/5ZsTcOdnA1KxqJCP7Ds2BGdpoH5D4= +github.com/hanfei1991/tipb v0.0.0-20200303064518-fd5903ecf553/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200303080311-11a18b48d2f4 h1:EkP/4HUtskvzbV9I1vUNhRIgRUq5W7Jn8Ccd8Xv6eb0= +github.com/hanfei1991/tipb v0.0.0-20200303080311-11a18b48d2f4/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200303080556-7771d573338c h1:aVCLzm7OROyDFL3GiKecMa9a2cbr6XtgSdI9XTcHYBY= +github.com/hanfei1991/tipb v0.0.0-20200303080556-7771d573338c/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200303082900-4ffac1fcd26b h1:igFXErIhQyfeie+yqg1p3Px9H8DIo/yb4ZsmS1kLfks= +github.com/hanfei1991/tipb v0.0.0-20200303082900-4ffac1fcd26b/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200303083815-6e7609cb04a0 h1:JLti46jFoMrTi5skLE48gMr465URzfLkip4eyiyKh2w= +github.com/hanfei1991/tipb v0.0.0-20200303083815-6e7609cb04a0/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200303084628-76b8e5187c82 h1:rp3GXk7hmnfQIxI8ftbNIF7zgmj42nKkzvkjiRdUF1g= +github.com/hanfei1991/tipb v0.0.0-20200303084628-76b8e5187c82/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200303085530-971fefe489c3 h1:qBKRidEHqpUEQsa3LlolBhEtNBc14W+un7F1MLh5D08= +github.com/hanfei1991/tipb v0.0.0-20200303085530-971fefe489c3/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200303121836-bac275f41413 h1:d7xX2zBZi2DUTj6VIzlSXZvmKIwbyKAQt3yc6t3agBo= +github.com/hanfei1991/tipb v0.0.0-20200303121836-bac275f41413/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 6cd9b397bb221..5aa791522a20c 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1407,9 +1407,57 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph return hashJoins } joins = append(joins, hashJoins...) + + broadCastJoins := p.tryToGetBroadCastJoin(prop) + joins = append(joins, broadCastJoins...) return joins } +func (p *LogicalJoin) tryToGetBroadCastJoin(prop * property.PhysicalProperty) []PhysicalPlan { + child0, ok0 := p.children[0].(*DataSource) + if (!ok0) { + return nil; + } + child1, ok1 := p.children[1].(*DataSource) + if (!ok1) { + return nil; + } + if !prop.IsEmpty() { + return nil; + } + if (prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiflashTaskType) { + return nil + } + + if (p.JoinType != InnerJoin || len(p.LeftConditions) != 0 || len(p.RightConditions) != 0) { + return nil + } + + lkeys, rkeys := p.GetJoinKeys() + baseJoin := basePhysicalJoin{ + JoinType: p.JoinType, + LeftConditions: p.LeftConditions, + RightConditions: p.RightConditions, + DefaultValues: p.DefaultValues, + LeftJoinKeys: lkeys, + RightJoinKeys: rkeys, + } + if child0.stats.Count() < child1.stats.Count() { + baseJoin.InnerChildIdx = 0; + } else { + baseJoin.InnerChildIdx = 1; + } + childrenReqProps := make([]*property.PhysicalProperty, 2) + childrenReqProps[baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopSingleReadTaskType} + childrenReqProps[1-baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiflashTaskType} + join := PhysicalBroadCastJoin { + basePhysicalJoin: baseJoin, + }.Init(p.ctx, p.stats, p.blockOffset, childrenReqProps...) + results := make([]PhysicalPlan, 0, 1); + results = append(results, join) + return results +} + // TryToGetChildProp will check if this sort property can be pushed or not. // When a sort column will be replaced by scalar function, we refuse it. // When a sort column will be replaced by a constant, we just remove it. @@ -1636,7 +1684,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy return nil } hashAggs := make([]PhysicalPlan, 0, len(wholeTaskTypes)) - taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType} + taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType, property.CopTiflashTaskType} if !la.aggHints.preferAggToCop { taskTypes = append(taskTypes, property.RootTaskType) } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index a4f440a885801..643d1edb950a1 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -362,6 +362,9 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida if len(path.Ranges) == 0 && !ds.ctx.GetSessionVars().StmtCtx.UseCache { return []*candidatePath{{path: path}} } + if path.StoreType != kv.TiFlash && prop.TaskTp == property.CopTiflashTaskType { + continue + } var currentCandidate *candidatePath if path.IsTablePath { currentCandidate = ds.getTableCandidate(path, prop) @@ -449,6 +452,8 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty) (t task, err t = invalidTask candidates := ds.skylinePruning(prop) + logutil.BgLogger().Debug("candidates size", zap.Int("candidates", len(candidates))) + for _, candidate := range candidates { path := candidate.path if path.PartialIndexPaths != nil { @@ -1099,6 +1104,12 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid tblColHists: ds.TblColHists, cst: cost, } + if ts.StoreType == kv.TiFlash { + if prop.TaskTp != property.CopTiflashTaskType && prop.TaskTp != property.RootTaskType { + logutil.BgLogger().Error("invalid task:", zap.String("type", prop.TaskTp.String())) + return invalidTask, nil + } + } task = copTask if candidate.isMatchProp { copTask.keepOrder = true diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 7892aab055921..de10564d4388f 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -314,6 +314,15 @@ func (p PhysicalHashJoin) Init(ctx sessionctx.Context, stats *property.StatsInfo return &p } +func (p PhysicalBroadCastJoin) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalBroadCastJoin { + tp := plancodec.TypeBroadcastJoin + p.basePhysicalPlan = newBasePhysicalPlan(ctx, tp, &p, offset) + p.childrenReqProps = props + p.stats = stats + return &p + +} + // Init initializes PhysicalMergeJoin. func (p PhysicalMergeJoin) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int) *PhysicalMergeJoin { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeMergeJoin, &p, offset) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index c77713f103b35..36c5d5e8958e0 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -51,6 +51,7 @@ var ( _ PhysicalPlan = &PhysicalStreamAgg{} _ PhysicalPlan = &PhysicalApply{} _ PhysicalPlan = &PhysicalIndexJoin{} + _ PhysicalPlan = &PhysicalBroadCastJoin{} _ PhysicalPlan = &PhysicalHashJoin{} _ PhysicalPlan = &PhysicalMergeJoin{} _ PhysicalPlan = &PhysicalUnionScan{} @@ -411,6 +412,10 @@ type PhysicalMergeJoin struct { Desc bool } +type PhysicalBroadCastJoin struct { + basePhysicalJoin +} + // PhysicalLock is the physical operator of lock, which is used for `select ... for update` clause. type PhysicalLock struct { basePhysicalPlan diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index e9511b97ce46f..0c295ceb3d799 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -152,6 +152,56 @@ func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) return &tipb.Executor{Tp: tipb.ExecType_TypeIndexScan, IdxScan: idxExec}, nil } +func getChildrenList(ctx sessionctx.Context, p PhysicalPlan, inner_side bool) (results []*tipb.Executor, err error) { + for { + planPB, err := p.ToPB(ctx) + if err != nil { + return nil, errors.Trace(err) + } + if planPB.Tp == tipb.ExecType_TypeTableScan { + if inner_side { + planPB.TblScan.NextReadEngine = tipb.EngineType_TiKV + } + } + results = append(results, planPB) + if len(p.Children()) == 0 { + return results ,nil + } else if len(p.Children()) == 1 { + p = p.Children()[0] + } else { + return nil, errors.New("plan error:" + p.TP()) + } + } + return results, nil +} + +func (p *PhysicalBroadCastJoin) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { + sc := ctx.GetSessionVars().StmtCtx + client := ctx.GetClient() + leftJoinKeys := make([]expression.Expression, 0, len(p.LeftJoinKeys)) + rightJoinKeys := make([]expression.Expression, 0, len(p.RightJoinKeys)) + lChildren, err := getChildrenList(ctx, p.children[0], 0 == p.InnerChildIdx) + if err != nil { + return nil, errors.Trace(err) + } + rChildren, err := getChildrenList(ctx, p.children[1], 1 == p.InnerChildIdx) + if err != nil { + return nil, errors.Trace(err) + } + + join := &tipb.Join { + JoinType: tipb.JoinType_TypeInnerJoin, + JoinExecType: tipb.JoinExecType_TypeHashJoin, + InnerIdx: int64(p.InnerChildIdx), + LeftJoinKeys: expression.ExpressionsToPBList(sc, leftJoinKeys, client), + RightJoinKeys: expression.ExpressionsToPBList(sc, rightJoinKeys, client), + LeftChildren: lChildren, + RightChildren: rChildren, + } + + return &tipb.Executor{Tp: tipb.ExecType_TypeJoin, Join: join}, nil +} + // SetPBColumnsDefaultValue sets the default values of tipb.ColumnInfos. func SetPBColumnsDefaultValue(ctx sessionctx.Context, pbColumns []*tipb.ColumnInfo, columns []*model.ColumnInfo) error { for i, c := range columns { diff --git a/planner/core/task.go b/planner/core/task.go index 46ed1e05243c8..3d48cfc945a8b 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -14,6 +14,8 @@ package core import ( + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" "math" "github.com/pingcap/parser/ast" @@ -98,6 +100,7 @@ func (t *copTask) copy() task { } func (t *copTask) plan() PhysicalPlan { + logutil.BgLogger().Info("index finish", zap.Bool("index finish", t.indexPlanFinished), zap.Bool("table is nil?", t.tablePlan == nil)) if t.indexPlanFinished { return t.tablePlan } @@ -518,10 +521,37 @@ func (p *PhysicalHashJoin) attach2Task(tasks ...task) task { lTask := finishCopTask(p.ctx, tasks[0].copy()) rTask := finishCopTask(p.ctx, tasks[1].copy()) p.SetChildren(lTask.plan(), rTask.plan()) - return &rootTask{ + p.schema = BuildPhysicalJoinSchema(p.JoinType, p) + task := &rootTask{ p: p, cst: lTask.cost() + rTask.cost() + p.GetCost(lTask.count(), rTask.count()), } + logutil.BgLogger().Info("hash join cost", zap.Float64("hj cost", task.cst)) + return task +} + +func (p *PhysicalBroadCastJoin) attach2Task(tasks ...task) task { + lTask, lok := tasks[0].(*copTask) + rTask, rok := tasks[1].(*copTask) + if !lok || !rok || (lTask.getStoreType() != kv.TiFlash && rTask.getStoreType() != kv.TiFlash) { + return invalidTask + } + p.SetChildren(lTask.plan(), rTask.plan()) + p.schema = BuildPhysicalJoinSchema(p.JoinType, p) + if !lTask.indexPlanFinished { + lTask.finishIndexPlan() + } + if !rTask.indexPlanFinished { + rTask.finishIndexPlan() + } + task := & copTask { + tblColHists: rTask.tblColHists, + indexPlanFinished: true, + tablePlan: p, + cst: lTask.cost() + rTask.cost(), + } + logutil.BgLogger().Info("bc join cost", zap.Float64("bc cost", task.cst)) + return task } // GetCost computes cost of merge join operator itself. diff --git a/planner/property/task_type.go b/planner/property/task_type.go index 93360fc14ce9b..f27d2ead3be69 100644 --- a/planner/property/task_type.go +++ b/planner/property/task_type.go @@ -27,6 +27,8 @@ const ( // CopDoubleReadTaskType stands for the a IndexLookup tasks executed in the // coprocessor layer. CopDoubleReadTaskType + + CopTiflashTaskType ) // String implements fmt.Stringer interface. @@ -38,6 +40,8 @@ func (t TaskType) String() string { return "copSingleReadTask" case CopDoubleReadTaskType: return "copDoubleReadTask" + case CopTiflashTaskType: + return "copTiflashTaskType" } return "UnknownTaskType" } diff --git a/util/plancodec/id.go b/util/plancodec/id.go index eea1fe12c7567..cf94434605c53 100644 --- a/util/plancodec/id.go +++ b/util/plancodec/id.go @@ -52,6 +52,8 @@ const ( TypeHashLeftJoin = "HashLeftJoin" // TypeHashRightJoin is the type of right hash join. TypeHashRightJoin = "HashRightJoin" + // TypeHashRightJoin is the type of right hash join. + TypeBroadcastJoin = "TypeBroadcastJoin" // TypeMergeJoin is the type of merge join. TypeMergeJoin = "MergeJoin" // TypeIndexJoin is the type of index look up join. From 943863b44ec0fb7435591687702ecb91bf3fcdd0 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Wed, 4 Mar 2020 15:56:49 +0800 Subject: [PATCH 04/61] refine --- planner/core/task.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/planner/core/task.go b/planner/core/task.go index 3d48cfc945a8b..e133ca5befdb3 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -730,7 +730,12 @@ func finishCopTask(ctx sessionctx.Context, task task) task { tp := t.tablePlan splitCopAvg2CountAndSum(tp) for len(tp.Children()) > 0 { - tp = tp.Children()[0] + if len(tp.Children()) == 1 { + tp = tp.Children()[0] + } else { + join := tp.(*PhysicalBroadCastJoin) + tp = join.children[1 - join.InnerChildIdx] + } } ts := tp.(*PhysicalTableScan) p := PhysicalTableReader{ From a654fda2fde4f8bcc2ff82395d93803bdccad514 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Wed, 4 Mar 2020 17:31:57 +0800 Subject: [PATCH 05/61] change pb --- executor/builder.go | 24 +++++++--- go.mod | 2 +- go.sum | 4 ++ planner/core/physical_plans.go | 6 ++- planner/core/plan.go | 3 +- planner/core/plan_to_pb.go | 80 +++++++++++++++++++--------------- planner/core/point_get_plan.go | 4 +- 7 files changed, 75 insertions(+), 48 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index e513c68bec230..939f841ea4cc5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1903,7 +1903,7 @@ func constructDistExec(sctx sessionctx.Context, plans []plannercore.PhysicalPlan streaming := true executors := make([]*tipb.Executor, 0, len(plans)) for _, p := range plans { - execPB, err := p.ToPB(sctx) + execPB, err := p.ToPB(sctx, kv.TiKV) if err != nil { return nil, false, err } @@ -1925,12 +1925,22 @@ func markChildrenUsedCols(outputSchema *expression.Schema, childSchema ...*expre return } -func (b *executorBuilder) constructDAGReq(plans []plannercore.PhysicalPlan) (dagReq *tipb.DAGRequest, streaming bool, err error) { +func constructDistExecForTiFlash(sctx sessionctx.Context, p plannercore.PhysicalPlan) ([]*tipb.Executor, bool, error) { + execPB, err := p.ToPB(sctx, kv.TiFlash) + return []*tipb.Executor{execPB}, false, errors.Trace(err) + +} + +func (b *executorBuilder) constructDAGReq(plans []plannercore.PhysicalPlan, storeType kv.StoreType) (dagReq *tipb.DAGRequest, streaming bool, err error) { dagReq = &tipb.DAGRequest{} dagReq.TimeZoneName, dagReq.TimeZoneOffset = timeutil.Zone(b.ctx.GetSessionVars().Location()) sc := b.ctx.GetSessionVars().StmtCtx dagReq.Flags = sc.PushDownFlags() - dagReq.Executors, streaming, err = constructDistExec(b.ctx, plans) + if storeType == kv.TiFlash { + dagReq.Executors, streaming, err = constructDistExecForTiFlash(b.ctx, plans[0]) + } else { + dagReq.Executors, streaming, err = constructDistExec(b.ctx, plans) + } distsql.SetEncodeType(b.ctx, dagReq) return dagReq, streaming, err @@ -2151,7 +2161,7 @@ func containsLimit(execs []*tipb.Executor) bool { } func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableReader) (*TableReaderExecutor, error) { - dagReq, streaming, err := b.constructDAGReq(v.TablePlans) + dagReq, streaming, err := b.constructDAGReq(v.TablePlans, v.StoreType) if err != nil { return nil, err } @@ -2229,7 +2239,7 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) * } func buildNoRangeIndexReader(b *executorBuilder, v *plannercore.PhysicalIndexReader) (*IndexReaderExecutor, error) { - dagReq, streaming, err := b.constructDAGReq(v.IndexPlans) + dagReq, streaming, err := b.constructDAGReq(v.IndexPlans, kv.TiKV) if err != nil { return nil, err } @@ -2303,7 +2313,7 @@ func (b *executorBuilder) buildIndexReader(v *plannercore.PhysicalIndexReader) * } func buildTableReq(b *executorBuilder, schemaLen int, plans []plannercore.PhysicalPlan) (dagReq *tipb.DAGRequest, streaming bool, val table.Table, err error) { - tableReq, tableStreaming, err := b.constructDAGReq(plans) + tableReq, tableStreaming, err := b.constructDAGReq(plans, kv.TiKV) if err != nil { return nil, false, nil, err } @@ -2316,7 +2326,7 @@ func buildTableReq(b *executorBuilder, schemaLen int, plans []plannercore.Physic } func buildIndexReq(b *executorBuilder, schemaLen int, plans []plannercore.PhysicalPlan) (dagReq *tipb.DAGRequest, streaming bool, err error) { - indexReq, indexStreaming, err := b.constructDAGReq(plans) + indexReq, indexStreaming, err := b.constructDAGReq(plans, kv.TiKV) if err != nil { return nil, false, err } diff --git a/go.mod b/go.mod index 95dcb916c669d..f3e1a35cd30b6 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module github.com/pingcap/tidb -replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200303121836-bac275f41413 +replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200304091852-aa600db6e32e require ( github.com/BurntSushi/toml v0.3.1 diff --git a/go.sum b/go.sum index 7ac276cc6e03d..e845ddd6f8df8 100644 --- a/go.sum +++ b/go.sum @@ -144,6 +144,10 @@ github.com/hanfei1991/tipb v0.0.0-20200303085530-971fefe489c3 h1:qBKRidEHqpUEQsa github.com/hanfei1991/tipb v0.0.0-20200303085530-971fefe489c3/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hanfei1991/tipb v0.0.0-20200303121836-bac275f41413 h1:d7xX2zBZi2DUTj6VIzlSXZvmKIwbyKAQt3yc6t3agBo= github.com/hanfei1991/tipb v0.0.0-20200303121836-bac275f41413/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200304084323-938d55c99bfc h1:L9YrRlM/u5GBn0hAAiJWL2W25To2jvlJdu9VPX7cMUk= +github.com/hanfei1991/tipb v0.0.0-20200304084323-938d55c99bfc/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200304091852-aa600db6e32e h1:Or0qTCdPJ/NBxsqYyTLY2c93ZZOfLpLqBLLGQhWn4lE= +github.com/hanfei1991/tipb v0.0.0-20200304091852-aa600db6e32e/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 36c5d5e8958e0..f4631ddcab9f2 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -94,7 +94,11 @@ func (sg *TiKVSingleGather) GetPhysicalIndexReader(schema *expression.Schema, st // SetChildren overrides PhysicalPlan SetChildren interface. func (p *PhysicalTableReader) SetChildren(children ...PhysicalPlan) { p.tablePlan = children[0] - p.TablePlans = flattenPushDownPlan(p.tablePlan) + if p.StoreType == kv.TiFlash { + p.TablePlans = append(p.TablePlans, p.tablePlan) + } else { + p.TablePlans = flattenPushDownPlan(p.tablePlan) + } } // PhysicalIndexReader is the index reader in tidb. diff --git a/planner/core/plan.go b/planner/core/plan.go index 4e2b06df95eb9..1e9a5cc8ae156 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -15,6 +15,7 @@ package core import ( "fmt" + "github.com/pingcap/tidb/kv" "math" "strconv" @@ -203,7 +204,7 @@ type PhysicalPlan interface { attach2Task(...task) task // ToPB converts physical plan to tipb executor. - ToPB(ctx sessionctx.Context) (*tipb.Executor, error) + ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) // getChildReqProps gets the required property by child index. GetChildReqProps(idx int) *property.PhysicalProperty diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 0c295ceb3d799..f137ac1ea12ad 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" @@ -28,12 +29,12 @@ import ( ) // ToPB implements PhysicalPlan ToPB interface. -func (p *basePhysicalPlan) ToPB(_ sessionctx.Context) (*tipb.Executor, error) { +func (p *basePhysicalPlan) ToPB(_ sessionctx.Context, _ kv.StoreType) (*tipb.Executor, error) { return nil, errors.Errorf("plan %s fails converts to PB", p.basePlan.ExplainID()) } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { +func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { sc := ctx.GetSessionVars().StmtCtx client := ctx.GetClient() aggExec := &tipb.Aggregation{ @@ -42,11 +43,18 @@ func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { for _, aggFunc := range p.AggFuncs { aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc)) } + if (storeType == kv.TiFlash) { + var err error + aggExec.Child, err = p.children[0].ToPB(ctx, storeType) + if err != nil { + return nil, errors.Trace(err) + } + } return &tipb.Executor{Tp: tipb.ExecType_TypeAggregation, Aggregation: aggExec}, nil } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { +func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb.Executor, error) { sc := ctx.GetSessionVars().StmtCtx client := ctx.GetClient() aggExec := &tipb.Aggregation{ @@ -59,17 +67,24 @@ func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalSelection) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { +func (p *PhysicalSelection) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { sc := ctx.GetSessionVars().StmtCtx client := ctx.GetClient() selExec := &tipb.Selection{ Conditions: expression.ExpressionsToPBList(sc, p.Conditions, client), } + if (storeType == kv.TiFlash) { + var err error + selExec.Child, err = p.children[0].ToPB(ctx, storeType) + if err != nil { + return nil, errors.Trace(err) + } + } return &tipb.Executor{Tp: tipb.ExecType_TypeSelection, Selection: selExec}, nil } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalTopN) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { +func (p *PhysicalTopN) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { sc := ctx.GetSessionVars().StmtCtx client := ctx.GetClient() topNExec := &tipb.TopN{ @@ -78,19 +93,33 @@ func (p *PhysicalTopN) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { for _, item := range p.ByItems { topNExec.OrderBy = append(topNExec.OrderBy, expression.SortByItemToPB(sc, client, item.Expr, item.Desc)) } + if (storeType == kv.TiFlash) { + var err error + topNExec.Child, err = p.children[0].ToPB(ctx, storeType) + if err != nil { + return nil, errors.Trace(err) + } + } return &tipb.Executor{Tp: tipb.ExecType_TypeTopN, TopN: topNExec}, nil } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalLimit) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { +func (p *PhysicalLimit) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { limitExec := &tipb.Limit{ Limit: p.Count, } + if (storeType == kv.TiFlash) { + var err error + limitExec.Child, err = p.children[0].ToPB(ctx, storeType) + if err != nil { + return nil, errors.Trace(err) + } + } return &tipb.Executor{Tp: tipb.ExecType_TypeLimit, Limit: limitExec}, nil } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { +func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { tsExec := &tipb.TableScan{ TableId: p.Table.ID, Columns: util.ColumnsToProto(p.Columns, p.Table.PKIsHandle), @@ -99,6 +128,9 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) if p.isPartition { tsExec.TableId = p.physicalTableID } + if storeType == kv.TiFlash && p.StoreType == kv.TiKV { + tsExec.NextReadEngine = tipb.EngineType_TiKV + } err := SetPBColumnsDefaultValue(ctx, tsExec.Columns, p.Columns) return &tipb.Executor{Tp: tipb.ExecType_TypeTableScan, TblScan: tsExec}, err } @@ -128,7 +160,7 @@ func findColumnInfoByID(infos []*model.ColumnInfo, id int64) *model.ColumnInfo { } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { +func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb.Executor, error) { columns := make([]*model.ColumnInfo, 0, p.schema.Len()) tableColumns := p.Table.Cols() for _, col := range p.schema.Columns { @@ -152,39 +184,16 @@ func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) return &tipb.Executor{Tp: tipb.ExecType_TypeIndexScan, IdxScan: idxExec}, nil } -func getChildrenList(ctx sessionctx.Context, p PhysicalPlan, inner_side bool) (results []*tipb.Executor, err error) { - for { - planPB, err := p.ToPB(ctx) - if err != nil { - return nil, errors.Trace(err) - } - if planPB.Tp == tipb.ExecType_TypeTableScan { - if inner_side { - planPB.TblScan.NextReadEngine = tipb.EngineType_TiKV - } - } - results = append(results, planPB) - if len(p.Children()) == 0 { - return results ,nil - } else if len(p.Children()) == 1 { - p = p.Children()[0] - } else { - return nil, errors.New("plan error:" + p.TP()) - } - } - return results, nil -} - -func (p *PhysicalBroadCastJoin) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { +func (p *PhysicalBroadCastJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { sc := ctx.GetSessionVars().StmtCtx client := ctx.GetClient() leftJoinKeys := make([]expression.Expression, 0, len(p.LeftJoinKeys)) rightJoinKeys := make([]expression.Expression, 0, len(p.RightJoinKeys)) - lChildren, err := getChildrenList(ctx, p.children[0], 0 == p.InnerChildIdx) + lChildren, err := p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } - rChildren, err := getChildrenList(ctx, p.children[1], 1 == p.InnerChildIdx) + rChildren, err := p.children[1].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } @@ -195,8 +204,7 @@ func (p *PhysicalBroadCastJoin) ToPB(ctx sessionctx.Context) (*tipb.Executor, er InnerIdx: int64(p.InnerChildIdx), LeftJoinKeys: expression.ExpressionsToPBList(sc, leftJoinKeys, client), RightJoinKeys: expression.ExpressionsToPBList(sc, rightJoinKeys, client), - LeftChildren: lChildren, - RightChildren: rChildren, + Children: []*tipb.Executor{lChildren, rChildren}, } return &tipb.Executor{Tp: tipb.ExecType_TypeJoin, Join: join}, nil diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 195c4be391a14..4b2299b39a5c4 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -87,7 +87,7 @@ func (p *PointGetPlan) attach2Task(...task) task { } // ToPB converts physical plan to tipb executor. -func (p *PointGetPlan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { +func (p *PointGetPlan) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb.Executor, error) { return nil, nil } @@ -220,7 +220,7 @@ func (p *BatchPointGetPlan) attach2Task(...task) task { } // ToPB converts physical plan to tipb executor. -func (p *BatchPointGetPlan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { +func (p *BatchPointGetPlan) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb.Executor, error) { return nil, nil } From 2d7e3c0f570b9e06e6df47917346ba9168f64c62 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Wed, 4 Mar 2020 19:07:01 +0800 Subject: [PATCH 06/61] push join --- executor/builder.go | 6 +++++- planner/core/physical_plans.go | 10 +++++----- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 939f841ea4cc5..9a5aebe55d0b7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2161,7 +2161,11 @@ func containsLimit(execs []*tipb.Executor) bool { } func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableReader) (*TableReaderExecutor, error) { - dagReq, streaming, err := b.constructDAGReq(v.TablePlans, v.StoreType) + tablePlans := v.TablePlans + if v.StoreType == kv.TiFlash { + tablePlans = []plannercore.PhysicalPlan{v.GetTablePlan()} + } + dagReq, streaming, err := b.constructDAGReq(tablePlans, v.StoreType) if err != nil { return nil, err } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index f4631ddcab9f2..5391b921cf5e2 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -73,6 +73,10 @@ type PhysicalTableReader struct { StoreType kv.StoreType } +func (p *PhysicalTableReader) GetTablePlan() PhysicalPlan { + return p.tablePlan +} + // GetPhysicalTableReader returns PhysicalTableReader for logical TiKVSingleGather. func (sg *TiKVSingleGather) GetPhysicalTableReader(schema *expression.Schema, stats *property.StatsInfo, props ...*property.PhysicalProperty) *PhysicalTableReader { reader := PhysicalTableReader{}.Init(sg.ctx, sg.blockOffset) @@ -94,11 +98,7 @@ func (sg *TiKVSingleGather) GetPhysicalIndexReader(schema *expression.Schema, st // SetChildren overrides PhysicalPlan SetChildren interface. func (p *PhysicalTableReader) SetChildren(children ...PhysicalPlan) { p.tablePlan = children[0] - if p.StoreType == kv.TiFlash { - p.TablePlans = append(p.TablePlans, p.tablePlan) - } else { - p.TablePlans = flattenPushDownPlan(p.tablePlan) - } + p.TablePlans = flattenPushDownPlan(p.tablePlan) } // PhysicalIndexReader is the index reader in tidb. From 7b6ad95444c5c50461e01fbb2d365c95a0941388 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Thu, 5 Mar 2020 13:37:00 +0800 Subject: [PATCH 07/61] fix --- planner/core/plan_to_pb.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index f137ac1ea12ad..4fb9f68dacc05 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -189,6 +189,12 @@ func (p *PhysicalBroadCastJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreT client := ctx.GetClient() leftJoinKeys := make([]expression.Expression, 0, len(p.LeftJoinKeys)) rightJoinKeys := make([]expression.Expression, 0, len(p.RightJoinKeys)) + for _, leftKey := range p.LeftJoinKeys { + leftJoinKeys = append(leftJoinKeys, leftKey) + } + for _, rightKey := range p.RightJoinKeys { + rightJoinKeys = append(rightJoinKeys, rightKey) + } lChildren, err := p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) From 4aae81bec81b682ac694ecb72c4fda0a544d151d Mon Sep 17 00:00:00 2001 From: Fei Han Date: Sat, 7 Mar 2020 21:32:53 +0800 Subject: [PATCH 08/61] add hint --- go.mod | 2 ++ go.sum | 2 ++ planner/core/exhaust_physical_plans.go | 3 +++ planner/core/hints.go | 2 ++ planner/core/logical_plan_builder.go | 12 +++++++++++- planner/core/logical_plans.go | 1 + planner/core/planbuilder.go | 5 +++++ 7 files changed, 26 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index f3e1a35cd30b6..dfb20024aa6ee 100644 --- a/go.mod +++ b/go.mod @@ -2,6 +2,8 @@ module github.com/pingcap/tidb replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200304091852-aa600db6e32e +replace github.com/pingcap/parser v0.0.0-20200305120128-bde9faa0df84 => github.com/hanfei1991/parser v0.0.0-20200307110717-8b60f336a538 + require ( github.com/BurntSushi/toml v0.3.1 github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d diff --git a/go.sum b/go.sum index e845ddd6f8df8..4b8dec2b596d6 100644 --- a/go.sum +++ b/go.sum @@ -126,6 +126,8 @@ github.com/grpc-ecosystem/grpc-gateway v1.9.5 h1:UImYN5qQ8tuGpGE16ZmjvcTtTw24zw1 github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.12.1 h1:zCy2xE9ablevUOrUZc3Dl72Dt+ya2FNAvC2yLYMHzi4= github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqCisSPWTxCZ7sBRjU6iH9c= +github.com/hanfei1991/parser v0.0.0-20200307110717-8b60f336a538 h1:s7PnMZNlVsFmju3e8qfNqERDf7Re7Who8fmEpvWgh3o= +github.com/hanfei1991/parser v0.0.0-20200307110717-8b60f336a538/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/tipb v0.0.0-20200226132147-3fc41406b29e h1:3O9Lc15gzfOWE+CQgsC42rH80GNgwJjPe9GOaT/ZPlk= github.com/hanfei1991/tipb v0.0.0-20200226132147-3fc41406b29e/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hanfei1991/tipb v0.0.0-20200303064518-fd5903ecf553 h1:WMXryw+8ikrQE/5ZsTcOdnA1KxqJCP7Ds2BGdpoH5D4= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 5aa791522a20c..cae97dd4392a9 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1410,6 +1410,9 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph broadCastJoins := p.tryToGetBroadCastJoin(prop) joins = append(joins, broadCastJoins...) + if (p.preferJoinType & preferBCJoin) > 0 { + return mergeJoins + } return joins } diff --git a/planner/core/hints.go b/planner/core/hints.go index 490f06ab9b107..fc74f541dbbeb 100644 --- a/planner/core/hints.go +++ b/planner/core/hints.go @@ -332,6 +332,8 @@ func genHintsFromPhysicalPlan(p PhysicalPlan, nodeType nodeType) (res []*ast.Tab }) case *PhysicalMergeJoin: res = append(res, getJoinHints(p.SCtx(), HintSMJ, p.SelectBlockOffset(), nodeType, pp.children...)...) + case *PhysicalBroadCastJoin: + res = append(res, getJoinHints(p.SCtx(), HintBCJ, p.SelectBlockOffset(), nodeType, pp.children...)...) case *PhysicalHashJoin: res = append(res, getJoinHints(p.SCtx(), HintHJ, p.SelectBlockOffset(), nodeType, pp.children...)...) case *PhysicalIndexJoin: diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 3142c3bf72e0c..05602b5f14887 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -56,6 +56,9 @@ const ( TiDBMergeJoin = "tidb_smj" // HintSMJ is hint enforce merge join. HintSMJ = "sm_join" + + TiDBBroadCastJoin = "tidb_bcj" + HintBCJ = "bc_join" // TiDBIndexNestedLoopJoin is hint enforce index nested loop join. TiDBIndexNestedLoopJoin = "tidb_inlj" // HintINLJ is hint enforce index nested loop join. @@ -396,6 +399,9 @@ func (p *LogicalJoin) setPreferredJoinType(hintInfo *tableHintInfo) { if hintInfo.ifPreferMergeJoin(lhsAlias, rhsAlias) { p.preferJoinType |= preferMergeJoin } + if hintInfo.ifPreferBroadcastJoin(lhsAlias, rhsAlias) { + p.preferJoinType |= preferBCJoin + } if hintInfo.ifPreferHashJoin(lhsAlias, rhsAlias) { p.preferJoinType |= preferHashJoin } @@ -2187,7 +2193,7 @@ func (b *PlanBuilder) unfoldWildStar(p LogicalPlan, selectFields []*ast.SelectFi func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, nodeType nodeType, currentLevel int) { hints = b.hintProcessor.getCurrentStmtHints(hints, nodeType, currentLevel) var ( - sortMergeTables, INLJTables, INLHJTables, INLMJTables, hashJoinTables []hintTableInfo + sortMergeTables, INLJTables, INLHJTables, INLMJTables, hashJoinTables, BCTables []hintTableInfo indexHintList, indexMergeHintList []indexHintInfo tiflashTables, tikvTables []hintTableInfo aggHints aggHintInfo @@ -2197,6 +2203,8 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, nodeType n switch hint.HintName.L { case TiDBMergeJoin, HintSMJ: sortMergeTables = append(sortMergeTables, tableNames2HintTableInfo(b.ctx, hint.Tables, b.hintProcessor, nodeType, currentLevel)...) + case TiDBBroadCastJoin, HintBCJ: + BCTables = append(BCTables, tableNames2HintTableInfo(b.ctx, hint.Tables, b.hintProcessor, nodeType, currentLevel)...) case TiDBIndexNestedLoopJoin, HintINLJ: INLJTables = append(INLJTables, tableNames2HintTableInfo(b.ctx, hint.Tables, b.hintProcessor, nodeType, currentLevel)...) case HintINLHJ: @@ -2269,6 +2277,7 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, nodeType n } b.tableHintInfo = append(b.tableHintInfo, tableHintInfo{ sortMergeJoinTables: sortMergeTables, + broadcastJoinTables: BCTables, indexNestedLoopJoinTables: indexNestedLoopJoinTables{INLJTables, INLHJTables, INLMJTables}, hashJoinTables: hashJoinTables, indexHintList: indexHintList, @@ -2286,6 +2295,7 @@ func (b *PlanBuilder) popTableHints() { b.appendUnmatchedJoinHintWarning(HintINLHJ, "", hintInfo.indexNestedLoopJoinTables.inlhjTables) b.appendUnmatchedJoinHintWarning(HintINLMJ, "", hintInfo.indexNestedLoopJoinTables.inlmjTables) b.appendUnmatchedJoinHintWarning(HintSMJ, TiDBMergeJoin, hintInfo.sortMergeJoinTables) + b.appendUnmatchedJoinHintWarning(HintBCJ, TiDBBroadCastJoin, hintInfo.broadcastJoinTables) b.appendUnmatchedJoinHintWarning(HintHJ, TiDBHashJoin, hintInfo.hashJoinTables) b.tableHintInfo = b.tableHintInfo[:len(b.tableHintInfo)-1] } diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 2632b25702ce3..a653fd08f44ff 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -106,6 +106,7 @@ const ( preferRightAsINLMJInner preferHashJoin preferMergeJoin + preferBCJoin preferHashAgg preferStreamAgg ) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index f5138db782f68..a10686bfaf6e4 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -70,6 +70,7 @@ type indexNestedLoopJoinTables struct { type tableHintInfo struct { indexNestedLoopJoinTables sortMergeJoinTables []hintTableInfo + broadcastJoinTables []hintTableInfo hashJoinTables []hintTableInfo indexHintList []indexHintInfo tiflashTables []hintTableInfo @@ -128,6 +129,10 @@ func (info *tableHintInfo) ifPreferMergeJoin(tableNames ...*hintTableInfo) bool return info.matchTableName(tableNames, info.sortMergeJoinTables) } +func (info *tableHintInfo) ifPreferBroadcastJoin(tableNames ...*hintTableInfo) bool { + return info.matchTableName(tableNames, info.broadcastJoinTables) +} + func (info *tableHintInfo) ifPreferHashJoin(tableNames ...*hintTableInfo) bool { return info.matchTableName(tableNames, info.hashJoinTables) } From 7c49728515855ab684a92228860094e039d27d78 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Sat, 7 Mar 2020 21:58:10 +0800 Subject: [PATCH 09/61] refine hint --- go.mod | 2 +- go.sum | 2 ++ planner/core/exhaust_physical_plans.go | 4 +++- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index dfb20024aa6ee..599c23b4834dc 100644 --- a/go.mod +++ b/go.mod @@ -2,7 +2,7 @@ module github.com/pingcap/tidb replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200304091852-aa600db6e32e -replace github.com/pingcap/parser v0.0.0-20200305120128-bde9faa0df84 => github.com/hanfei1991/parser v0.0.0-20200307110717-8b60f336a538 +replace github.com/pingcap/parser v0.0.0-20200305120128-bde9faa0df84 => github.com/hanfei1991/parser v0.0.0-20200307135251-fda93fd3e461 require ( github.com/BurntSushi/toml v0.3.1 diff --git a/go.sum b/go.sum index 4b8dec2b596d6..342c5dc80b2ef 100644 --- a/go.sum +++ b/go.sum @@ -128,6 +128,8 @@ github.com/grpc-ecosystem/grpc-gateway v1.12.1 h1:zCy2xE9ablevUOrUZc3Dl72Dt+ya2F github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqCisSPWTxCZ7sBRjU6iH9c= github.com/hanfei1991/parser v0.0.0-20200307110717-8b60f336a538 h1:s7PnMZNlVsFmju3e8qfNqERDf7Re7Who8fmEpvWgh3o= github.com/hanfei1991/parser v0.0.0-20200307110717-8b60f336a538/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/hanfei1991/parser v0.0.0-20200307135251-fda93fd3e461 h1:0PegpI/0sFxkvKSRj8MT6NG4DpqoT7oVWMnO9hyBd9U= +github.com/hanfei1991/parser v0.0.0-20200307135251-fda93fd3e461/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/tipb v0.0.0-20200226132147-3fc41406b29e h1:3O9Lc15gzfOWE+CQgsC42rH80GNgwJjPe9GOaT/ZPlk= github.com/hanfei1991/tipb v0.0.0-20200226132147-3fc41406b29e/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hanfei1991/tipb v0.0.0-20200303064518-fd5903ecf553 h1:WMXryw+8ikrQE/5ZsTcOdnA1KxqJCP7Ds2BGdpoH5D4= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index cae97dd4392a9..bb127491e8e10 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1404,6 +1404,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph hashJoins := p.getHashJoins(prop) if (p.preferJoinType & preferHashJoin) > 0 { + logutil.BgLogger().Info("prefer hash join") return hashJoins } joins = append(joins, hashJoins...) @@ -1411,7 +1412,8 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph broadCastJoins := p.tryToGetBroadCastJoin(prop) joins = append(joins, broadCastJoins...) if (p.preferJoinType & preferBCJoin) > 0 { - return mergeJoins + logutil.BgLogger().Info("prefer bc join") + return broadCastJoins } return joins } From a0a88f392981f62c4f2aeb9e115555bff0a9b3b6 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Sun, 8 Mar 2020 21:44:05 +0800 Subject: [PATCH 10/61] add ranges --- go.mod | 2 +- go.sum | 4 ++++ planner/core/exhaust_physical_plans.go | 2 +- planner/core/plan_to_pb.go | 7 +++++++ planner/core/task.go | 2 +- 5 files changed, 14 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 599c23b4834dc..8ee4ad7c29b8a 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module github.com/pingcap/tidb -replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200304091852-aa600db6e32e +replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00 replace github.com/pingcap/parser v0.0.0-20200305120128-bde9faa0df84 => github.com/hanfei1991/parser v0.0.0-20200307135251-fda93fd3e461 diff --git a/go.sum b/go.sum index 342c5dc80b2ef..a650c368076f7 100644 --- a/go.sum +++ b/go.sum @@ -152,6 +152,10 @@ github.com/hanfei1991/tipb v0.0.0-20200304084323-938d55c99bfc h1:L9YrRlM/u5GBn0h github.com/hanfei1991/tipb v0.0.0-20200304084323-938d55c99bfc/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hanfei1991/tipb v0.0.0-20200304091852-aa600db6e32e h1:Or0qTCdPJ/NBxsqYyTLY2c93ZZOfLpLqBLLGQhWn4lE= github.com/hanfei1991/tipb v0.0.0-20200304091852-aa600db6e32e/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200308103716-ec2b2d0a9393 h1:4EYXTYqvDmOzjgZ+n7yHpG0+FGG9hQS5fz6O7i5x8SU= +github.com/hanfei1991/tipb v0.0.0-20200308103716-ec2b2d0a9393/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00 h1:wRO6S3mWrAcoS78mGn1hvjPQjqTgA+MBzbq5NkKRQRM= +github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index bb127491e8e10..b547c8faef563 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1412,7 +1412,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph broadCastJoins := p.tryToGetBroadCastJoin(prop) joins = append(joins, broadCastJoins...) if (p.preferJoinType & preferBCJoin) > 0 { - logutil.BgLogger().Info("prefer bc join") + logutil.BgLogger().Info("prefer bc join", zap.Int("bc count", len(broadCastJoins))) return broadCastJoins } return joins diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 4fb9f68dacc05..97071d6ced9c7 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -16,6 +16,7 @@ package core import ( "github.com/pingcap/errors" "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" @@ -24,6 +25,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" ) @@ -130,6 +132,11 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) } if storeType == kv.TiFlash && p.StoreType == kv.TiKV { tsExec.NextReadEngine = tipb.EngineType_TiKV + ranges := distsql.TableRangesToKVRanges(tsExec.TableId, p.Ranges, nil) + for _, keyRange := range ranges { + tsExec.Ranges = append(tsExec.Ranges, tipb.KeyRange{Low:keyRange.StartKey, High:keyRange.EndKey}) + } + logutil.BgLogger().Info("make range for table.") } err := SetPBColumnsDefaultValue(ctx, tsExec.Columns, p.Columns) return &tipb.Executor{Tp: tipb.ExecType_TypeTableScan, TblScan: tsExec}, err diff --git a/planner/core/task.go b/planner/core/task.go index e133ca5befdb3..4353a6a84ea35 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -100,7 +100,7 @@ func (t *copTask) copy() task { } func (t *copTask) plan() PhysicalPlan { - logutil.BgLogger().Info("index finish", zap.Bool("index finish", t.indexPlanFinished), zap.Bool("table is nil?", t.tablePlan == nil)) + //logutil.BgLogger().Info("index finish", zap.Bool("index finish", t.indexPlanFinished), zap.Bool("table is nil?", t.tablePlan == nil)) if t.indexPlanFinished { return t.tablePlan } From e90731b868cc3688cf34c5dbc6afe014feb1b427 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Mon, 9 Mar 2020 13:15:04 +0800 Subject: [PATCH 11/61] fix --- executor/builder.go | 2 +- planner/core/physical_plans.go | 15 +++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/executor/builder.go b/executor/builder.go index 9a5aebe55d0b7..49f54ef09d4d8 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2235,7 +2235,7 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) * return nil } - ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) + ts := v.GetTableScan() ret.ranges = ts.Ranges sctx := b.ctx.GetSessionVars().StmtCtx sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 5391b921cf5e2..c1640835a2903 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -77,6 +77,21 @@ func (p *PhysicalTableReader) GetTablePlan() PhysicalPlan { return p.tablePlan } +func (p *PhysicalTableReader) GetTableScan() *PhysicalTableScan { + curPlan := p.tablePlan + for { + chCnt := len(curPlan.Children()) + if chCnt == 0{ + return curPlan.(*PhysicalTableScan) + } else if chCnt == 1 { + curPlan = curPlan.Children()[0] + } else { + join := curPlan.(*PhysicalBroadCastJoin) + curPlan = join.children[1-join.InnerChildIdx] + } + } +} + // GetPhysicalTableReader returns PhysicalTableReader for logical TiKVSingleGather. func (sg *TiKVSingleGather) GetPhysicalTableReader(schema *expression.Schema, stats *property.StatsInfo, props ...*property.PhysicalProperty) *PhysicalTableReader { reader := PhysicalTableReader{}.Init(sg.ctx, sg.blockOffset) From 239cc177f467e697f3dadb6060c57df8df14d3d5 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 10 Mar 2020 17:04:02 +0800 Subject: [PATCH 12/61] fix --- executor/builder.go | 3 ++- planner/core/plan_to_pb.go | 11 +++++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 49f54ef09d4d8..ba8c804ae6c34 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2169,7 +2169,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea if err != nil { return nil, err } - ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) + ts := v.GetTableScan() tbl, _ := b.is.TableByID(ts.Table.ID) isPartition, physicalTableID := ts.IsPartition() if isPartition { @@ -2236,6 +2236,7 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) * } ts := v.GetTableScan() + logutil.BgLogger().Info("build table reader", zap.Int64("my table id", ts.Table.ID), zap.String("table name", ts.Table.Name.L), zap.String("range str", ts.Ranges[0].String())) ret.ranges = ts.Ranges sctx := b.ctx.GetSessionVars().StmtCtx sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 97071d6ced9c7..238f651d54ca1 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -45,7 +45,7 @@ func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) ( for _, aggFunc := range p.AggFuncs { aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc)) } - if (storeType == kv.TiFlash) { + if storeType == kv.TiFlash { var err error aggExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { @@ -56,7 +56,7 @@ func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) ( } // ToPB implements PhysicalPlan ToPB interface. -func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb.Executor, error) { +func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { sc := ctx.GetSessionVars().StmtCtx client := ctx.GetClient() aggExec := &tipb.Aggregation{ @@ -65,6 +65,13 @@ func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb. for _, aggFunc := range p.AggFuncs { aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc)) } + if storeType == kv.TiFlash { + var err error + aggExec.Child, err = p.children[0].ToPB(ctx, storeType) + if err != nil { + return nil, errors.Trace(err) + } + } return &tipb.Executor{Tp: tipb.ExecType_TypeStreamAgg, Aggregation: aggExec}, nil } From 00d417ad4e2f0b54b2f07766e65eb1a8962b2ff9 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 10 Mar 2020 21:27:36 +0800 Subject: [PATCH 13/61] fix push down --- planner/core/task.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/planner/core/task.go b/planner/core/task.go index 4353a6a84ea35..11e54ec7f05c3 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -153,6 +153,9 @@ func (t *copTask) getStoreType() kv.StoreType { } tp := t.tablePlan for len(tp.Children()) > 0 { + if len(tp.Children()) > 1 { + return kv.TiFlash + } tp = tp.Children()[0] } if ts, ok := tp.(*PhysicalTableScan); ok { From deb6fd2359c3c3d9cba3cb464b2414afec11a538 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Wed, 11 Mar 2020 21:02:56 +0800 Subject: [PATCH 14/61] fix index --- planner/core/resolve_indices.go | 42 +++++++++++++++++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index 207b0b7000b8c..e0306018d3e12 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -113,6 +113,48 @@ func (p *PhysicalHashJoin) ResolveIndices() (err error) { return } +func (p *PhysicalBroadCastJoin) ResolveIndices() (err error) { + err = p.physicalSchemaProducer.ResolveIndices() + if err != nil { + return err + } + lSchema := p.children[0].Schema() + rSchema := p.children[1].Schema() + for i, col := range p.LeftJoinKeys { + newKey, err := col.ResolveIndices(lSchema) + if err != nil { + return err + } + p.LeftJoinKeys[i] = newKey.(*expression.Column) + } + for i, col := range p.RightJoinKeys { + newKey, err := col.ResolveIndices(rSchema) + if err != nil { + return err + } + p.RightJoinKeys[i] = newKey.(*expression.Column) + } + for i, expr := range p.LeftConditions { + p.LeftConditions[i], err = expr.ResolveIndices(lSchema) + if err != nil { + return err + } + } + for i, expr := range p.RightConditions { + p.RightConditions[i], err = expr.ResolveIndices(rSchema) + if err != nil { + return err + } + } + for i, expr := range p.OtherConditions { + p.OtherConditions[i], err = expr.ResolveIndices(expression.MergeSchema(lSchema, rSchema)) + if err != nil { + return err + } + } + return +} + // ResolveIndices implements Plan interface. func (p *PhysicalMergeJoin) ResolveIndices() (err error) { err = p.physicalSchemaProducer.ResolveIndices() From 55477c767fa84b61e2c6bcf27e0a5f85546d3417 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Thu, 12 Mar 2020 13:05:47 +0800 Subject: [PATCH 15/61] enable distsql for join --- distsql/request_builder.go | 3 +++ kv/kv.go | 4 ++-- store/tikv/coprocessor.go | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index cf15b51b9fd48..d6825903f8b11 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -78,6 +78,9 @@ func (builder *RequestBuilder) SetDAGRequest(dag *tipb.DAGRequest) *RequestBuild builder.Request.Tp = kv.ReqTypeDAG builder.Request.Cacheable = true builder.Request.Data, builder.err = dag.Marshal() + if dag.Executors[0].Tp == tipb.ExecType_TypeJoin { + builder.CopTaskBatch = true + } } return builder diff --git a/kv/kv.go b/kv/kv.go index fb4ed36528066..5357751b21550 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -293,8 +293,8 @@ type Request struct { Cacheable bool // SchemaVer is for any schema-ful storage to validate schema correctness if necessary. SchemaVar int64 - // CopTaskBatchSize is the max batch size for cop task - CopTaskBatchSize int + // CopTaskBatch means whether send cop in batch. + CopTaskBatch bool } // ResultSubset represents a result subset from a single storage unit. diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 8f62022c415d2..114f459df1cd1 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -52,7 +52,7 @@ type CopClient struct { // Send builds the request and gets the coprocessor iterator response. func (c *CopClient) Send(ctx context.Context, req *kv.Request, vars *kv.Variables) kv.Response { - if req.StoreType == kv.TiFlash { + if req.StoreType == kv.TiFlash && req.CopTaskBatch{ logutil.BgLogger().Info("send batch requests") return c.SendBatch(ctx, req, vars) } From 1dbbb564cb8703c39126b2ec6b9371030376c31c Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 10 Mar 2020 20:02:24 +0800 Subject: [PATCH 16/61] add a session var to disable/enable broadcast join --- planner/core/exhaust_physical_plans.go | 12 +++++++----- sessionctx/variable/session.go | 5 +++++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 3 +++ sessionctx/variable/varsutil.go | 2 +- 5 files changed, 17 insertions(+), 6 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b547c8faef563..8130777d27bb2 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1409,11 +1409,13 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph } joins = append(joins, hashJoins...) - broadCastJoins := p.tryToGetBroadCastJoin(prop) - joins = append(joins, broadCastJoins...) - if (p.preferJoinType & preferBCJoin) > 0 { - logutil.BgLogger().Info("prefer bc join", zap.Int("bc count", len(broadCastJoins))) - return broadCastJoins + if p.ctx.GetSessionVars().AllowBCJ { + broadCastJoins := p.tryToGetBroadCastJoin(prop) + joins = append(joins, broadCastJoins...) + if (p.preferJoinType & preferBCJoin) > 0 { + logutil.BgLogger().Info("prefer bc join", zap.Int("bc count", len(broadCastJoins))) + return broadCastJoins + } } return joins } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index d2521e89a0351..383db266cc3c5 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -384,6 +384,8 @@ type SessionVars struct { // AllowAggPushDown can be set to false to forbid aggregation push down. AllowAggPushDown bool + AllowBCJ bool + // AllowWriteRowID can be set to false to forbid write data to _tidb_rowid. // This variable is currently not recommended to be turned on. AllowWriteRowID bool @@ -632,6 +634,7 @@ func NewSessionVars() *SessionVars { Status: mysql.ServerStatusAutocommit, StmtCtx: new(stmtctx.StatementContext), AllowAggPushDown: false, + AllowBCJ: false, OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, RetryLimit: DefTiDBRetryLimit, DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, @@ -1009,6 +1012,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.SkipUTF8Check = TiDBOptOn(val) case TiDBOptAggPushDown: s.AllowAggPushDown = TiDBOptOn(val) + case TiDBOptBCJ: + s.AllowBCJ = TiDBOptOn(val) case TiDBOptWriteRowID: s.AllowWriteRowID = TiDBOptOn(val) case TiDBOptInSubqToJoinAndAgg: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 5d17a9daa5715..df33cb5a72288 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -608,6 +608,7 @@ var defaultSysVars = []*SysVar{ /* TiDB specific variables */ {ScopeSession, TiDBSnapshot, ""}, {ScopeSession, TiDBOptAggPushDown, BoolToIntStr(DefOptAggPushDown)}, + {ScopeSession, TiDBOptBCJ, BoolToIntStr(DefOptBCJ)}, {ScopeSession, TiDBOptWriteRowID, BoolToIntStr(DefOptWriteRowID)}, {ScopeGlobal | ScopeSession, TiDBBuildStatsConcurrency, strconv.Itoa(DefBuildStatsConcurrency)}, {ScopeGlobal, TiDBAutoAnalyzeRatio, strconv.FormatFloat(DefAutoAnalyzeRatio, 'f', -1, 64)}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 28d73102cc5f2..0278eff8d1bd8 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -45,6 +45,8 @@ const ( // tidb_opt_agg_push_down is used to enable/disable the optimizer rule of aggregation push down. TiDBOptAggPushDown = "tidb_opt_agg_push_down" + TiDBOptBCJ = "tidb_opt_broadcast_join" + // tidb_opt_write_row_id is used to enable/disable the operations of insert、replace and update to _tidb_rowid. TiDBOptWriteRowID = "tidb_opt_write_row_id" @@ -379,6 +381,7 @@ const ( DefChecksumTableConcurrency = 4 DefSkipUTF8Check = false DefOptAggPushDown = false + DefOptBCJ = false DefOptWriteRowID = false DefOptCorrelationThreshold = 0.9 DefOptCorrelationExpFactor = 1 diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index f841da222e35a..057f33f433f98 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -391,7 +391,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return "1", nil } return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) - case TiDBSkipUTF8Check, TiDBOptAggPushDown, + case TiDBSkipUTF8Check, TiDBOptAggPushDown, TiDBOptBCJ, TiDBOptInSubqToJoinAndAgg, TiDBEnableFastAnalyze, TiDBBatchInsert, TiDBDisableTxnAutoRetry, TiDBEnableStreaming, TiDBEnableChunkRPC, TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction, TiDBPProfSQLCPU, From c4b2e13b132c857a065517d34a419e533becbfd7 Mon Sep 17 00:00:00 2001 From: xufei Date: Fri, 13 Mar 2020 09:19:58 +0800 Subject: [PATCH 17/61] fix bug --- distsql/request_builder.go | 31 ++++++++++++++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index d6825903f8b11..edc8e42d6078a 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -72,13 +72,42 @@ func (builder *RequestBuilder) SetTableHandles(tid int64, handles []int64) *Requ return builder } +func hasJoinNode(executor *tipb.Executor) bool { + if executor.Tp == tipb.ExecType_TypeJoin { + return true + } + switch executor.Tp { + case tipb.ExecType_TypeAggregation: + if executor.Aggregation.Child != nil { + return hasJoinNode(executor.Aggregation.Child) + } + return false + case tipb.ExecType_TypeSelection: + if executor.Selection.Child != nil { + return hasJoinNode(executor.Selection.Child) + } + return false + case tipb.ExecType_TypeLimit: + if executor.Limit.Child != nil { + return hasJoinNode(executor.Limit.Child) + } + return false + case tipb.ExecType_TypeTopN: + if executor.TopN.Child != nil { + return hasJoinNode(executor.TopN.Child) + } + return false + default: + return false + } +} // SetDAGRequest sets the request type to "ReqTypeDAG" and construct request data. func (builder *RequestBuilder) SetDAGRequest(dag *tipb.DAGRequest) *RequestBuilder { if builder.err == nil { builder.Request.Tp = kv.ReqTypeDAG builder.Request.Cacheable = true builder.Request.Data, builder.err = dag.Marshal() - if dag.Executors[0].Tp == tipb.ExecType_TypeJoin { + if hasJoinNode(dag.Executors[0]) { builder.CopTaskBatch = true } } From 2c92a299d24afc345364d2f0d9a057e749289832 Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 16 Mar 2020 09:24:03 +0800 Subject: [PATCH 18/61] fix bug --- distsql/request_builder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index edc8e42d6078a..c4799a61c0cd4 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -77,7 +77,7 @@ func hasJoinNode(executor *tipb.Executor) bool { return true } switch executor.Tp { - case tipb.ExecType_TypeAggregation: + case tipb.ExecType_TypeAggregation, tipb.ExecType_TypeStreamAgg: if executor.Aggregation.Child != nil { return hasJoinNode(executor.Aggregation.Child) } From 05ade28897768c031679568ee50fe003360557b2 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Mon, 16 Mar 2020 12:32:23 +0800 Subject: [PATCH 19/61] tiny fix --- executor/table_reader.go | 5 +++++ store/tikv/batch_coprocessor.go | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/executor/table_reader.go b/executor/table_reader.go index 8cea2e21b8bf2..45b305481c01c 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -217,6 +217,11 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra SetMemTracker(e.memTracker). SetStoreType(e.storeType). Build() + for _, p := range e.plans { + if len(p.Children()) > 1 { + kvReq.CopTaskBatch = true + } + } if err != nil { return nil, err } diff --git a/store/tikv/batch_coprocessor.go b/store/tikv/batch_coprocessor.go index a107880ef3ab5..287e9c55b8c16 100644 --- a/store/tikv/batch_coprocessor.go +++ b/store/tikv/batch_coprocessor.go @@ -119,7 +119,7 @@ func buildBatchCopTasks(bo *Backoffer, cache *RegionCache, ranges *copRanges, re ranges: ranges.slice(i, nextI), // Channel buffer is 2 for handling region split. // In a common case, two region split tasks will not be blocked. - respChan: make(chan *copResponse, 1024), + respChan: make(chan *copResponse, 1024 * 1024), cmdType: cmdType, storeType: req.StoreType, }) From 4f14acfd1e2983157bff6674d688cd640733517c Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 16 Mar 2020 17:38:14 +0800 Subject: [PATCH 20/61] enable cast decimal pushdown to tiflash --- expression/expression.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/expression/expression.go b/expression/expression.go index 5e26cd8d8f6eb..7675baa0af973 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -984,6 +984,14 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { ast.Like, ast.UnaryNot, ast.Case, ast.Month, ast.Substr, ast.Substring, ast.TimestampDiff: return true + case ast.Cast: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_CastIntAsDecimal, tipb.ScalarFuncSig_CastRealAsDecimal, + tipb.ScalarFuncSig_CastDecimalAsDecimal: + return true + default: + return false + } default: return false } From cd7f225bac78046e9076f72a5411919b46a7c2e6 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 17 Mar 2020 16:52:02 +0800 Subject: [PATCH 21/61] fix --- store/tikv/batch_coprocessor.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/store/tikv/batch_coprocessor.go b/store/tikv/batch_coprocessor.go index 287e9c55b8c16..43f9f686539e8 100644 --- a/store/tikv/batch_coprocessor.go +++ b/store/tikv/batch_coprocessor.go @@ -119,7 +119,7 @@ func buildBatchCopTasks(bo *Backoffer, cache *RegionCache, ranges *copRanges, re ranges: ranges.slice(i, nextI), // Channel buffer is 2 for handling region split. // In a common case, two region split tasks will not be blocked. - respChan: make(chan *copResponse, 1024 * 1024), + respChan: make(chan *copResponse), cmdType: cmdType, storeType: req.StoreType, }) @@ -146,7 +146,7 @@ func buildBatchCopTasks(bo *Backoffer, cache *RegionCache, ranges *copRanges, re batchCop.regionTaskMap[task.region.id] = task } else { batchTask := &batchCopTask { - respChan : make(chan * batchCopResponse, 10), + respChan : make(chan * batchCopResponse, 2048), storeAddr: rpcCtx.Addr, cmdType : task.cmdType, regionTaskMap: make(map[uint64]*copTask), From 04b4cf694dd336e4f42bcef00ce50e88eef32607 Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 30 Mar 2020 16:58:53 +0800 Subject: [PATCH 22/61] fix bc join bug --- go.mod | 2 +- go.sum | 2 ++ planner/core/exhaust_physical_plans.go | 22 +++++++++++----------- 3 files changed, 14 insertions(+), 12 deletions(-) diff --git a/go.mod b/go.mod index d5edd3b771dd7..c760e0d2e37db 100644 --- a/go.mod +++ b/go.mod @@ -2,7 +2,7 @@ module github.com/pingcap/tidb replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00 -replace github.com/pingcap/parser v0.0.0-20200305120128-bde9faa0df84 => github.com/hanfei1991/parser v0.0.0-20200307135251-fda93fd3e461 +replace github.com/pingcap/parser v0.0.0-20200326020624-68d423641be5 => github.com/windtalker/parser v0.0.0-20200330084704-cc2bee1a3633 require ( github.com/BurntSushi/toml v0.3.1 diff --git a/go.sum b/go.sum index 71c1d30bcdd8f..18339eb8ba9e4 100644 --- a/go.sum +++ b/go.sum @@ -375,6 +375,8 @@ github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijb github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/urfave/negroni v0.3.0 h1:PaXOb61mWeZJxc1Ji2xJjpVg9QfPo0rrB+lHyBxGNSU= github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= +github.com/windtalker/parser v0.0.0-20200330084704-cc2bee1a3633 h1:ubnmco1ONtSC6CeIuV+b7l+dlvJcNMkPTZwdQruzAUo= +github.com/windtalker/parser v0.0.0-20200330084704-cc2bee1a3633/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b891aaf94af81..cb6d0b99afdb1 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1449,21 +1449,21 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph func (p *LogicalJoin) tryToGetBroadCastJoin(prop * property.PhysicalProperty) []PhysicalPlan { child0, ok0 := p.children[0].(*DataSource) - if (!ok0) { - return nil; + if !ok0 { + return nil } child1, ok1 := p.children[1].(*DataSource) - if (!ok1) { - return nil; + if !ok1 { + return nil } if !prop.IsEmpty() { - return nil; + return nil } - if (prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiflashTaskType) { + if prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiflashTaskType { return nil } - if (p.JoinType != InnerJoin || len(p.LeftConditions) != 0 || len(p.RightConditions) != 0) { + if p.JoinType != InnerJoin || len(p.LeftConditions) != 0 || len(p.RightConditions) != 0 { return nil } @@ -1477,17 +1477,17 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop * property.PhysicalProperty) [] RightJoinKeys: rkeys, } if child0.stats.Count() < child1.stats.Count() { - baseJoin.InnerChildIdx = 0; + baseJoin.InnerChildIdx = 0 } else { - baseJoin.InnerChildIdx = 1; + baseJoin.InnerChildIdx = 1 } childrenReqProps := make([]*property.PhysicalProperty, 2) - childrenReqProps[baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopSingleReadTaskType} + childrenReqProps[baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiflashTaskType} childrenReqProps[1-baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiflashTaskType} join := PhysicalBroadCastJoin { basePhysicalJoin: baseJoin, }.Init(p.ctx, p.stats, p.blockOffset, childrenReqProps...) - results := make([]PhysicalPlan, 0, 1); + results := make([]PhysicalPlan, 0, 1) results = append(results, join) return results } From 40e39909b452c5f01ede33d43d304112b6a5b917 Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 31 Mar 2020 14:03:04 +0800 Subject: [PATCH 23/61] make broadcast plan stable --- planner/core/exhaust_physical_plans.go | 14 +++++++++----- planner/core/find_best_task.go | 21 ++++++++++++++++++--- planner/core/physical_plans.go | 2 ++ planner/core/plan_to_pb.go | 4 ++-- planner/core/planbuilder.go | 1 + planner/property/task_type.go | 14 +++++++++++--- planner/util/path.go | 3 +++ 7 files changed, 46 insertions(+), 13 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index cb6d0b99afdb1..d30731bf9e1d9 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1459,11 +1459,11 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop * property.PhysicalProperty) [] if !prop.IsEmpty() { return nil } - if prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiflashTaskType { + if prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiFlashLocalReadTaskType && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { return nil } - if p.JoinType != InnerJoin || len(p.LeftConditions) != 0 || len(p.RightConditions) != 0 { + if p.JoinType != InnerJoin || len(p.LeftConditions) != 0 || len(p.RightConditions) != 0 || len(p.OtherConditions) != 0 { return nil } @@ -1482,8 +1482,12 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop * property.PhysicalProperty) [] baseJoin.InnerChildIdx = 1 } childrenReqProps := make([]*property.PhysicalProperty, 2) - childrenReqProps[baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiflashTaskType} - childrenReqProps[1-baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiflashTaskType} + childrenReqProps[baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} + if prop.TaskTp == property.CopTiFlashGlobalReadTaskType { + childrenReqProps[1-baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} + } else { + childrenReqProps[1-baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiFlashLocalReadTaskType} + } join := PhysicalBroadCastJoin { basePhysicalJoin: baseJoin, }.Init(p.ctx, p.stats, p.blockOffset, childrenReqProps...) @@ -1718,7 +1722,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy return nil } hashAggs := make([]PhysicalPlan, 0, len(wholeTaskTypes)) - taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType, property.CopTiflashTaskType} + taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType, property.CopTiFlashLocalReadTaskType, property.CopTiFlashGlobalReadTaskType} if !la.aggHints.preferAggToCop { taskTypes = append(taskTypes, property.RootTaskType) } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 8898038e7e39f..bd3c0ac9b3745 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -362,12 +362,26 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida if len(path.Ranges) == 0 && !ds.ctx.GetSessionVars().StmtCtx.UseCache { return []*candidatePath{{path: path}} } - if path.StoreType != kv.TiFlash && prop.TaskTp == property.CopTiflashTaskType { + if path.StoreType != kv.TiFlash && (prop.TaskTp == property.CopTiFlashLocalReadTaskType || prop.TaskTp == property.CopTiFlashGlobalReadTaskType) { continue } var currentCandidate *candidatePath if path.IsTablePath { - currentCandidate = ds.getTableCandidate(path, prop) + if path.StoreType == kv.TiFlash { + if path.IsRemoteRead && prop.TaskTp == property.CopTiFlashGlobalReadTaskType { + currentCandidate = ds.getTableCandidate(path, prop) + } + if !path.IsRemoteRead && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { + currentCandidate = ds.getTableCandidate(path, prop) + } + } else { + if !path.IsRemoteRead { + currentCandidate = ds.getTableCandidate(path, prop) + } + } + if currentCandidate == nil { + continue + } } else { coveredByIdx := isCoveringIndex(ds.schema.Columns, path.FullIdxCols, path.FullIdxColLens, ds.tableInfo.PKIsHandle) if len(path.AccessConds) > 0 || !prop.IsEmpty() || path.Forced || coveredByIdx { @@ -1097,7 +1111,7 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid cst: cost, } if ts.StoreType == kv.TiFlash { - if prop.TaskTp != property.CopTiflashTaskType && prop.TaskTp != property.RootTaskType { + if prop.TaskTp != property.CopTiFlashLocalReadTaskType && prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { logutil.BgLogger().Error("invalid task:", zap.String("type", prop.TaskTp.String())) return invalidTask, nil } @@ -1282,6 +1296,7 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper AccessCondition: path.AccessConds, filterCondition: path.TableFilters, StoreType: path.StoreType, + IsGlobalRead: path.IsRemoteRead, }.Init(ds.ctx, ds.blockOffset) ts.SetSchema(ds.schema.Clone()) if ts.Table.PKIsHandle { diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index b9448267b6e12..7095fd3c65be8 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -265,6 +265,8 @@ type PhysicalTableScan struct { StoreType kv.StoreType + IsGlobalRead bool + // The table scan may be a partition, rather than a real table. isPartition bool // KeepOrder is true, if sort data by scanning pkcol, diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 238f651d54ca1..53661190faeba 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -137,8 +137,8 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) if p.isPartition { tsExec.TableId = p.physicalTableID } - if storeType == kv.TiFlash && p.StoreType == kv.TiKV { - tsExec.NextReadEngine = tipb.EngineType_TiKV + if storeType == kv.TiFlash && p.IsGlobalRead { + tsExec.NextReadEngine = tipb.EngineType_TiFlash ranges := distsql.TableRangesToKVRanges(tsExec.TableId, p.Ranges, nil) for _, keyRange := range ranges { tsExec.Ranges = append(tsExec.Ranges, tipb.KeyRange{Low:keyRange.StartKey, High:keyRange.EndKey}) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 73bd0616100cc..5e8ee6b23fc2e 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -691,6 +691,7 @@ func (b *PlanBuilder) getPossibleAccessPaths(indexHints []*ast.IndexHint, tbl ta publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: tp}) if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Available { publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: kv.TiFlash}) + publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: kv.TiFlash, IsRemoteRead: true}) } for _, index := range tblInfo.Indices { if index.State == model.StatePublic { diff --git a/planner/property/task_type.go b/planner/property/task_type.go index f27d2ead3be69..d04e4ae53d8ec 100644 --- a/planner/property/task_type.go +++ b/planner/property/task_type.go @@ -28,7 +28,13 @@ const ( // coprocessor layer. CopDoubleReadTaskType - CopTiflashTaskType + // CopTiFlashLocalReadTaskType stands for flash coprocessor that read data locally, + // and only a part of the data is read in one cop task + CopTiFlashLocalReadTaskType + + // CopTiFlashGlobalReadTaskType stands for flash coprocessor that read data globally + // and all the data of given table will be read in one cop task + CopTiFlashGlobalReadTaskType ) // String implements fmt.Stringer interface. @@ -40,8 +46,10 @@ func (t TaskType) String() string { return "copSingleReadTask" case CopDoubleReadTaskType: return "copDoubleReadTask" - case CopTiflashTaskType: - return "copTiflashTaskType" + case CopTiFlashLocalReadTaskType: + return "copTiFlashLocalReadTask" + case CopTiFlashGlobalReadTaskType: + return "copTiFlashRemoteReadTask" } return "UnknownTaskType" } diff --git a/planner/util/path.go b/planner/util/path.go index bc216a1bf6242..086769a146ab4 100644 --- a/planner/util/path.go +++ b/planner/util/path.go @@ -50,6 +50,9 @@ type AccessPath struct { // IsTablePath indicates whether this path is table path. IsTablePath bool + + // IsGlobalRead indicates whether this path is a remote read path for tiflash + IsRemoteRead bool // Forced means this path is generated by `use/force index()`. Forced bool } From 617fde62180237f50f70473a595d28d10aca7f36 Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 31 Mar 2020 14:07:15 +0800 Subject: [PATCH 24/61] refine code --- planner/core/find_best_task.go | 8 ++++---- planner/core/planbuilder.go | 2 +- planner/property/task_type.go | 2 +- planner/util/path.go | 3 ++- 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index bd3c0ac9b3745..0ea898ae91d58 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -368,14 +368,14 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida var currentCandidate *candidatePath if path.IsTablePath { if path.StoreType == kv.TiFlash { - if path.IsRemoteRead && prop.TaskTp == property.CopTiFlashGlobalReadTaskType { + if path.IsGlobalRead && prop.TaskTp == property.CopTiFlashGlobalReadTaskType { currentCandidate = ds.getTableCandidate(path, prop) } - if !path.IsRemoteRead && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { + if !path.IsGlobalRead && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { currentCandidate = ds.getTableCandidate(path, prop) } } else { - if !path.IsRemoteRead { + if !path.IsGlobalRead { currentCandidate = ds.getTableCandidate(path, prop) } } @@ -1296,7 +1296,7 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper AccessCondition: path.AccessConds, filterCondition: path.TableFilters, StoreType: path.StoreType, - IsGlobalRead: path.IsRemoteRead, + IsGlobalRead: path.IsGlobalRead, }.Init(ds.ctx, ds.blockOffset) ts.SetSchema(ds.schema.Clone()) if ts.Table.PKIsHandle { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 5e8ee6b23fc2e..8ed0c2b681a1f 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -691,7 +691,7 @@ func (b *PlanBuilder) getPossibleAccessPaths(indexHints []*ast.IndexHint, tbl ta publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: tp}) if tblInfo.TiFlashReplica != nil && tblInfo.TiFlashReplica.Available { publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: kv.TiFlash}) - publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: kv.TiFlash, IsRemoteRead: true}) + publicPaths = append(publicPaths, &util.AccessPath{IsTablePath: true, StoreType: kv.TiFlash, IsGlobalRead: true}) } for _, index := range tblInfo.Indices { if index.State == model.StatePublic { diff --git a/planner/property/task_type.go b/planner/property/task_type.go index d04e4ae53d8ec..fb89b6eabff0e 100644 --- a/planner/property/task_type.go +++ b/planner/property/task_type.go @@ -49,7 +49,7 @@ func (t TaskType) String() string { case CopTiFlashLocalReadTaskType: return "copTiFlashLocalReadTask" case CopTiFlashGlobalReadTaskType: - return "copTiFlashRemoteReadTask" + return "copTiFlashGlobalReadTask" } return "UnknownTaskType" } diff --git a/planner/util/path.go b/planner/util/path.go index 086769a146ab4..7555142372252 100644 --- a/planner/util/path.go +++ b/planner/util/path.go @@ -52,7 +52,8 @@ type AccessPath struct { IsTablePath bool // IsGlobalRead indicates whether this path is a remote read path for tiflash - IsRemoteRead bool + IsGlobalRead bool + // Forced means this path is generated by `use/force index()`. Forced bool } From d3ce0bdc0e1323811dc124ee19d897b240939d7e Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 31 Mar 2020 16:36:39 +0800 Subject: [PATCH 25/61] fix bug --- planner/core/find_best_task.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 0ea898ae91d58..20b5748be9411 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -194,6 +194,12 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTas // combine best child tasks with parent physical plan. curTask := pp.attach2Task(childTasks...) + if prop.TaskTp == property.CopTiFlashLocalReadTaskType || prop.TaskTp == property.CopTiFlashGlobalReadTaskType { + if _, ok := curTask.(*copTask); !ok { + continue + } + } + // enforce curTask property if prop.Enforced { curTask = enforceProperty(prop, curTask, p.basePlan.ctx) @@ -1121,6 +1127,9 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid copTask.keepOrder = true } ts.addPushedDownSelection(copTask, ds.stats.ScaleByExpectCnt(prop.ExpectedCnt)) + if (prop.TaskTp == property.CopTiFlashGlobalReadTaskType || prop.TaskTp == property.CopTiFlashLocalReadTaskType) && len(copTask.rootTaskConds) != 0 { + return invalidTask, nil + } if prop.TaskTp == property.RootTaskType { task = finishCopTask(ds.ctx, task) } else if _, ok := task.(*rootTask); ok { From 2580fcad8715d4a2a2439436a83ac1bac64c2fd5 Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 2 Apr 2020 13:37:39 +0800 Subject: [PATCH 26/61] basic support for multi table broadcast join --- planner/core/common_plans.go | 2 + planner/core/exhaust_physical_plans.go | 121 +++++++++++++++++-------- planner/core/explain.go | 3 + planner/core/find_best_task.go | 6 +- planner/core/physical_plans.go | 3 +- planner/property/physical_property.go | 17 ++++ planner/property/task_type.go | 7 +- 7 files changed, 111 insertions(+), 48 deletions(-) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 2f100d1f3237d..18e1b6068a8ce 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -855,6 +855,8 @@ func (e *Explain) explainPlanInRowFormat(p Plan, taskType, driverSide, indent st buildSide = plan.InnerChildIdx ^ 1 case *PhysicalIndexHashJoin: buildSide = plan.InnerChildIdx ^ 1 + case *PhysicalBroadCastJoin: + buildSide = plan.InnerChildIdx } if buildSide != -1 { diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index d30731bf9e1d9..91097515d45a4 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -37,6 +37,9 @@ import ( ) func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { + if prop.IsFlashOnlyProp() { + return nil + } childProp := prop.Clone() us := PhysicalUnionScan{ Conditions: p.conditions, @@ -1416,11 +1419,23 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ // If the hint is not matched, it will get other candidates. // If the hint is not figured, we will pick all candidates. func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { + joins := make([]PhysicalPlan, 0, 5) + if p.ctx.GetSessionVars().AllowBCJ { + broadCastJoins := p.tryToGetBroadCastJoin(prop) + if (p.preferJoinType & preferBCJoin) > 0 { + logutil.BgLogger().Info("prefer bc join", zap.Int("bc count", len(broadCastJoins))) + return broadCastJoins + } + joins = append(joins, broadCastJoins...) + } + if prop.IsFlashOnlyProp() { + return joins + } + mergeJoins := p.GetMergeJoin(prop, p.schema, p.Stats(), p.children[0].statsInfo(), p.children[1].statsInfo()) if (p.preferJoinType & preferMergeJoin) > 0 { return mergeJoins } - joins := make([]PhysicalPlan, 0, 5) joins = append(joins, mergeJoins...) indexJoins, forced := p.tryToGetIndexJoin(prop) @@ -1435,31 +1450,25 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph return hashJoins } joins = append(joins, hashJoins...) - - if p.ctx.GetSessionVars().AllowBCJ { - broadCastJoins := p.tryToGetBroadCastJoin(prop) - joins = append(joins, broadCastJoins...) - if (p.preferJoinType & preferBCJoin) > 0 { - logutil.BgLogger().Info("prefer bc join", zap.Int("bc count", len(broadCastJoins))) - return broadCastJoins - } - } return joins } -func (p *LogicalJoin) tryToGetBroadCastJoin(prop * property.PhysicalProperty) []PhysicalPlan { - child0, ok0 := p.children[0].(*DataSource) - if !ok0 { - return nil +func getAllDataSourceRowCount(plan LogicalPlan) int64 { + if ds, ok := plan.(*DataSource); ok { + return ds.statsInfo().Count() } - child1, ok1 := p.children[1].(*DataSource) - if !ok1 { - return nil + ret := int64(0) + for _, child := range plan.Children() { + ret += getAllDataSourceRowCount(child) } + return ret +} + +func (p *LogicalJoin) tryToGetBroadCastJoin(prop * property.PhysicalProperty) []PhysicalPlan { if !prop.IsEmpty() { return nil } - if prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiFlashLocalReadTaskType && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { + if prop.TaskTp != property.RootTaskType && !prop.IsFlashOnlyProp() { return nil } @@ -1476,20 +1485,28 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop * property.PhysicalProperty) [] LeftJoinKeys: lkeys, RightJoinKeys: rkeys, } - if child0.stats.Count() < child1.stats.Count() { + // todo: currently, build side is the one has less rowcont and global read side + // is the one has less datasource row count(which mean less remote read), need + // to use cbo to decide the build side and global read side + if p.children[0].statsInfo().Count() < p.children[1].statsInfo().Count() { baseJoin.InnerChildIdx = 0 } else { baseJoin.InnerChildIdx = 1 } + globalIndex := baseJoin.InnerChildIdx + if prop.TaskTp != property.CopTiFlashGlobalReadTaskType && getAllDataSourceRowCount(p.children[globalIndex]) > getAllDataSourceRowCount(p.children[1 - globalIndex]) { + globalIndex = 1 - globalIndex + } childrenReqProps := make([]*property.PhysicalProperty, 2) - childrenReqProps[baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} + childrenReqProps[globalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} if prop.TaskTp == property.CopTiFlashGlobalReadTaskType { - childrenReqProps[1-baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} + childrenReqProps[1-globalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} } else { - childrenReqProps[1-baseJoin.InnerChildIdx] = &property.PhysicalProperty{TaskTp: property.CopTiFlashLocalReadTaskType} + childrenReqProps[1-globalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashLocalReadTaskType} } join := PhysicalBroadCastJoin { basePhysicalJoin: baseJoin, + globalChildIndex: globalIndex, }.Init(p.ctx, p.stats, p.blockOffset, childrenReqProps...) results := make([]PhysicalPlan, 0, 1) results = append(results, join) @@ -1500,6 +1517,9 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop * property.PhysicalProperty) [] // When a sort column will be replaced by scalar function, we refuse it. // When a sort column will be replaced by a constant, we just remove it. func (p *LogicalProjection) TryToGetChildProp(prop *property.PhysicalProperty) (*property.PhysicalProperty, bool) { + if prop.IsFlashOnlyProp() { + return nil, false + } newProp := &property.PhysicalProperty{TaskTp: property.RootTaskType, ExpectedCnt: prop.ExpectedCnt} newCols := make([]property.Item, 0, len(prop.Items)) for _, col := range prop.Items { @@ -1529,9 +1549,10 @@ func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty return []PhysicalPlan{proj} } -func (lt *LogicalTopN) getPhysTopN() []PhysicalPlan { - ret := make([]PhysicalPlan, 0, 3) - for _, tp := range wholeTaskTypes { +func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPlan { + allTaskTypes := prop.GetAllPossibleChildTaskTypes() + ret := make([]PhysicalPlan, 0, len(allTaskTypes)) + for _, tp := range allTaskTypes { resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: math.MaxFloat64} topN := PhysicalTopN{ ByItems: lt.ByItems, @@ -1543,14 +1564,15 @@ func (lt *LogicalTopN) getPhysTopN() []PhysicalPlan { return ret } -func (lt *LogicalTopN) getPhysLimits() []PhysicalPlan { - prop, canPass := GetPropByOrderByItems(lt.ByItems) +func (lt *LogicalTopN) getPhysLimits(prop *property.PhysicalProperty) []PhysicalPlan { + p, canPass := GetPropByOrderByItems(lt.ByItems) if !canPass { return nil } + allTaskTypes := prop.GetAllPossibleChildTaskTypes() ret := make([]PhysicalPlan, 0, 3) - for _, tp := range wholeTaskTypes { - resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(lt.Count + lt.Offset), Items: prop.Items} + for _, tp := range allTaskTypes { + resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(lt.Count + lt.Offset), Items: p.Items} limit := PhysicalLimit{ Count: lt.Count, Offset: lt.Offset, @@ -1562,7 +1584,7 @@ func (lt *LogicalTopN) getPhysLimits() []PhysicalPlan { // MatchItems checks if this prop's columns can match by items totally. func MatchItems(p *property.PhysicalProperty, items []*ByItems) bool { - if len(items) < len(p.Items) { + if len(items) < len(p.Items) || p.IsFlashOnlyProp() { return false } for i, col := range p.Items { @@ -1576,7 +1598,7 @@ func MatchItems(p *property.PhysicalProperty, items []*ByItems) bool { func (lt *LogicalTopN) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { if MatchItems(prop, lt.ByItems) { - return append(lt.getPhysTopN(), lt.getPhysLimits()...) + return append(lt.getPhysTopN(prop), lt.getPhysLimits(prop)...) } return nil } @@ -1587,7 +1609,7 @@ func (la *LogicalApply) GetHashJoin(prop *property.PhysicalProperty) *PhysicalHa } func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { - if !prop.AllColsFromSchema(la.children[0].Schema()) { // for convenient, we don't pass through any prop + if !prop.AllColsFromSchema(la.children[0].Schema()) || prop.IsFlashOnlyProp() { // for convenient, we don't pass through any prop return nil } join := la.GetHashJoin(prop) @@ -1604,6 +1626,9 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) [] } func (p *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { + if prop.IsFlashOnlyProp() { + return nil + } var byItems []property.Item byItems = append(byItems, p.PartitionBy...) byItems = append(byItems, p.OrderBy...) @@ -1637,8 +1662,12 @@ func (la *LogicalAggregation) canPushToCop() bool { } func (la *LogicalAggregation) getEnforcedStreamAggs(prop *property.PhysicalProperty) []PhysicalPlan { + if prop.IsFlashOnlyProp() { + return nil + } _, desc := prop.AllSameOrder() - enforcedAggs := make([]PhysicalPlan, 0, len(wholeTaskTypes)) + allTaskTypes := prop.GetAllPossibleChildTaskTypes() + enforcedAggs := make([]PhysicalPlan, 0, len(allTaskTypes)) childProp := &property.PhysicalProperty{ ExpectedCnt: math.Max(prop.ExpectedCnt*la.inputCount/la.stats.RowCount, prop.ExpectedCnt), Enforced: true, @@ -1665,6 +1694,10 @@ func (la *LogicalAggregation) getEnforcedStreamAggs(prop *property.PhysicalPrope } func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []PhysicalPlan { + // todo support CopTiFlash task type in stream agg + if prop.IsFlashOnlyProp() { + return nil + } all, desc := prop.AllSameOrder() if !all { return nil @@ -1680,7 +1713,8 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P return nil } - streamAggs := make([]PhysicalPlan, 0, len(la.possibleProperties)*(len(wholeTaskTypes)-1)+len(wholeTaskTypes)) + allTaskTypes := prop.GetAllPossibleChildTaskTypes() + streamAggs := make([]PhysicalPlan, 0, len(la.possibleProperties)*(len(allTaskTypes)-1)+len(allTaskTypes)) childProp := &property.PhysicalProperty{ ExpectedCnt: math.Max(prop.ExpectedCnt*la.inputCount/la.stats.RowCount, prop.ExpectedCnt), } @@ -1721,11 +1755,14 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy if !prop.IsEmpty() { return nil } - hashAggs := make([]PhysicalPlan, 0, len(wholeTaskTypes)) - taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType, property.CopTiFlashLocalReadTaskType, property.CopTiFlashGlobalReadTaskType} + hashAggs := make([]PhysicalPlan, 0, len(prop.GetAllPossibleChildTaskTypes())) + taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType, property.CopTiFlashLocalReadTaskType} if !la.aggHints.preferAggToCop { taskTypes = append(taskTypes, property.RootTaskType) } + if prop.IsFlashOnlyProp() { + taskTypes = []property.TaskType{prop.TaskTp} + } for _, taskTp := range taskTypes { agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, TaskTp: taskTp}) agg.SetSchema(la.schema.Clone()) @@ -1795,8 +1832,9 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) []P if !prop.IsEmpty() { return nil } - ret := make([]PhysicalPlan, 0, len(wholeTaskTypes)) - for _, tp := range wholeTaskTypes { + allTaskTypes := prop.GetAllPossibleChildTaskTypes() + ret := make([]PhysicalPlan, 0, len(allTaskTypes)) + for _, tp := range allTaskTypes { resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(p.Count + p.Offset)} limit := PhysicalLimit{ Offset: p.Offset, @@ -1808,6 +1846,9 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) []P } func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { + if prop.IsFlashOnlyProp() { + return nil + } childProp := prop.Clone() lock := PhysicalLock{ Lock: p.Lock, @@ -1819,7 +1860,7 @@ func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { // TODO: UnionAll can not pass any order, but we can change it to sort merge to keep order. - if !prop.IsEmpty() { + if !prop.IsEmpty() || prop.IsFlashOnlyProp() { return nil } chReqProps := make([]*property.PhysicalProperty, 0, len(p.children)) @@ -1860,7 +1901,7 @@ func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) []P } func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { - if !prop.IsEmpty() { + if !prop.IsEmpty() || prop.IsFlashOnlyProp() { return nil } mor := PhysicalMaxOneRow{}.Init(p.ctx, p.stats, p.blockOffset, &property.PhysicalProperty{ExpectedCnt: 2}) diff --git a/planner/core/explain.go b/planner/core/explain.go index 126817e8db9ad..7c8bd8094c866 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -213,6 +213,9 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { if p.stats.StatsVersion == statistics.PseudoVersion && !normalized { buffer.WriteString("stats:pseudo, ") } + if p.IsGlobalRead { + buffer.WriteString("global read, ") + } buffer.Truncate(buffer.Len() - 2) return buffer.String() } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 20b5748be9411..bdb0ca4b0b0ec 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -60,10 +60,6 @@ var aggFuncFactor = map[string]float64{ "default": 1.5, } -// wholeTaskTypes records all possible kinds of task that a plan can return. For Agg, TopN and Limit, we will try to get -// these tasks one by one. -var wholeTaskTypes = [...]property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType, property.RootTaskType} - var invalidTask = &rootTask{cst: math.MaxFloat64} // GetPropByOrderByItems will check if this sort property can be pushed or not. In order to simplify the problem, we only @@ -146,7 +142,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTas return bestTask, nil } - if prop.TaskTp != property.RootTaskType { + if prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiFlashLocalReadTaskType && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { // Currently all plan cannot totally push down. p.storeTask(prop, invalidTask) return invalidTask, nil diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 7095fd3c65be8..36d4bc756f6a8 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -87,7 +87,7 @@ func (p *PhysicalTableReader) GetTableScan() *PhysicalTableScan { curPlan = curPlan.Children()[0] } else { join := curPlan.(*PhysicalBroadCastJoin) - curPlan = join.children[1-join.InnerChildIdx] + curPlan = join.children[1-join.globalChildIndex] } } } @@ -438,6 +438,7 @@ type PhysicalMergeJoin struct { type PhysicalBroadCastJoin struct { basePhysicalJoin + globalChildIndex int } // PhysicalLock is the physical operator of lock, which is used for `select ... for update` clause. diff --git a/planner/property/physical_property.go b/planner/property/physical_property.go index 38da639c71942..62612ec6bac63 100644 --- a/planner/property/physical_property.go +++ b/planner/property/physical_property.go @@ -20,6 +20,10 @@ import ( "github.com/pingcap/tidb/util/codec" ) +// wholeTaskTypes records all possible kinds of task that a plan can return. For Agg, TopN and Limit, we will try to get +// these tasks one by one. +var wholeTaskTypes = []TaskType{CopSingleReadTaskType, CopDoubleReadTaskType, RootTaskType} + // Item wraps the column and its order. type Item struct { Col *expression.Column @@ -83,6 +87,19 @@ func (p *PhysicalProperty) AllColsFromSchema(schema *expression.Schema) bool { return true } +// IsFlashOnlyProp return true if this physical property is only allowed to generate flash related task +func (p *PhysicalProperty) IsFlashOnlyProp() bool { + return p.TaskTp == CopTiFlashLocalReadTaskType || p.TaskTp == CopTiFlashGlobalReadTaskType +} + +func (p *PhysicalProperty) GetAllPossibleChildTaskTypes() []TaskType { + if p.TaskTp == RootTaskType { + return wholeTaskTypes + } + // todo for CopSingleReadTaskType and CopDoubleReadTaskType, this function should never be called + return []TaskType{p.TaskTp} +} + // IsPrefix checks whether the order property is the prefix of another. func (p *PhysicalProperty) IsPrefix(prop *PhysicalProperty) bool { if len(p.Items) > len(prop.Items) { diff --git a/planner/property/task_type.go b/planner/property/task_type.go index fb89b6eabff0e..3f91509ca710b 100644 --- a/planner/property/task_type.go +++ b/planner/property/task_type.go @@ -29,11 +29,14 @@ const ( CopDoubleReadTaskType // CopTiFlashLocalReadTaskType stands for flash coprocessor that read data locally, - // and only a part of the data is read in one cop task + // and only a part of the data is read in one cop task, if the current task type is + // CopTiFlashLocalReadTaskType, all its children prop's task type is CopTiFlashLocalReadTaskType CopTiFlashLocalReadTaskType // CopTiFlashGlobalReadTaskType stands for flash coprocessor that read data globally - // and all the data of given table will be read in one cop task + // and all the data of given table will be read in one cop task, if the current task + // type is CopTiFlashGlobalReadTaskType, all its children prop's task type is + // CopTiFlashGlobalReadTaskType CopTiFlashGlobalReadTaskType ) From 8a51e02ab98f27161518d78c9b7f97fd7512f494 Mon Sep 17 00:00:00 2001 From: xufei Date: Fri, 3 Apr 2020 11:55:36 +0800 Subject: [PATCH 27/61] fix bug --- planner/core/exhaust_physical_plans.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 91097515d45a4..bb1cbcbe9283e 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1419,6 +1419,9 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ // If the hint is not matched, it will get other candidates. // If the hint is not figured, we will pick all candidates. func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { + if prop.IsFlashOnlyProp() && ((p.preferJoinType & preferMergeJoin) > 0 || (p.preferJoinType & preferHashJoin) > 0) { + return nil + } joins := make([]PhysicalPlan, 0, 5) if p.ctx.GetSessionVars().AllowBCJ { broadCastJoins := p.tryToGetBroadCastJoin(prop) From 0d0830235dfbe6efca906f790d636c531952b9ff Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 8 Apr 2020 12:35:23 +0800 Subject: [PATCH 28/61] basic cbo for broadcast join --- planner/core/exhaust_physical_plans.go | 40 +++++++++++--------- planner/core/find_best_task.go | 3 ++ planner/core/task.go | 52 +++++++++++++++++++++++++- store/tikv/coprocessor.go | 8 ++++ store/tikv/region_cache.go | 16 ++++++++ 5 files changed, 100 insertions(+), 19 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index bb1cbcbe9283e..83260672962a9 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1456,13 +1456,15 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph return joins } -func getAllDataSourceRowCount(plan LogicalPlan) int64 { +func getAllDataSourceTotalRowSize(plan LogicalPlan) float64 { if ds, ok := plan.(*DataSource); ok { - return ds.statsInfo().Count() + rowCount := ds.statsInfo().Count() + rowSize := ds.TblColHists.GetTableAvgRowSize(ds.ctx, ds.schema.Columns, kv.StoreType(ds.preferStoreType), ds.handleCol != nil) + return float64(rowCount) * rowSize } - ret := int64(0) + ret := float64(0) for _, child := range plan.Children() { - ret += getAllDataSourceRowCount(child) + ret += getAllDataSourceTotalRowSize(child) } return ret } @@ -1488,28 +1490,30 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop * property.PhysicalProperty) [] LeftJoinKeys: lkeys, RightJoinKeys: rkeys, } - // todo: currently, build side is the one has less rowcont and global read side - // is the one has less datasource row count(which mean less remote read), need - // to use cbo to decide the build side and global read side - if p.children[0].statsInfo().Count() < p.children[1].statsInfo().Count() { - baseJoin.InnerChildIdx = 0 - } else { - baseJoin.InnerChildIdx = 1 + + preferredBuildIndex := 0 + if p.children[0].statsInfo().Count() > p.children[1].statsInfo().Count() { + preferredBuildIndex = 1 } - globalIndex := baseJoin.InnerChildIdx - if prop.TaskTp != property.CopTiFlashGlobalReadTaskType && getAllDataSourceRowCount(p.children[globalIndex]) > getAllDataSourceRowCount(p.children[1 - globalIndex]) { - globalIndex = 1 - globalIndex + preferredGlobalIndex := preferredBuildIndex + if prop.TaskTp != property.CopTiFlashGlobalReadTaskType && getAllDataSourceTotalRowSize(p.children[preferredGlobalIndex]) > getAllDataSourceTotalRowSize(p.children[1 - preferredGlobalIndex]) { + preferredGlobalIndex = 1 - preferredGlobalIndex } + // todo: currently, build side is the one has less rowcount and global read side + // is the one has less datasource row size(which mean less remote read), need + // to use cbo to decide the build side and global read side if preferred build index + // is not equal to preferred global index + baseJoin.InnerChildIdx = preferredBuildIndex childrenReqProps := make([]*property.PhysicalProperty, 2) - childrenReqProps[globalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} + childrenReqProps[preferredGlobalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} if prop.TaskTp == property.CopTiFlashGlobalReadTaskType { - childrenReqProps[1-globalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} + childrenReqProps[1-preferredGlobalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} } else { - childrenReqProps[1-globalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashLocalReadTaskType} + childrenReqProps[1-preferredGlobalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashLocalReadTaskType} } join := PhysicalBroadCastJoin { basePhysicalJoin: baseJoin, - globalChildIndex: globalIndex, + globalChildIndex: preferredGlobalIndex, }.Init(p.ctx, p.stats, p.blockOffset, childrenReqProps...) results := make([]PhysicalPlan, 0, 1) results = append(results, join) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index bdb0ca4b0b0ec..4c51b6707abc0 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1345,6 +1345,9 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper } sessVars := ds.ctx.GetSessionVars() cost := rowCount * rowSize * sessVars.ScanFactor + if ts.IsGlobalRead { + cost += rowCount * sessVars.NetworkFactor * rowSize + } if isMatchProp { if prop.Items[0].Desc { ts.Desc = true diff --git a/planner/core/task.go b/planner/core/task.go index ceeb0aa28047b..ed98f0aa90f20 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/plancodec" @@ -441,6 +442,37 @@ func getAvgRowSize(stats *property.StatsInfo, schema *expression.Schema) (size f return } +// GetCost computes cost of broadcast join operator itself. +func (p *PhysicalBroadCastJoin) GetCost(lCnt, rCnt float64) float64 { + buildCnt := lCnt + if p.InnerChildIdx == 1 { + buildCnt = rCnt + } + sessVars := p.ctx.GetSessionVars() + // Cost of building hash table. + cpuCost := buildCnt * sessVars.CopCPUFactor + memoryCost := buildCnt * sessVars.MemoryFactor + // Number of matched row pairs regarding the equal join conditions. + helper := &fullJoinRowCountHelper{ + cartesian: false, + leftProfile: p.children[0].statsInfo(), + rightProfile: p.children[1].statsInfo(), + leftJoinKeys: p.LeftJoinKeys, + rightJoinKeys: p.RightJoinKeys, + leftSchema: p.children[0].Schema(), + rightSchema: p.children[1].Schema(), + } + numPairs := helper.estimate() + probeCost := numPairs * sessVars.CopCPUFactor + // should divided by the cop concurrency, which is decide by TiFlash, but TiDB + // can not get the information from TiFlash, so just use `sessVars.HashJoinConcurrency` + // as a workaround + probeCost /= float64(sessVars.HashJoinConcurrency) + cpuCost += probeCost + + return cpuCost + memoryCost +} + // GetCost computes cost of hash join operator itself. func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64) float64 { buildCnt, probeCnt := lCnt, rCnt @@ -547,11 +579,24 @@ func (p *PhysicalBroadCastJoin) attach2Task(tasks ...task) task { if !rTask.indexPlanFinished { rTask.finishIndexPlan() } + copTaskNumber := int32(1) + copClient, ok := p.ctx.GetClient().(*tikv.CopClient) + if ok { + copTaskNumber = copClient.GetBatchCopTaskNumber() + } + + buildSideCost, probeSideCost := lTask.cost(), rTask.cost() + if p.InnerChildIdx == 1 { + buildSideCost, probeSideCost = rTask.cost(), lTask.cost() + } task := & copTask { tblColHists: rTask.tblColHists, indexPlanFinished: true, tablePlan: p, - cst: lTask.cost() + rTask.cost(), + // the cost model for broadcast join is + // buildSideCost * copTaskNumber + probeSizeCost + broadcast operator cost + // because for broadcast join, the build side is executed in every copTask. + cst: buildSideCost * float64(copTaskNumber) + probeSideCost + p.GetCost(lTask.count(), rTask.count()), } logutil.BgLogger().Info("bc join cost", zap.Float64("bc cost", task.cst)) return task @@ -708,6 +753,11 @@ func finishCopTask(ctx sessionctx.Context, task task) task { // is Min(DistSQLScanConcurrency, numRegionsInvolvedInScan), since we cannot infer // the number of regions involved, we simply use DistSQLScanConcurrency. copIterWorkers := float64(t.plan().SCtx().GetSessionVars().DistSQLScanConcurrency) + if t.tablePlan != nil && t.tablePlan.TP() == plancodec.TypeBroadcastJoin { + if copClient, ok := ctx.GetClient().(*tikv.CopClient); ok { + copIterWorkers = math.Min(float64(copClient.GetBatchCopTaskNumber()), copIterWorkers) + } + } t.finishIndexPlan() // Network cost of transferring rows of table scan to TiDB. if t.tablePlan != nil { diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 3b5187cf135ff..c2064f07238ad 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -54,6 +54,14 @@ type CopClient struct { replicaReadSeed uint32 } +func (c *CopClient) GetBatchCopTaskNumber() (ret int32) { + ret = c.store.regionCache.storeMu.flashStoreNumber + if ret <= 0 { + ret = 1 + } + return ret +} + // Send builds the request and gets the coprocessor iterator response. func (c *CopClient) Send(ctx context.Context, req *kv.Request, vars *kv.Variables) kv.Response { if req.StoreType == kv.TiFlash && req.CopTaskBatch{ diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 9d2f375999482..49f8058eeb508 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -217,6 +217,7 @@ type RegionCache struct { storeMu struct { sync.RWMutex stores map[uint64]*Store + flashStoreNumber int32 } notifyDieCh chan []string notifyCheckCh chan struct{} @@ -231,6 +232,7 @@ func NewRegionCache(pdClient pd.Client) *RegionCache { c.mu.regions = make(map[RegionVerID]*Region) c.mu.sorted = btree.New(btreeDegree) c.storeMu.stores = make(map[uint64]*Store) + c.storeMu.flashStoreNumber = 0 c.notifyCheckCh = make(chan struct{}, 1) c.notifyDieCh = make(chan []string, 1) c.closeCh = make(chan struct{}) @@ -971,6 +973,13 @@ func (c *RegionCache) getStoreAddr(bo *Backoffer, region *Region, store *Store, return case unresolved: addr, err = store.initResolve(bo, c) + if store.storeType == kv.TiFlash { + c.storeMu.Lock() + if _,exists := c.storeMu.stores[store.storeID]; exists { + c.storeMu.flashStoreNumber++ + } + c.storeMu.Unlock() + } return case deleted: addr = c.changeToActiveStore(region, store, storeIdx) @@ -1383,7 +1392,14 @@ func (s *Store) reResolve(c *RegionCache) { newStore := &Store{storeID: s.storeID, addr: addr, storeType: storeType} newStore.state = *(*uint64)(unsafe.Pointer(&state)) c.storeMu.Lock() + orgStore,exists := c.storeMu.stores[newStore.storeID] + if exists && orgStore.storeType == kv.TiFlash { + c.storeMu.flashStoreNumber-- + } c.storeMu.stores[newStore.storeID] = newStore + if newStore.storeType == kv.TiFlash { + c.storeMu.flashStoreNumber++ + } c.storeMu.Unlock() retryMarkDel: // all region used those From 1b165c5eb14b822db8a4f611bdc28985b6b037d9 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 8 Apr 2020 15:00:07 +0800 Subject: [PATCH 29/61] improve --- planner/core/task.go | 65 +++++++++++++++++++++++--------------------- 1 file changed, 34 insertions(+), 31 deletions(-) diff --git a/planner/core/task.go b/planner/core/task.go index ed98f0aa90f20..f83a32e548e38 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -442,37 +442,6 @@ func getAvgRowSize(stats *property.StatsInfo, schema *expression.Schema) (size f return } -// GetCost computes cost of broadcast join operator itself. -func (p *PhysicalBroadCastJoin) GetCost(lCnt, rCnt float64) float64 { - buildCnt := lCnt - if p.InnerChildIdx == 1 { - buildCnt = rCnt - } - sessVars := p.ctx.GetSessionVars() - // Cost of building hash table. - cpuCost := buildCnt * sessVars.CopCPUFactor - memoryCost := buildCnt * sessVars.MemoryFactor - // Number of matched row pairs regarding the equal join conditions. - helper := &fullJoinRowCountHelper{ - cartesian: false, - leftProfile: p.children[0].statsInfo(), - rightProfile: p.children[1].statsInfo(), - leftJoinKeys: p.LeftJoinKeys, - rightJoinKeys: p.RightJoinKeys, - leftSchema: p.children[0].Schema(), - rightSchema: p.children[1].Schema(), - } - numPairs := helper.estimate() - probeCost := numPairs * sessVars.CopCPUFactor - // should divided by the cop concurrency, which is decide by TiFlash, but TiDB - // can not get the information from TiFlash, so just use `sessVars.HashJoinConcurrency` - // as a workaround - probeCost /= float64(sessVars.HashJoinConcurrency) - cpuCost += probeCost - - return cpuCost + memoryCost -} - // GetCost computes cost of hash join operator itself. func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64) float64 { buildCnt, probeCnt := lCnt, rCnt @@ -565,6 +534,40 @@ func (p *PhysicalHashJoin) attach2Task(tasks ...task) task { return task } +// GetCost computes cost of broadcast join operator itself. +func (p *PhysicalBroadCastJoin) GetCost(lCnt, rCnt float64) float64 { + buildCnt := lCnt + if p.InnerChildIdx == 1 { + buildCnt = rCnt + } + sessVars := p.ctx.GetSessionVars() + // Cost of building hash table. + cpuCost := buildCnt * sessVars.CopCPUFactor + memoryCost := buildCnt * sessVars.MemoryFactor + // Number of matched row pairs regarding the equal join conditions. + helper := &fullJoinRowCountHelper{ + cartesian: false, + leftProfile: p.children[0].statsInfo(), + rightProfile: p.children[1].statsInfo(), + leftJoinKeys: p.LeftJoinKeys, + rightJoinKeys: p.RightJoinKeys, + leftSchema: p.children[0].Schema(), + rightSchema: p.children[1].Schema(), + } + numPairs := helper.estimate() + probeCost := numPairs * sessVars.CopCPUFactor + // should divided by the cop concurrency, which is decide by TiFlash, but TiDB + // can not get the information from TiFlash, so just use `sessVars.HashJoinConcurrency` + // as a workaround + probeCost /= float64(sessVars.HashJoinConcurrency) + cpuCost += probeCost + + // todo since TiFlash join is significant faster than TiDB join, maybe + // need to add a variable like 'tiflash_accelerate_factor', and divide + // the final cost by that factor + return cpuCost + memoryCost +} + func (p *PhysicalBroadCastJoin) attach2Task(tasks ...task) task { lTask, lok := tasks[0].(*copTask) rTask, rok := tasks[1].(*copTask) From ccfb91ce6443aafee143023ee4f98afe7b0606d7 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 8 Apr 2020 15:23:25 +0800 Subject: [PATCH 30/61] fix bug --- planner/core/task.go | 32 ++++++++++++++++++++------------ 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/planner/core/task.go b/planner/core/task.go index f83a32e548e38..425284cb78351 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -571,6 +571,8 @@ func (p *PhysicalBroadCastJoin) GetCost(lCnt, rCnt float64) float64 { func (p *PhysicalBroadCastJoin) attach2Task(tasks ...task) task { lTask, lok := tasks[0].(*copTask) rTask, rok := tasks[1].(*copTask) + lGlobalRead := p.childrenReqProps[0].TaskTp == property.CopTiFlashGlobalReadTaskType + rGlobalRead := p.childrenReqProps[1].TaskTp == property.CopTiFlashGlobalReadTaskType if !lok || !rok || (lTask.getStoreType() != kv.TiFlash && rTask.getStoreType() != kv.TiFlash) { return invalidTask } @@ -582,24 +584,30 @@ func (p *PhysicalBroadCastJoin) attach2Task(tasks ...task) task { if !rTask.indexPlanFinished { rTask.finishIndexPlan() } - copTaskNumber := int32(1) - copClient, ok := p.ctx.GetClient().(*tikv.CopClient) - if ok { - copTaskNumber = copClient.GetBatchCopTaskNumber() - } - buildSideCost, probeSideCost := lTask.cost(), rTask.cost() - if p.InnerChildIdx == 1 { - buildSideCost, probeSideCost = rTask.cost(), lTask.cost() + lCost := lTask.cost() + rCost := rTask.cost() + if !(lGlobalRead && rGlobalRead) { + // the cost model for top level broadcast join is + // globalReadSideCost * copTaskNumber + localReadSideCost + broadcast operator cost + // because for broadcast join, the global side is executed in every copTask. + copTaskNumber := int32(1) + copClient, ok := p.ctx.GetClient().(*tikv.CopClient) + if ok { + copTaskNumber = copClient.GetBatchCopTaskNumber() + } + if lGlobalRead { + lCost = lCost * float64(copTaskNumber) + } else { + rCost = rCost * float64(copTaskNumber) + } } + task := & copTask { tblColHists: rTask.tblColHists, indexPlanFinished: true, tablePlan: p, - // the cost model for broadcast join is - // buildSideCost * copTaskNumber + probeSizeCost + broadcast operator cost - // because for broadcast join, the build side is executed in every copTask. - cst: buildSideCost * float64(copTaskNumber) + probeSideCost + p.GetCost(lTask.count(), rTask.count()), + cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count()), } logutil.BgLogger().Info("bc join cost", zap.Float64("bc cost", task.cst)) return task From 97248241956075db6912d611b013b340dc6436d7 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Thu, 9 Apr 2020 15:27:07 +0800 Subject: [PATCH 31/61] remote useless code --- store/tikv/region_cache.go | 1 - 1 file changed, 1 deletion(-) diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 1dd34c9e0aa21..52cfc5eadb188 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -381,7 +381,6 @@ func (c *RegionCache) GetTiFlashRPCContext(bo *Backoffer, id RegionVerID) (*RPCC cachedRegion := c.getCachedRegionWithRLock(id) if cachedRegion == nil { - logutil.BgLogger().Info("not found region", zap.Uint64("region id", id.id)) return nil, nil } if !cachedRegion.checkRegionCacheTTL(ts) { From d931b1829eca914d1f6c876dde94bce74246a821 Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 13 Apr 2020 14:59:56 +0800 Subject: [PATCH 32/61] add tests --- planner/core/integration_test.go | 62 +++++++++++++++++++ .../testdata/integration_serial_suite_in.json | 7 +++ .../integration_serial_suite_out.json | 37 +++++++++++ 3 files changed, 106 insertions(+) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 2033d3038ee7d..b4f3620c96d1a 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -341,6 +341,68 @@ func (s *testIntegrationSerialSuite) TestSelPushDownTiFlash(c *C) { } } +func (s *testIntegrationSerialSuite) TestBroadcastJoin(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists d1_t") + tk.MustExec("create table d1_t(d1_k int, value int)") + tk.MustExec("insert into d1_t values(1,2),(2,3)") + tk.MustExec("analyze table d1_t") + tk.MustExec("drop table if exists d2_t") + tk.MustExec("create table d2_t(d2_k decimal(10,2), value int)") + tk.MustExec("insert into d2_t values(10.11,2),(10.12,3)") + tk.MustExec("analyze table d2_t") + tk.MustExec("drop table if exists d3_t") + tk.MustExec("create table d3_t(d3_k date, value int)") + tk.MustExec("insert into d3_t values(date'2010-01-01',2),(date'2010-01-02',3)") + tk.MustExec("analyze table d3_t") + tk.MustExec("drop table if exists fact_t") + tk.MustExec("create table fact_t(d1_k int, d2_k decimal(10,2), d3_k date, col1 int, col2 int, col3 int)") + tk.MustExec("insert into fact_t values(1,10.11,date'2010-01-01',1,2,3),(1,10.11,date'2010-01-02',1,2,3),(1,10.12,date'2010-01-01',1,2,3),(1,10.12,date'2010-01-02',1,2,3)") + tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") + tk.MustExec("analyze table fact_t") + + // 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 == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_opt_broadcast_join = 1") + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } + + // out join not supported + _, err := tk.Exec("explain select /*+ tidb_bcj(fact_t, d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") + // join with non-equal condition not supported + _, err = tk.Exec("explain select /*+ tidb_bcj(fact_t, d1_t) */ count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") +} + func (s *testIntegrationSerialSuite) TestIssue15110(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index eba7a5c2da5db..353e3ba83e4b5 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -6,6 +6,13 @@ "explain select * from t where cast(t.a as float) + 3 = 5.1" ] }, + { + "name": "TestBroadcastJoin", + "cases": [ + "explain select /*+ tidb_bcj(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k" + ] + }, { "name": "TestReadFromStorageHint", "cases": [ diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 1e34ea443b96e..ad902202c25fd 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -20,6 +20,43 @@ } ] }, + { + "Name": "TestBroadcastJoin", + "Cases": [ + { + "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "Plan": [ + "StreamAgg_25 1.00 root funcs:count(Column#14)->Column#11", + "└─TableReader_26 1.00 root data:StreamAgg_13", + " └─StreamAgg_13 1.00 cop[tiflash] funcs:count(1)->Column#14", + " └─TypeBroadcastJoin_24 8.00 cop[tiflash] ", + " ├─Selection_18(Build) 0.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan_17 0.00 cop[tiflash] table:d1_t keep order:false, global read", + " └─Selection_16(Probe) 0.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan_15 0.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", + "Plan": [ + "StreamAgg_35 1.00 root funcs:count(Column#20)->Column#17", + "└─TableReader_36 1.00 root data:StreamAgg_17", + " └─StreamAgg_17 1.00 cop[tiflash] funcs:count(1)->Column#20", + " └─TypeBroadcastJoin_34 8.00 cop[tiflash] ", + " ├─Selection_28(Build) 0.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan_27 0.00 cop[tiflash] table:d3_t keep order:false, global read", + " └─TypeBroadcastJoin_19(Probe) 8.00 cop[tiflash] ", + " ├─Selection_26(Build) 0.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan_25 0.00 cop[tiflash] table:d2_t keep order:false, global read", + " └─TypeBroadcastJoin_20(Probe) 8.00 cop[tiflash] ", + " ├─Selection_24(Build) 0.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan_23 0.00 cop[tiflash] table:d1_t keep order:false, global read", + " └─Selection_22(Probe) 0.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan_21 8.00 cop[tiflash] table:fact_t keep order:false" + ] + } + ] + }, { "Name": "TestReadFromStorageHint", "Cases": [ From 7a2eede0b43383e29bc40c52598b7b02c792ffb0 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Mon, 20 Apr 2020 18:33:07 +0800 Subject: [PATCH 33/61] pass unit tests --- executor/builder.go | 1 - executor/table_readers_required_rows_test.go | 2 +- go.mod | 2 +- go.sum | 4 ++++ planner/core/exhaust_physical_plans.go | 7 +++++-- planner/core/find_best_task.go | 6 ------ planner/core/task.go | 14 ++++---------- store/tikv/region_cache.go | 6 +++--- 8 files changed, 18 insertions(+), 24 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index dce71e9c52e23..402ae164fad5d 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2314,7 +2314,6 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) * } ts := v.GetTableScan() - logutil.BgLogger().Info("build table reader", zap.Int64("my table id", ts.Table.ID), zap.String("table name", ts.Table.Name.L), zap.String("range str", ts.Ranges[0].String())) ret.ranges = ts.Ranges sctx := b.ctx.GetSessionVars().StmtCtx sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) diff --git a/executor/table_readers_required_rows_test.go b/executor/table_readers_required_rows_test.go index 9e5aab6b53fa1..095a9c5ba68a7 100644 --- a/executor/table_readers_required_rows_test.go +++ b/executor/table_readers_required_rows_test.go @@ -128,7 +128,7 @@ func buildMockDAGRequest(sctx sessionctx.Context) *tipb.DAGRequest { Columns: []*model.ColumnInfo{}, Table: &model.TableInfo{ID: 12345, PKIsHandle: false}, Desc: false, - }}) + }}, kv.TiKV) if err != nil { panic(err) } diff --git a/go.mod b/go.mod index 11a977dad0e52..ca10d868a69a8 100644 --- a/go.mod +++ b/go.mod @@ -2,7 +2,7 @@ module github.com/pingcap/tidb replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00 -replace github.com/pingcap/parser v0.0.0-20200326020624-68d423641be5 => github.com/windtalker/parser v0.0.0-20200330084704-cc2bee1a3633 +replace github.com/pingcap/parser v0.0.0-20200407074807-436f1c8c4cff => github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c require ( github.com/BurntSushi/toml v0.3.1 diff --git a/go.sum b/go.sum index 83dd3a9f674fa..a01109e41cdca 100644 --- a/go.sum +++ b/go.sum @@ -163,6 +163,10 @@ github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqC github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= github.com/hanfei1991/parser v0.0.0-20200307135251-fda93fd3e461 h1:0PegpI/0sFxkvKSRj8MT6NG4DpqoT7oVWMnO9hyBd9U= github.com/hanfei1991/parser v0.0.0-20200307135251-fda93fd3e461/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/hanfei1991/parser v0.0.0-20200420095648-f4bcaaa96aa8 h1:JwAXZcOCNZ1Jb9qzDZu19N/sUdg6aTq6mZrNZDbu6V0= +github.com/hanfei1991/parser v0.0.0-20200420095648-f4bcaaa96aa8/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c h1:akJYnz7Fe6AXtCTcZai3gf+HrZxuoVbnaGXxEmtxvpo= +github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00 h1:wRO6S3mWrAcoS78mGn1hvjPQjqTgA+MBzbq5NkKRQRM= github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 190feab96cb0e..8500512015934 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1519,7 +1519,7 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop *property.PhysicalProperty) []P preferredBuildIndex = 1 } preferredGlobalIndex := preferredBuildIndex - if prop.TaskTp != property.CopTiFlashGlobalReadTaskType && getAllDataSourceTotalRowSize(p.children[preferredGlobalIndex]) > getAllDataSourceTotalRowSize(p.children[1 - preferredGlobalIndex]) { + if prop.TaskTp != property.CopTiFlashGlobalReadTaskType && getAllDataSourceTotalRowSize(p.children[preferredGlobalIndex]) > getAllDataSourceTotalRowSize(p.children[1-preferredGlobalIndex]) { preferredGlobalIndex = 1 - preferredGlobalIndex } // todo: currently, build side is the one has less rowcount and global read side @@ -1815,7 +1815,10 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy return nil } hashAggs := make([]PhysicalPlan, 0, len(prop.GetAllPossibleChildTaskTypes())) - taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType, property.CopTiFlashLocalReadTaskType} + taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType} + if la.ctx.GetSessionVars().AllowBCJ { + taskTypes = append(taskTypes, property.CopTiFlashLocalReadTaskType) + } if la.HasDistinct() { // TODO: remove this logic after the cost estimation of distinct pushdown is implemented. // If AllowDistinctAggPushDown is set to true, we should not consider RootTask. diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 4c51b6707abc0..7953ac94fb9ac 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1112,12 +1112,6 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid tblColHists: ds.TblColHists, cst: cost, } - if ts.StoreType == kv.TiFlash { - if prop.TaskTp != property.CopTiFlashLocalReadTaskType && prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { - logutil.BgLogger().Error("invalid task:", zap.String("type", prop.TaskTp.String())) - return invalidTask, nil - } - } task = copTask if candidate.isMatchProp { copTask.keepOrder = true diff --git a/planner/core/task.go b/planner/core/task.go index d2ac6dffd9ef8..d61312df139dd 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -14,8 +14,6 @@ package core import ( - "github.com/pingcap/tidb/util/logutil" - "go.uber.org/zap" "math" "github.com/pingcap/parser/ast" @@ -101,7 +99,6 @@ func (t *copTask) copy() task { } func (t *copTask) plan() PhysicalPlan { - //logutil.BgLogger().Info("index finish", zap.Bool("index finish", t.indexPlanFinished), zap.Bool("table is nil?", t.tablePlan == nil)) if t.indexPlanFinished { return t.tablePlan } @@ -525,12 +522,10 @@ func (p *PhysicalHashJoin) attach2Task(tasks ...task) task { lTask := finishCopTask(p.ctx, tasks[0].copy()) rTask := finishCopTask(p.ctx, tasks[1].copy()) p.SetChildren(lTask.plan(), rTask.plan()) - p.schema = BuildPhysicalJoinSchema(p.JoinType, p) task := &rootTask{ p: p, cst: lTask.cost() + rTask.cost() + p.GetCost(lTask.count(), rTask.count()), } - logutil.BgLogger().Info("hash join cost", zap.Float64("hj cost", task.cst)) return task } @@ -603,13 +598,12 @@ func (p *PhysicalBroadCastJoin) attach2Task(tasks ...task) task { } } - task := & copTask { - tblColHists: rTask.tblColHists, + task := &copTask{ + tblColHists: rTask.tblColHists, indexPlanFinished: true, - tablePlan: p, - cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count()), + tablePlan: p, + cst: lCost + rCost + p.GetCost(lTask.count(), rTask.count()), } - logutil.BgLogger().Info("bc join cost", zap.Float64("bc cost", task.cst)) return task } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 53d2ac82d1049..9f5dfd92874e1 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -216,7 +216,7 @@ type RegionCache struct { } storeMu struct { sync.RWMutex - stores map[uint64]*Store + stores map[uint64]*Store flashStoreNumber int32 } notifyDieCh chan []string @@ -955,7 +955,7 @@ func (c *RegionCache) getStoreAddr(bo *Backoffer, region *Region, store *Store, addr, err = store.initResolve(bo, c) if store.storeType == kv.TiFlash { c.storeMu.Lock() - if _,exists := c.storeMu.stores[store.storeID]; exists { + if _, exists := c.storeMu.stores[store.storeID]; exists { c.storeMu.flashStoreNumber++ } c.storeMu.Unlock() @@ -1380,7 +1380,7 @@ func (s *Store) reResolve(c *RegionCache) { newStore := &Store{storeID: s.storeID, addr: addr, storeType: storeType} newStore.state = *(*uint64)(unsafe.Pointer(&state)) c.storeMu.Lock() - orgStore,exists := c.storeMu.stores[newStore.storeID] + orgStore, exists := c.storeMu.stores[newStore.storeID] if exists && orgStore.storeType == kv.TiFlash { c.storeMu.flashStoreNumber-- } From ed3065d287773a654e38e3faa4271417253dbd2e Mon Sep 17 00:00:00 2001 From: Fei Han Date: Tue, 21 Apr 2020 13:35:58 +0800 Subject: [PATCH 34/61] refine code --- planner/core/find_best_task.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 7953ac94fb9ac..c8af681c70b46 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -190,7 +190,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTas // combine best child tasks with parent physical plan. curTask := pp.attach2Task(childTasks...) - if prop.TaskTp == property.CopTiFlashLocalReadTaskType || prop.TaskTp == property.CopTiFlashGlobalReadTaskType { + if prop.IsFlashOnlyProp() { if _, ok := curTask.(*copTask); !ok { continue } @@ -377,7 +377,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida currentCandidate = ds.getTableCandidate(path, prop) } } else { - if !path.IsGlobalRead { + if !path.IsGlobalRead && !prop.IsFlashOnlyProp() { currentCandidate = ds.getTableCandidate(path, prop) } } @@ -1117,7 +1117,7 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid copTask.keepOrder = true } ts.addPushedDownSelection(copTask, ds.stats.ScaleByExpectCnt(prop.ExpectedCnt)) - if (prop.TaskTp == property.CopTiFlashGlobalReadTaskType || prop.TaskTp == property.CopTiFlashLocalReadTaskType) && len(copTask.rootTaskConds) != 0 { + if prop.IsFlashOnlyProp() && len(copTask.rootTaskConds) != 0 { return invalidTask, nil } if prop.TaskTp == property.RootTaskType { From 8a7fc255a244f946feb85e2220c65f9497062ac7 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 22 Apr 2020 10:15:04 +0800 Subject: [PATCH 35/61] support execute summary info for broadcast join --- distsql/select_result.go | 9 ++++-- executor/builder.go | 4 ++- go.mod | 3 +- go.sum | 4 +++ planner/core/exhaust_physical_plans.go | 3 ++ planner/core/initialize.go | 39 ++++++++++++++++++++++---- planner/core/plan_to_pb.go | 29 ++++++++++++++----- 7 files changed, 75 insertions(+), 16 deletions(-) diff --git a/distsql/select_result.go b/distsql/select_result.go index 5f180388b730e..6c1e72e2ff594 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -246,9 +246,14 @@ func (r *selectResult) updateCopRuntimeStats(detail *execdetails.ExecDetails, re for i, detail := range r.selectResp.GetExecutionSummaries() { if detail != nil && detail.TimeProcessedNs != nil && detail.NumProducedRows != nil && detail.NumIterations != nil { - planID := r.copPlanIDs[i] + planID := "" + if detail.GetExecutorId() != "" { + planID = detail.GetExecutorId() + } else { + planID = r.copPlanIDs[i].String() + } r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl. - RecordOneCopTask(planID.String(), callee, detail) + RecordOneCopTask(planID, callee, detail) } } } diff --git a/executor/builder.go b/executor/builder.go index 402ae164fad5d..108385da0875f 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1994,7 +1994,9 @@ func (b *executorBuilder) constructDAGReq(plans []plannercore.PhysicalPlan, stor sc := b.ctx.GetSessionVars().StmtCtx dagReq.Flags = sc.PushDownFlags() if storeType == kv.TiFlash { - dagReq.Executors, streaming, err = constructDistExecForTiFlash(b.ctx, plans[0]) + var executors []*tipb.Executor + executors, streaming, err = constructDistExecForTiFlash(b.ctx, plans[0]) + dagReq.RootExecutor = executors[0] } else { dagReq.Executors, streaming, err = constructDistExec(b.ctx, plans) } diff --git a/go.mod b/go.mod index ca10d868a69a8..d747b0d26944d 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module github.com/pingcap/tidb -replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00 +replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200421083959-59279b989667 replace github.com/pingcap/parser v0.0.0-20200407074807-436f1c8c4cff => github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c @@ -27,6 +27,7 @@ require ( github.com/gorilla/mux v1.7.3 github.com/gorilla/websocket v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 + github.com/hanfei1991/tipb v0.0.0-20200421083959-59279b989667 // indirect github.com/jeremywohl/flatten v0.0.0-20190921043622-d936035e55cf github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 github.com/konsorten/go-windows-terminal-sequences v1.0.2 // indirect diff --git a/go.sum b/go.sum index a01109e41cdca..bfd4cb3ccca4e 100644 --- a/go.sum +++ b/go.sum @@ -167,8 +167,12 @@ github.com/hanfei1991/parser v0.0.0-20200420095648-f4bcaaa96aa8 h1:JwAXZcOCNZ1Jb github.com/hanfei1991/parser v0.0.0-20200420095648-f4bcaaa96aa8/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c h1:akJYnz7Fe6AXtCTcZai3gf+HrZxuoVbnaGXxEmtxvpo= github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/hanfei1991/tipb v0.0.0-20200212061130-c4d518eb1d60 h1:i4siZGSlOhQSQYyDszniOVsGp8X72rAbl0FLZN9nxCI= +github.com/hanfei1991/tipb v0.0.0-20200212061130-c4d518eb1d60/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00 h1:wRO6S3mWrAcoS78mGn1hvjPQjqTgA+MBzbq5NkKRQRM= github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/tipb v0.0.0-20200421083959-59279b989667 h1:hrW/fUBSqAWYp0LFv5sct69QuvSjmYz89jjfWyZr7YM= +github.com/hanfei1991/tipb v0.0.0-20200421083959-59279b989667/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 8500512015934..b56bd49b8bfbf 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1452,6 +1452,9 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph logutil.BgLogger().Info("prefer bc join", zap.Int("bc count", len(broadCastJoins))) return broadCastJoins } + if len(broadCastJoins) > 0 { + return broadCastJoins + } joins = append(joins, broadCastJoins...) } if prop.IsFlashOnlyProp() { diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 2cef703238991..60b2a07e4c164 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -462,15 +462,44 @@ func (p BatchPointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsInf return &p } +// isTreePlan returns true if the plan is actually a tree +func isTreePlan(plan PhysicalPlan) bool { + if len(plan.Children()) > 1 { + return true + } + if len(plan.Children()) == 0 { + return false + } + for _, child := range plan.Children() { + if isTreePlan(child) { + return true + } + } + return false +} + +func flattenTreePlan(plan PhysicalPlan, plans []PhysicalPlan) []PhysicalPlan { + plans = append(plans, plan) + for _, child := range plan.Children() { + plans = flattenTreePlan(child, plans) + } + return plans +} + // flattenPushDownPlan converts a plan tree to a list, whose head is the leaf node like table scan. func flattenPushDownPlan(p PhysicalPlan) []PhysicalPlan { plans := make([]PhysicalPlan, 0, 5) - for { - plans = append(plans, p) - if len(p.Children()) == 0 { - break + if isTreePlan(p) { + // tree plan is only used by TiFlash + plans = flattenTreePlan(p, plans) + } else { + for { + plans = append(plans, p) + if len(p.Children()) == 0 { + break + } + p = p.Children()[0] } - p = p.Children()[0] } for i := 0; i < len(plans)/2; i++ { j := len(plans) - i - 1 diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 8c64b9d360f69..de15e447c3963 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -45,14 +45,16 @@ func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) ( for _, aggFunc := range p.AggFuncs { aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc)) } + executorId := "" if storeType == kv.TiFlash { var err error aggExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } + executorId = p.ExplainID().String() } - return &tipb.Executor{Tp: tipb.ExecType_TypeAggregation, Aggregation: aggExec}, nil + return &tipb.Executor{Tp: tipb.ExecType_TypeAggregation, Aggregation: aggExec, ExecutorId: &executorId}, nil } // ToPB implements PhysicalPlan ToPB interface. @@ -65,14 +67,16 @@ func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) for _, aggFunc := range p.AggFuncs { aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc)) } + executorId := "" if storeType == kv.TiFlash { var err error aggExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } + executorId = p.ExplainID().String() } - return &tipb.Executor{Tp: tipb.ExecType_TypeStreamAgg, Aggregation: aggExec}, nil + return &tipb.Executor{Tp: tipb.ExecType_TypeStreamAgg, Aggregation: aggExec, ExecutorId: &executorId}, nil } // ToPB implements PhysicalPlan ToPB interface. @@ -82,14 +86,16 @@ func (p *PhysicalSelection) ToPB(ctx sessionctx.Context, storeType kv.StoreType) selExec := &tipb.Selection{ Conditions: expression.ExpressionsToPBList(sc, p.Conditions, client), } + executorId := "" if storeType == kv.TiFlash { var err error selExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } + executorId = p.ExplainID().String() } - return &tipb.Executor{Tp: tipb.ExecType_TypeSelection, Selection: selExec}, nil + return &tipb.Executor{Tp: tipb.ExecType_TypeSelection, Selection: selExec, ExecutorId: &executorId}, nil } // ToPB implements PhysicalPlan ToPB interface. @@ -102,14 +108,16 @@ func (p *PhysicalTopN) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*ti for _, item := range p.ByItems { topNExec.OrderBy = append(topNExec.OrderBy, expression.SortByItemToPB(sc, client, item.Expr, item.Desc)) } + executorId := "" if storeType == kv.TiFlash { var err error topNExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } + executorId = p.ExplainID().String() } - return &tipb.Executor{Tp: tipb.ExecType_TypeTopN, TopN: topNExec}, nil + return &tipb.Executor{Tp: tipb.ExecType_TypeTopN, TopN: topNExec, ExecutorId: &executorId}, nil } // ToPB implements PhysicalPlan ToPB interface. @@ -117,14 +125,16 @@ func (p *PhysicalLimit) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*t limitExec := &tipb.Limit{ Limit: p.Count, } + executorId := "" if storeType == kv.TiFlash { var err error limitExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } + executorId = p.ExplainID().String() } - return &tipb.Executor{Tp: tipb.ExecType_TypeLimit, Limit: limitExec}, nil + return &tipb.Executor{Tp: tipb.ExecType_TypeLimit, Limit: limitExec, ExecutorId: &executorId}, nil } // ToPB implements PhysicalPlan ToPB interface. @@ -137,6 +147,7 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) if p.isPartition { tsExec.TableId = p.physicalTableID } + executorId := "" if storeType == kv.TiFlash && p.IsGlobalRead { tsExec.NextReadEngine = tipb.EngineType_TiFlash ranges := distsql.TableRangesToKVRanges(tsExec.TableId, p.Ranges, nil) @@ -145,8 +156,11 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) } logutil.BgLogger().Info("make range for table.") } + if storeType == kv.TiFlash { + executorId = p.ExplainID().String() + } err := SetPBColumnsDefaultValue(ctx, tsExec.Columns, p.Columns) - return &tipb.Executor{Tp: tipb.ExecType_TypeTableScan, TblScan: tsExec}, err + return &tipb.Executor{Tp: tipb.ExecType_TypeTableScan, TblScan: tsExec, ExecutorId: &executorId}, err } // checkCoverIndex checks whether we can pass unique info to TiKV. We should push it if and only if the length of @@ -227,7 +241,8 @@ func (p *PhysicalBroadCastJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreT Children: []*tipb.Executor{lChildren, rChildren}, } - return &tipb.Executor{Tp: tipb.ExecType_TypeJoin, Join: join}, nil + executorId := p.ExplainID().String() + return &tipb.Executor{Tp: tipb.ExecType_TypeJoin, Join: join, ExecutorId: &executorId}, nil } // SetPBColumnsDefaultValue sets the default values of tipb.ColumnInfos. From 87d9fdf387a4a4fa5a24596c47eef1a009705974 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 22 Apr 2020 14:42:24 +0800 Subject: [PATCH 36/61] fix bug in explain for broadcast join --- planner/core/exhaust_physical_plans.go | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b56bd49b8bfbf..c38c959e2db8d 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1531,16 +1531,21 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop *property.PhysicalProperty) []P // is not equal to preferred global index baseJoin.InnerChildIdx = preferredBuildIndex childrenReqProps := make([]*property.PhysicalProperty, 2) - childrenReqProps[preferredGlobalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} + childrenReqProps[preferredGlobalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType, ExpectedCnt: math.MaxFloat64} if prop.TaskTp == property.CopTiFlashGlobalReadTaskType { - childrenReqProps[1-preferredGlobalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType} + childrenReqProps[1-preferredGlobalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType, ExpectedCnt: math.MaxFloat64} } else { - childrenReqProps[1-preferredGlobalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashLocalReadTaskType} + childrenReqProps[1-preferredGlobalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashLocalReadTaskType, ExpectedCnt: math.MaxFloat64} } + if prop.ExpectedCnt < p.stats.RowCount { + expCntScale := prop.ExpectedCnt / p.stats.RowCount + childrenReqProps[1-baseJoin.InnerChildIdx].ExpectedCnt = p.children[1-baseJoin.InnerChildIdx].statsInfo().RowCount * expCntScale + } + join := PhysicalBroadCastJoin{ basePhysicalJoin: baseJoin, globalChildIndex: preferredGlobalIndex, - }.Init(p.ctx, p.stats, p.blockOffset, childrenReqProps...) + }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, childrenReqProps...) results := make([]PhysicalPlan, 0, 1) results = append(results, join) return results From 23bf1581eeedf69cce69adddcafb80ab77b18dd6 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 22 Apr 2020 14:44:33 +0800 Subject: [PATCH 37/61] format code --- go.mod | 1 - 1 file changed, 1 deletion(-) diff --git a/go.mod b/go.mod index d747b0d26944d..3b9eee8c60a28 100644 --- a/go.mod +++ b/go.mod @@ -27,7 +27,6 @@ require ( github.com/gorilla/mux v1.7.3 github.com/gorilla/websocket v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 - github.com/hanfei1991/tipb v0.0.0-20200421083959-59279b989667 // indirect github.com/jeremywohl/flatten v0.0.0-20190921043622-d936035e55cf github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 github.com/konsorten/go-windows-terminal-sequences v1.0.2 // indirect From 86cc61a7029ea8577ec0411060c0998af4009b17 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 22 Apr 2020 15:05:03 +0800 Subject: [PATCH 38/61] remove un-needed code --- planner/core/exhaust_physical_plans.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index c38c959e2db8d..90cc4dc2536fe 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1452,9 +1452,6 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph logutil.BgLogger().Info("prefer bc join", zap.Int("bc count", len(broadCastJoins))) return broadCastJoins } - if len(broadCastJoins) > 0 { - return broadCastJoins - } joins = append(joins, broadCastJoins...) } if prop.IsFlashOnlyProp() { From 7b43599d1714b7724ffdef94db952b2da70808c6 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Wed, 22 Apr 2020 15:52:01 +0800 Subject: [PATCH 39/61] fix make dev --- go.sum | 10 ---------- planner/core/initialize.go | 1 + planner/core/logical_plan_builder.go | 6 +++++- planner/core/physical_plans.go | 3 +++ planner/core/plan_to_pb.go | 1 + planner/core/resolve_indices.go | 1 + planner/property/physical_property.go | 1 + store/tikv/coprocessor.go | 2 ++ util/plancodec/id.go | 2 +- 9 files changed, 15 insertions(+), 12 deletions(-) diff --git a/go.sum b/go.sum index a01109e41cdca..bb6ae8bcff75a 100644 --- a/go.sum +++ b/go.sum @@ -161,10 +161,6 @@ github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t github.com/grpc-ecosystem/grpc-gateway v1.12.1 h1:zCy2xE9ablevUOrUZc3Dl72Dt+ya2FNAvC2yLYMHzi4= github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqCisSPWTxCZ7sBRjU6iH9c= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= -github.com/hanfei1991/parser v0.0.0-20200307135251-fda93fd3e461 h1:0PegpI/0sFxkvKSRj8MT6NG4DpqoT7oVWMnO9hyBd9U= -github.com/hanfei1991/parser v0.0.0-20200307135251-fda93fd3e461/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/hanfei1991/parser v0.0.0-20200420095648-f4bcaaa96aa8 h1:JwAXZcOCNZ1Jb9qzDZu19N/sUdg6aTq6mZrNZDbu6V0= -github.com/hanfei1991/parser v0.0.0-20200420095648-f4bcaaa96aa8/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c h1:akJYnz7Fe6AXtCTcZai3gf+HrZxuoVbnaGXxEmtxvpo= github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00 h1:wRO6S3mWrAcoS78mGn1hvjPQjqTgA+MBzbq5NkKRQRM= @@ -277,8 +273,6 @@ github.com/pingcap/kvproto v0.0.0-20200409034505-a5af800ca2ef/go.mod h1:IOdRDPLy github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfEgRJ4T9NGgGTxdHpJerent7rM= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20200407074807-436f1c8c4cff h1:FolYHDHFe4y0WXj6A8nxnlFO4f7gRo0XYBew6JNu8Uk= -github.com/pingcap/parser v0.0.0-20200407074807-436f1c8c4cff/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/pingcap/pd/v4 v4.0.0-beta.1.0.20200305072537-61d9f9cc35d3 h1:Yrp99FnjHAEuDrSBql2l0IqCtJX7KwJbTsD5hIArkvk= github.com/pingcap/pd/v4 v4.0.0-beta.1.0.20200305072537-61d9f9cc35d3/go.mod h1:25GfNw6+Jcr9kca5rtmTb4gKCJ4jOpow2zV2S9Dgafs= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= @@ -289,8 +283,6 @@ github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1/go.mod h1:EB/852NM github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incompatible h1:84F7MFMfdAYObrznvRslmVu43aoihrlL+7mMyMlOi0o= github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= -github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 h1:aJPXrT1u4VfUSGFA2oQVwl4pOXzqe+YI6wed01cjDH4= -github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -378,8 +370,6 @@ github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijb github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= github.com/urfave/negroni v0.3.0 h1:PaXOb61mWeZJxc1Ji2xJjpVg9QfPo0rrB+lHyBxGNSU= github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= -github.com/windtalker/parser v0.0.0-20200330084704-cc2bee1a3633 h1:ubnmco1ONtSC6CeIuV+b7l+dlvJcNMkPTZwdQruzAUo= -github.com/windtalker/parser v0.0.0-20200330084704-cc2bee1a3633/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 2cef703238991..eab880cc99998 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -312,6 +312,7 @@ func (p PhysicalHashJoin) Init(ctx sessionctx.Context, stats *property.StatsInfo return &p } +// Init initializes BatchPointGetPlan. func (p PhysicalBroadCastJoin) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalBroadCastJoin { tp := plancodec.TypeBroadcastJoin p.basePhysicalPlan = newBasePhysicalPlan(ctx, tp, &p, offset) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index d036d35f38c87..e48f08b745605 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -58,8 +58,12 @@ const ( // HintSMJ is hint enforce merge join. HintSMJ = "merge_join" + // TiDBBroadCastJoin indicates applying broadcast join by force. TiDBBroadCastJoin = "tidb_bcj" - HintBCJ = "bc_join" + + // HintBCJ indicates applying broadcast join by force. + HintBCJ = "bc_join" + // TiDBIndexNestedLoopJoin is hint enforce index nested loop join. TiDBIndexNestedLoopJoin = "tidb_inlj" // HintINLJ is hint enforce index nested loop join. diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index b2a6b963cbc35..92e14178d8d39 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -73,10 +73,12 @@ type PhysicalTableReader struct { StoreType kv.StoreType } +// GetTablePlan exports the tablePlan. func (p *PhysicalTableReader) GetTablePlan() PhysicalPlan { return p.tablePlan } +// GetTableScan exports the tableScan that contained in tablePlan. func (p *PhysicalTableReader) GetTableScan() *PhysicalTableScan { curPlan := p.tablePlan for { @@ -436,6 +438,7 @@ type PhysicalMergeJoin struct { Desc bool } +// PhysicalBroadCastJoin only works for TiFlash Engine, which broadcast the small table to every replica of probe side of tables. type PhysicalBroadCastJoin struct { basePhysicalJoin globalChildIndex int diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 8c64b9d360f69..58e0a4b464c8b 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -198,6 +198,7 @@ func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context, _ kv.StoreType) (*tipb. return &tipb.Executor{Tp: tipb.ExecType_TypeIndexScan, IdxScan: idxExec}, nil } +// ToPB implements PhysicalPlan ToPB interface. func (p *PhysicalBroadCastJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { sc := ctx.GetSessionVars().StmtCtx client := ctx.GetClient() diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index 4ecc66c85b32f..aa8bbd1159685 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -113,6 +113,7 @@ func (p *PhysicalHashJoin) ResolveIndices() (err error) { return } +// ResolveIndices implements Plan interface. func (p *PhysicalBroadCastJoin) ResolveIndices() (err error) { err = p.physicalSchemaProducer.ResolveIndices() if err != nil { diff --git a/planner/property/physical_property.go b/planner/property/physical_property.go index 62612ec6bac63..35a66ade47579 100644 --- a/planner/property/physical_property.go +++ b/planner/property/physical_property.go @@ -92,6 +92,7 @@ func (p *PhysicalProperty) IsFlashOnlyProp() bool { return p.TaskTp == CopTiFlashLocalReadTaskType || p.TaskTp == CopTiFlashGlobalReadTaskType } +// GetAllPossibleChildTaskTypes enumrates the possible types of tasks for children. func (p *PhysicalProperty) GetAllPossibleChildTaskTypes() []TaskType { if p.TaskTp == RootTaskType { return wholeTaskTypes diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index f704aaf497dd6..7034cb8dc1541 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -55,6 +55,8 @@ type CopClient struct { replicaReadSeed uint32 } +// GetBatchCopTaskNumber returns the number of stores that tiflash engines have. +// This information is important for generating tiflash plan, for example, it can help calcuate the cost of broadcasting small table. func (c *CopClient) GetBatchCopTaskNumber() (ret int32) { ret = c.store.regionCache.storeMu.flashStoreNumber if ret <= 0 { diff --git a/util/plancodec/id.go b/util/plancodec/id.go index c859b685699a4..ecca150f3c4d7 100644 --- a/util/plancodec/id.go +++ b/util/plancodec/id.go @@ -50,7 +50,7 @@ const ( TypeLimit = "Limit" // TypeHashJoin is the type of hash join. TypeHashJoin = "HashJoin" - // TypeHashRightJoin is the type of right hash join. + // TypeBroadcastJoin is the type of broad cast join. TypeBroadcastJoin = "TypeBroadcastJoin" // TypeMergeJoin is the type of merge join. TypeMergeJoin = "MergeJoin" From d85145b530b26460da393bb8ebf96699c41a2ede Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 23 Apr 2020 10:28:12 +0800 Subject: [PATCH 40/61] address comments --- planner/core/initialize.go | 29 +---------------------------- 1 file changed, 1 insertion(+), 28 deletions(-) diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 60b2a07e4c164..4120d88760bf3 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -462,22 +462,6 @@ func (p BatchPointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsInf return &p } -// isTreePlan returns true if the plan is actually a tree -func isTreePlan(plan PhysicalPlan) bool { - if len(plan.Children()) > 1 { - return true - } - if len(plan.Children()) == 0 { - return false - } - for _, child := range plan.Children() { - if isTreePlan(child) { - return true - } - } - return false -} - func flattenTreePlan(plan PhysicalPlan, plans []PhysicalPlan) []PhysicalPlan { plans = append(plans, plan) for _, child := range plan.Children() { @@ -489,18 +473,7 @@ func flattenTreePlan(plan PhysicalPlan, plans []PhysicalPlan) []PhysicalPlan { // flattenPushDownPlan converts a plan tree to a list, whose head is the leaf node like table scan. func flattenPushDownPlan(p PhysicalPlan) []PhysicalPlan { plans := make([]PhysicalPlan, 0, 5) - if isTreePlan(p) { - // tree plan is only used by TiFlash - plans = flattenTreePlan(p, plans) - } else { - for { - plans = append(plans, p) - if len(p.Children()) == 0 { - break - } - p = p.Children()[0] - } - } + plans = flattenTreePlan(p, plans) for i := 0; i < len(plans)/2; i++ { j := len(plans) - i - 1 plans[i], plans[j] = plans[j], plans[i] From a6b79cad84d4b6b4eb3f5e20281150c30ee6ef0c Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 6 May 2020 14:19:26 +0800 Subject: [PATCH 41/61] Hanfei/join merge (#7) * enable exec details for batch cop * format code --- store/tikv/batch_coprocessor.go | 21 +++++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/store/tikv/batch_coprocessor.go b/store/tikv/batch_coprocessor.go index 7a446eec95d77..40f7b708c20a6 100644 --- a/store/tikv/batch_coprocessor.go +++ b/store/tikv/batch_coprocessor.go @@ -64,7 +64,7 @@ func (rs *batchCopResponse) GetStartKey() kv.Key { // GetExecDetails is unavailable currently, because TiFlash has not collected exec details for batch cop. // TODO: Will fix in near future. func (rs *batchCopResponse) GetExecDetails() *execdetails.ExecDetails { - return &execdetails.ExecDetails{} + return rs.detail } // MemSize returns how many bytes of memory this response use @@ -282,7 +282,7 @@ func (b *batchCopIterator) handleTask(ctx context.Context, bo *Backoffer, task * for idx := 0; idx < len(tasks); idx++ { ret, err := b.handleTaskOnce(ctx, bo, task) if err != nil { - resp := &batchCopResponse{err: errors.Trace(err)} + resp := &batchCopResponse{err: errors.Trace(err), detail: new(execdetails.ExecDetails)} b.sendToRespCh(resp) break } @@ -389,9 +389,22 @@ func (b *batchCopIterator) handleBatchCopResponse(bo *Backoffer, response *copro return errors.Trace(err) } - b.sendToRespCh(&batchCopResponse{ + resp := batchCopResponse{ pbResp: response, - }) + detail: new(execdetails.ExecDetails), + } + + resp.detail.BackoffTime = time.Duration(bo.totalSleep) * time.Millisecond + resp.detail.BackoffSleep = make(map[string]time.Duration, len(bo.backoffTimes)) + resp.detail.BackoffTimes = make(map[string]int, len(bo.backoffTimes)) + for backoff := range bo.backoffTimes { + backoffName := backoff.String() + resp.detail.BackoffTimes[backoffName] = bo.backoffTimes[backoff] + resp.detail.BackoffSleep[backoffName] = time.Duration(bo.backoffSleepMS[backoff]) * time.Millisecond + } + resp.detail.CalleeAddress = task.storeAddr + + b.sendToRespCh(&resp) return } From f13a6cdc8c2ab6a66393fa941df1f077cd6edb94 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Wed, 6 May 2020 17:46:18 +0800 Subject: [PATCH 42/61] fix test --- .../integration_serial_suite_out.json | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 6b17c16da7fa4..bd28fa364d0bd 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -30,10 +30,10 @@ "└─TableReader_26 1.00 root data:StreamAgg_13", " └─StreamAgg_13 1.00 cop[tiflash] funcs:count(1)->Column#14", " └─TypeBroadcastJoin_24 8.00 cop[tiflash] ", - " ├─Selection_18(Build) 0.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan_17 0.00 cop[tiflash] table:d1_t keep order:false, global read", - " └─Selection_16(Probe) 0.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan_15 0.00 cop[tiflash] table:fact_t keep order:false" + " ├─Selection_18(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan_17 2.00 cop[tiflash] table:d1_t keep order:false, global read", + " └─Selection_16(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan_15 8.00 cop[tiflash] table:fact_t keep order:false" ] }, { @@ -43,15 +43,15 @@ "└─TableReader_36 1.00 root data:StreamAgg_17", " └─StreamAgg_17 1.00 cop[tiflash] funcs:count(1)->Column#20", " └─TypeBroadcastJoin_34 8.00 cop[tiflash] ", - " ├─Selection_28(Build) 0.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan_27 0.00 cop[tiflash] table:d3_t keep order:false, global read", + " ├─Selection_28(Build) 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan_27 2.00 cop[tiflash] table:d3_t keep order:false, global read", " └─TypeBroadcastJoin_19(Probe) 8.00 cop[tiflash] ", - " ├─Selection_26(Build) 0.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan_25 0.00 cop[tiflash] table:d2_t keep order:false, global read", + " ├─Selection_26(Build) 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan_25 2.00 cop[tiflash] table:d2_t keep order:false, global read", " └─TypeBroadcastJoin_20(Probe) 8.00 cop[tiflash] ", - " ├─Selection_24(Build) 0.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan_23 0.00 cop[tiflash] table:d1_t keep order:false, global read", - " └─Selection_22(Probe) 0.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " ├─Selection_24(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan_23 2.00 cop[tiflash] table:d1_t keep order:false, global read", + " └─Selection_22(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", " └─TableFullScan_21 8.00 cop[tiflash] table:fact_t keep order:false" ] } From 39b046c4ad4b4a07a2340df227f30849509acbb0 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Thu, 7 May 2020 15:37:33 +0800 Subject: [PATCH 43/61] change tidb_opt_broadcast_join to global vars --- session/session.go | 1 + sessionctx/variable/sysvar.go | 2 +- sessionctx/variable/tidb_vars.go | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/session/session.go b/session/session.go index b327c701eb9f6..36e08975ec21f 100644 --- a/session/session.go +++ b/session/session.go @@ -1948,6 +1948,7 @@ var builtinGlobalVariable = []string{ variable.TiDBEnableIndexMerge, variable.TiDBTxnMode, variable.TiDBAllowBatchCop, + variable.TiDBOptBCJ, variable.TiDBRowFormatVersion, variable.TiDBEnableStmtSummary, variable.TiDBStmtSummaryInternalQuery, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 564a88a79aa38..4c64bd681bcce 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -610,7 +610,7 @@ var defaultSysVars = []*SysVar{ /* TiDB specific variables */ {ScopeSession, TiDBSnapshot, ""}, {ScopeSession, TiDBOptAggPushDown, BoolToIntStr(DefOptAggPushDown)}, - {ScopeSession, TiDBOptBCJ, BoolToIntStr(DefOptBCJ)}, + {ScopeGlobal | ScopeSession, TiDBOptBCJ, BoolToIntStr(DefOptBCJ)}, {ScopeSession, TiDBOptDistinctAggPushDown, BoolToIntStr(DefOptDistinctAggPushDown)}, {ScopeSession, TiDBOptWriteRowID, BoolToIntStr(DefOptWriteRowID)}, {ScopeGlobal | ScopeSession, TiDBBuildStatsConcurrency, strconv.Itoa(DefBuildStatsConcurrency)}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 6fd71f2f01ef8..a8b6bb2335149 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -448,7 +448,7 @@ const ( DefTiDBHashJoinConcurrency = 5 DefTiDBProjectionConcurrency = 4 DefTiDBOptimizerSelectivityLevel = 0 - DefTiDBAllowBatchCop = 1 + DefTiDBAllowBatchCop = 0 DefTiDBTxnMode = "" DefTiDBRowFormatV1 = 1 DefTiDBRowFormatV2 = 2 From a28e1ee53eb33a410abf75ad6192f1b69f2baf98 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=99=8E?= Date: Fri, 8 May 2020 14:31:31 +0800 Subject: [PATCH 44/61] Ban cartesian join to be pushed down to TiFlash (#8) --- planner/core/exhaust_physical_plans.go | 2 +- planner/core/integration_test.go | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 90cc4dc2536fe..25d0ef615cf4e 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1500,7 +1500,7 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop *property.PhysicalProperty) []P return nil } - if p.JoinType != InnerJoin || len(p.LeftConditions) != 0 || len(p.RightConditions) != 0 || len(p.OtherConditions) != 0 { + if p.JoinType != InnerJoin || len(p.LeftConditions) != 0 || len(p.RightConditions) != 0 || len(p.OtherConditions) != 0 || len(p.EqualConditions) == 0 { return nil } diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 0f53a32b9b2ac..99db04f075223 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -401,6 +401,10 @@ func (s *testIntegrationSerialSuite) TestBroadcastJoin(c *C) { _, err = tk.Exec("explain select /*+ tidb_bcj(fact_t, d1_t) */ count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value") c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") + // cartsian join not supported + _, err = tk.Exec("explain select /*+ tidb_bcj(fact_t, d1_t) */ count(*) from fact_t join d1_t") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") } func (s *testIntegrationSerialSuite) TestIssue15110(c *C) { From e2659a13f2ecdd4f9342de9ef6c39dd8649e20ff Mon Sep 17 00:00:00 2001 From: xufei Date: Sat, 9 May 2020 12:51:35 +0800 Subject: [PATCH 45/61] merge master (#10) * merge master * fix bug * fix bug * fix ut --- .github/workflows/assign_project.yml | 3 +- .github/workflows/issue_assigned.yml | 20 + Makefile | 11 +- bindinfo/bind_test.go | 33 +- bindinfo/cache.go | 8 +- cmd/ddltest/ddl_test.go | 2 +- cmd/ddltest/index_test.go | 6 +- cmd/explaintest/r/black_list.result | 8 +- cmd/explaintest/r/explain_easy.result | 18 +- .../explain_generate_column_substitute.result | 59 ++- cmd/explaintest/r/partition_pruning.result | 176 ++++++- cmd/explaintest/t/black_list.test | 8 +- .../t/explain_generate_column_substitute.test | 37 +- cmd/explaintest/t/partition_pruning.test | 37 +- cmd/importcheck/importcheck.go | 119 +++++ cmd/importer/stats.go | 2 +- config/config.go | 111 ++-- config/config.toml.example | 36 +- config/config_test.go | 19 +- config/config_util.go | 22 + config/config_util_test.go | 49 ++ ddl/column_change_test.go | 4 +- ddl/column_test.go | 22 +- ddl/db_change_test.go | 33 +- ddl/db_integration_test.go | 152 +++++- ddl/db_partition_test.go | 70 ++- ddl/db_test.go | 87 ++-- ddl/ddl.go | 5 - ddl/ddl_api.go | 429 +++++++++------- ddl/ddl_worker.go | 6 +- ddl/ddl_worker_test.go | 8 +- ddl/failtest/fail_db_test.go | 31 +- ddl/index.go | 32 +- ddl/index_change_test.go | 20 +- ddl/mock.go | 10 +- ddl/partition.go | 155 +++--- ddl/rollingback.go | 21 +- ddl/sequence_test.go | 5 + ddl/serial_test.go | 55 +- ddl/split_region.go | 4 +- ddl/table.go | 66 ++- ddl/util/syncer_test.go | 2 +- distsql/distsql.go | 2 +- distsql/request_builder.go | 28 +- distsql/request_builder_test.go | 8 +- distsql/select_result.go | 10 +- distsql/stream.go | 20 +- ...19-11-14-tidb-builtin-diagnostics-zh_CN.md | 4 +- .../2019-11-14-tidb-builtin-diagnostics.md | 4 +- docs/design/2020-04-17-sql-sequence.md | 115 +++++ docs/design/2020-04-20-brie.md | 331 ++++++++++++ domain/domain.go | 70 ++- domain/domain_test.go | 3 +- domain/global_vars_cache.go | 33 +- domain/global_vars_cache_test.go | 72 ++- domain/infosync/info.go | 6 +- domain/infosync/info_test.go | 13 + errno/errcode.go | 5 + errno/errname.go | 6 + executor/adapter.go | 64 ++- executor/admin.go | 38 +- executor/admin_test.go | 88 ++-- executor/aggfuncs/builder.go | 2 +- executor/aggfuncs/func_bitfuncs.go | 31 +- executor/aggfuncs/func_first_row.go | 5 +- executor/aggfuncs/func_lead_lag_test.go | 4 +- executor/aggregate.go | 4 +- executor/aggregate_test.go | 11 + executor/analyze.go | 24 +- executor/analyze_test.go | 41 +- executor/batch_checker.go | 6 +- executor/batch_point_get.go | 16 +- executor/benchmark_test.go | 102 ++-- executor/brie.go | 470 +++++++++++++++++ executor/builder.go | 63 ++- executor/chunk_size_control_test.go | 14 +- executor/ddl.go | 106 ++-- executor/ddl_test.go | 188 ++++++- executor/delete.go | 24 +- executor/distsql.go | 80 +-- executor/distsql_test.go | 31 +- executor/errors.go | 5 + executor/executor.go | 89 ++-- executor/executor_required_rows_test.go | 4 + executor/executor_test.go | 150 +++++- executor/explainfor_test.go | 39 ++ executor/index_lookup_hash_join.go | 4 +- executor/index_merge_reader.go | 48 +- executor/index_merge_reader_test.go | 11 + executor/infoschema_reader.go | 36 ++ executor/insert.go | 26 +- executor/insert_common.go | 11 +- executor/insert_test.go | 11 + executor/inspection_result.go | 1 - executor/inspection_summary.go | 12 +- executor/inspection_summary_test.go | 23 +- executor/join.go | 4 +- executor/join_test.go | 60 ++- executor/joiner.go | 2 +- executor/load_data.go | 10 +- executor/mem_reader.go | 24 +- executor/memtable_reader.go | 55 +- executor/memtable_reader_test.go | 66 ++- executor/merge_join_test.go | 32 +- executor/oomtest/oom_test.go | 211 ++++++++ executor/point_get.go | 63 ++- executor/point_get_test.go | 44 +- executor/prepared.go | 2 +- executor/reload_expr_pushdown_blacklist.go | 22 +- executor/replace.go | 10 +- executor/seqtest/seq_executor_test.go | 262 +++------- executor/set.go | 8 +- executor/set_config.go | 207 ++++++++ executor/set_test.go | 132 +++-- executor/show.go | 72 ++- executor/show_test.go | 125 +++++ executor/simple.go | 14 +- executor/slow_query.go | 51 +- executor/slow_query_test.go | 17 +- executor/split.go | 12 +- executor/split_test.go | 15 +- executor/union_scan.go | 48 +- executor/union_scan_test.go | 38 ++ executor/update.go | 17 +- executor/update_test.go | 21 +- executor/window_test.go | 25 + executor/write.go | 52 +- executor/write_test.go | 20 +- expression/aggregation/aggregation.go | 10 +- expression/aggregation/descriptor.go | 18 + expression/bench_test.go | 29 +- expression/builtin.go | 32 +- expression/builtin_arithmetic.go | 80 ++- expression/builtin_cast.go | 38 +- expression/builtin_cast_bench_test.go | 5 +- expression/builtin_cast_test.go | 55 +- expression/builtin_cast_vec_test.go | 2 +- expression/builtin_compare.go | 31 +- expression/builtin_compare_test.go | 8 +- expression/builtin_compare_vec.go | 4 +- expression/builtin_control.go | 31 +- expression/builtin_control_test.go | 4 +- expression/builtin_encryption.go | 60 ++- expression/builtin_encryption_test.go | 4 +- expression/builtin_info.go | 111 +++- expression/builtin_info_test.go | 4 +- expression/builtin_info_vec.go | 2 +- expression/builtin_info_vec_test.go | 3 +- expression/builtin_json.go | 230 +++++---- expression/builtin_json_test.go | 2 + expression/builtin_json_vec.go | 177 ++++++- expression/builtin_json_vec_test.go | 30 +- expression/builtin_like.go | 10 +- expression/builtin_like_vec.go | 4 + expression/builtin_math.go | 138 ++++- expression/builtin_math_test.go | 28 +- expression/builtin_miscellaneous.go | 72 ++- expression/builtin_miscellaneous_test.go | 2 +- expression/builtin_miscellaneous_vec.go | 2 +- expression/builtin_op.go | 142 +++++- expression/builtin_op_test.go | 44 +- expression/builtin_other.go | 35 +- expression/builtin_other_test.go | 10 +- expression/builtin_string.go | 233 +++++++-- expression/builtin_string_test.go | 76 +-- expression/builtin_string_vec.go | 2 +- expression/builtin_time.go | 322 +++++++++--- expression/builtin_time_test.go | 40 +- expression/builtin_time_vec.go | 47 +- expression/builtin_time_vec_test.go | 21 + expression/builtin_vectorized_test.go | 10 +- expression/collation.go | 12 + expression/column.go | 2 +- expression/constant.go | 40 +- expression/constant_fold.go | 2 +- expression/constant_propagation.go | 37 +- expression/constant_test.go | 6 +- expression/distsql_builtin.go | 17 +- expression/expr_to_pb.go | 7 +- expression/expr_to_pb_test.go | 86 ++-- expression/expression.go | 246 +++++++-- expression/expression_test.go | 61 ++- expression/function_traits.go | 8 +- expression/integration_test.go | 341 ++++++++++++- expression/scalar_function_test.go | 6 +- expression/simple_rewriter.go | 6 +- expression/testdata/expression_suite_in.json | 4 +- expression/testdata/expression_suite_out.json | 12 + expression/testdata/partition_pruner_out.json | 1 - expression/util.go | 78 ++- expression/util_test.go | 50 +- go.mod | 48 +- go.sum | 255 +++++++++- infoschema/builder.go | 40 +- infoschema/cluster.go | 10 +- infoschema/infoschema.go | 11 + infoschema/infoschema_test.go | 1 + infoschema/metric_table_def.go | 26 +- infoschema/perfschema/const.go | 125 ++--- infoschema/perfschema/tables.go | 174 +++---- infoschema/perfschema/tables_test.go | 333 ------------ infoschema/tables.go | 160 ++++-- infoschema/tables_test.go | 375 +++++++++++++- kv/buffer_store.go | 8 + kv/key.go | 268 +++++++++- kv/key_test.go | 78 +++ kv/memdb/memdb.go | 1 + kv/memdb/memdb_norace_test.go | 109 ++++ kv/memdb/memdb_test.go | 103 +--- kv/mock_test.go | 2 +- meta/autoid/autoid.go | 71 ++- meta/autoid/autoid_test.go | 10 +- meta/autoid/errors.go | 2 +- metrics/bindinfo.go | 6 +- metrics/ddl.go | 12 +- metrics/distsql.go | 4 +- metrics/domain.go | 2 +- metrics/gc_worker.go | 2 +- metrics/grafana/overview.json | 68 ++- metrics/grafana/tidb.json | 254 ++++++++- metrics/grafana/tidb_summary.json | 38 +- metrics/meta.go | 4 +- metrics/metrics.go | 8 +- metrics/oracles.go | 2 +- metrics/owner.go | 2 +- metrics/server.go | 10 +- metrics/session.go | 18 +- metrics/stats.go | 2 +- metrics/tikvclient.go | 66 ++- owner/fail_test.go | 2 +- .../testdata/integration_suite_in.json | 3 +- .../testdata/integration_suite_out.json | 17 +- planner/cascades/transformation_rules.go | 12 +- planner/core/cache.go | 4 +- planner/core/cacheable_checker.go | 24 +- planner/core/cacheable_checker_test.go | 110 ++-- planner/core/cbo_test.go | 7 +- planner/core/common_plans.go | 87 ++-- planner/core/exhaust_physical_plans.go | 147 +++--- planner/core/expression_rewriter.go | 44 +- planner/core/find_best_task.go | 157 +++--- planner/core/find_best_task_test.go | 274 ++++++++++ planner/core/initialize.go | 2 +- planner/core/integration_test.go | 99 +++- planner/core/logical_plan_builder.go | 32 +- planner/core/logical_plan_test.go | 20 + planner/core/memtable_predicate_extractor.go | 50 +- .../core/memtable_predicate_extractor_test.go | 4 - planner/core/optimizer.go | 4 + planner/core/physical_plan_test.go | 34 ++ planner/core/plan.go | 7 +- planner/core/plan_to_pb.go | 30 +- planner/core/planbuilder.go | 81 ++- planner/core/planbuilder_test.go | 2 + planner/core/point_get_plan.go | 133 ++--- planner/core/point_get_plan_test.go | 2 +- planner/core/prepare_test.go | 75 ++- planner/core/preprocess.go | 10 +- planner/core/rule_aggregation_elimination.go | 6 +- planner/core/rule_aggregation_push_down.go | 31 +- planner/core/rule_build_key_info.go | 1 + planner/core/rule_column_pruning.go | 6 +- planner/core/rule_decorrelate.go | 2 +- .../core/rule_generate_column_substitute.go | 2 +- planner/core/rule_inject_extra_projection.go | 7 +- planner/core/rule_partition_processor.go | 46 +- planner/core/rule_predicate_push_down.go | 6 +- planner/core/stats.go | 24 +- planner/core/task.go | 76 ++- .../integration_serial_suite_out.json | 6 +- .../core/testdata/integration_suite_in.json | 16 +- .../core/testdata/integration_suite_out.json | 124 ++++- planner/core/testdata/plan_suite_in.json | 24 +- planner/core/testdata/plan_suite_out.json | 253 ++++++--- .../testdata/plan_suite_unexported_in.json | 5 +- .../testdata/plan_suite_unexported_out.json | 13 +- planner/implementation/join.go | 1 - planner/memo/expr_iterator_test.go | 38 +- privilege/privilege.go | 1 + privilege/privileges/cache.go | 4 +- privilege/privileges/cache_test.go | 4 +- privilege/privileges/privileges_test.go | 50 +- server/conn.go | 131 +++-- server/conn_stmt.go | 32 +- server/conn_stmt_test.go | 2 +- server/conn_test.go | 30 +- server/driver.go | 9 +- server/driver_tidb.go | 41 +- server/http_handler.go | 113 +++- server/http_handler_test.go | 94 +++- server/http_status.go | 2 + server/tidb_test.go | 57 ++- session/bootstrap.go | 481 ++++++++++-------- session/bootstrap_test.go | 22 +- session/pessimistic_test.go | 164 +++++- session/session.go | 147 ++++-- session/session_test.go | 47 +- session/tidb.go | 7 +- session/tidb_test.go | 2 +- session/txn.go | 12 +- sessionctx/binloginfo/binloginfo.go | 1 + sessionctx/binloginfo/binloginfo_test.go | 28 + sessionctx/context.go | 2 +- sessionctx/stmtctx/stmtctx.go | 1 + sessionctx/variable/session.go | 48 +- sessionctx/variable/sysvar.go | 18 +- sessionctx/variable/sysvar_test.go | 3 + sessionctx/variable/tidb_vars.go | 17 +- sessionctx/variable/varsutil.go | 23 +- sessionctx/variable/varsutil_test.go | 64 ++- statistics/cmsketch.go | 23 +- statistics/cmsketch_test.go | 2 +- statistics/sample.go | 20 +- statistics/sample_test.go | 2 +- statistics/scalar_test.go | 4 +- statistics/statistics_test.go | 6 +- store/helper/helper.go | 14 +- store/mockstore/cluster/cluster.go | 49 ++ store/mockstore/mocktikv/analyze.go | 2 +- store/mockstore/mocktikv/cluster.go | 21 +- store/mockstore/mocktikv/cluster_test.go | 7 +- store/mockstore/mocktikv/cop_handler_dag.go | 14 +- store/mockstore/mocktikv/executor.go | 8 +- store/mockstore/mocktikv/executor_test.go | 2 +- store/mockstore/mocktikv/mock_tikv_test.go | 10 +- store/mockstore/mocktikv/mvcc_leveldb.go | 12 +- store/mockstore/mocktikv/pd.go | 40 +- store/mockstore/mocktikv/rpc.go | 6 +- store/mockstore/tikv.go | 2 +- store/store_test.go | 2 +- store/tikv/2pc.go | 117 ++++- store/tikv/2pc_slow_test.go | 42 +- store/tikv/2pc_test.go | 19 +- store/tikv/backoff.go | 10 + store/tikv/backoff_test.go | 1 + store/tikv/client.go | 23 +- store/tikv/client_batch.go | 101 +--- store/tikv/client_collapse.go | 92 ++++ store/tikv/client_test.go | 83 --- store/tikv/coprocessor.go | 10 +- store/tikv/delete_range.go | 2 +- store/tikv/delete_range_test.go | 12 +- store/tikv/error.go | 1 + store/tikv/gcworker/gc_worker.go | 296 +++++++---- store/tikv/gcworker/gc_worker_test.go | 421 +++++++++++++-- store/tikv/isolation_test.go | 2 - store/tikv/kv.go | 5 +- store/tikv/lock_resolver.go | 7 +- store/tikv/lock_test.go | 31 +- store/tikv/oracle/oracles/local_test.go | 6 +- store/tikv/pd_codec.go | 25 +- store/tikv/range_task_test.go | 10 +- store/tikv/rawkv_test.go | 12 +- store/tikv/region_cache.go | 259 +++++++--- store/tikv/region_cache_test.go | 43 +- store/tikv/region_request.go | 4 + store/tikv/region_request_test.go | 66 +++ store/tikv/scan.go | 12 +- store/tikv/scan_mock_test.go | 4 +- store/tikv/snapshot_test.go | 36 +- store/tikv/split_test.go | 10 +- store/tikv/store_test.go | 10 +- store/tikv/txn.go | 17 + table/column.go | 2 + table/index.go | 10 +- table/table.go | 16 +- table/tables/index.go | 77 +-- table/tables/index_test.go | 36 +- table/tables/partition.go | 10 +- table/tables/partition_test.go | 12 +- table/tables/tables.go | 111 ++-- table/tables/tables_test.go | 24 +- tablecodec/bench_test.go | 12 +- tablecodec/tablecodec.go | 47 +- tablecodec/tablecodec_test.go | 32 +- tidb-server/main.go | 19 + tidb-server/main_test.go | 18 + tools/check/go.mod | 2 +- tools/check/go.sum | 12 + types/benchmark_test.go | 6 +- types/binary_literal.go | 2 +- types/compare.go | 2 +- types/compare_test.go | 18 +- types/convert.go | 2 +- types/convert_test.go | 6 +- types/datum.go | 10 +- types/datum_test.go | 16 +- types/enum.go | 9 + types/field_type_test.go | 18 +- types/json/binary_functions.go | 44 +- types/parser_driver/special_cmt_ctrl.go | 15 +- types/parser_driver/value_expr.go | 3 +- types/set.go | 9 + types/time.go | 2 +- util/admin/admin.go | 49 +- util/admin/admin_integration_test.go | 21 +- util/admin/admin_test.go | 28 + util/chunk/chunk.go | 2 +- util/chunk/chunk_util.go | 2 +- util/chunk/column.go | 2 +- util/chunk/column_test.go | 2 +- util/chunk/compare.go | 5 +- util/chunk/list.go | 2 +- util/chunk/mutrow.go | 12 + util/collate/collate.go | 2 + util/disk/tracker.go | 3 + util/encrypt/aes.go | 2 +- util/execdetails/execdetails.go | 13 +- util/execdetails/execdetails_test.go | 3 +- util/hint/hint_processor.go | 32 +- util/logutil/log.go | 8 +- util/memory/tracker.go | 53 +- util/memory/tracker_test.go | 49 ++ util/mock/context.go | 2 +- util/ranger/checker.go | 17 +- util/ranger/ranger.go | 4 +- util/ranger/ranger_test.go | 38 ++ util/ranger/testdata/ranger_suite_in.json | 12 + util/ranger/testdata/ranger_suite_out.json | 61 +++ util/rowDecoder/decoder.go | 37 +- util/rowDecoder/decoder_test.go | 5 +- util/rowcodec/bench_test.go | 5 +- util/rowcodec/common.go | 2 +- util/rowcodec/decoder.go | 131 +++-- util/rowcodec/encoder.go | 9 +- util/rowcodec/rowcodec_test.go | 157 ++++-- util/stmtsummary/statement_summary.go | 34 +- util/stmtsummary/statement_summary_test.go | 28 +- util/sys/linux/sys_linux.go | 3 +- util/testkit/testkit.go | 2 +- util/testutil/testutil.go | 2 +- util/timeutil/time.go | 4 +- 432 files changed, 14478 insertions(+), 5111 deletions(-) create mode 100644 .github/workflows/issue_assigned.yml create mode 100644 cmd/importcheck/importcheck.go create mode 100644 docs/design/2020-04-17-sql-sequence.md create mode 100644 docs/design/2020-04-20-brie.md create mode 100644 executor/brie.go create mode 100644 executor/oomtest/oom_test.go create mode 100644 executor/set_config.go create mode 100644 kv/memdb/memdb_norace_test.go create mode 100644 planner/core/find_best_task_test.go mode change 100755 => 100644 session/txn.go create mode 100644 store/mockstore/cluster/cluster.go create mode 100644 store/tikv/client_collapse.go diff --git a/.github/workflows/assign_project.yml b/.github/workflows/assign_project.yml index c1d2477b06211..488734e814219 100644 --- a/.github/workflows/assign_project.yml +++ b/.github/workflows/assign_project.yml @@ -23,7 +23,8 @@ jobs: uses: srggrs/assign-one-project-github-action@1.2.0 if: | contains(github.event.issue.labels.*.name, 'component/planner') || - contains(github.event.issue.labels.*.name, 'component/statistics') + contains(github.event.issue.labels.*.name, 'component/statistics') || + contains(github.event.issue.labels.*.name, 'component/bindinfo') with: project: 'https://github.com/pingcap/tidb/projects/39' column_name: 'Issue Backlog: Need Triage' diff --git a/.github/workflows/issue_assigned.yml b/.github/workflows/issue_assigned.yml new file mode 100644 index 0000000000000..90735cf64b36c --- /dev/null +++ b/.github/workflows/issue_assigned.yml @@ -0,0 +1,20 @@ +name: Auto Assign Project Local + +on: + issues: + types: [assigned] +env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + +jobs: + assign_to_project_column: + runs-on: ubuntu-latest + name: Assign to Project Column + steps: + - name: Run issues assignment to Robust test + uses: srggrs/assign-one-project-github-action@1.2.0 + if: | + contains(github.event.issue.labels.*.name, 'component/test') + with: + project: 'https://github.com/pingcap/tidb/projects/32' + column_name: 'Task Assigned' diff --git a/Makefile b/Makefile index dcc8fc2d8d916..9efc60354ecfd 100644 --- a/Makefile +++ b/Makefile @@ -82,7 +82,7 @@ build: # Install the check tools. check-setup:tools/bin/revive tools/bin/goword tools/bin/gometalinter tools/bin/gosec -check: fmt errcheck lint tidy testSuite check-static vet staticcheck +check: fmt errcheck unconvert lint tidy testSuite check-static vet staticcheck # These need to be fixed before they can be ran regularly check-fail: goword check-slow @@ -90,6 +90,7 @@ check-fail: goword check-slow fmt: @echo "gofmt (simplify)" @gofmt -s -l -w $(FILES) 2>&1 | $(FAIL_ON_STDOUT) + @cd cmd/importcheck && $(GO) run . ../.. goword:tools/bin/goword tools/bin/goword $(FILES) 2>&1 | $(FAIL_ON_STDOUT) @@ -112,6 +113,10 @@ errcheck:tools/bin/errcheck @echo "errcheck" @GO111MODULE=on tools/bin/errcheck -exclude ./tools/check/errcheck_excludes.txt -ignoretests -blank $(PACKAGES) +unconvert:tools/bin/unconvert + @echo "unconvert check" + @GO111MODULE=on tools/bin/unconvert ./... + gogenerate: @echo "go generate ./..." ./tools/check/check-gogenerate.sh @@ -283,6 +288,10 @@ tools/bin/errcheck: tools/check/go.mod cd tools/check; \ $(GO) build -o ../bin/errcheck github.com/kisielk/errcheck +tools/bin/unconvert: tools/check/go.mod + cd tools/check; \ + $(GO) build -o ../bin/unconvert github.com/mdempsky/unconvert + tools/bin/failpoint-ctl: go.mod $(GO) build -o $@ github.com/pingcap/failpoint/failpoint-ctl diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 6c39a33bd2cfc..087b06b3f3ca4 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -32,6 +33,7 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/stmtsummary" @@ -51,10 +53,9 @@ func TestT(t *testing.T) { var _ = Suite(&testSuite{}) type testSuite struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - domain *domain.Domain + cluster cluster.Cluster + store kv.Storage + domain *domain.Domain *parser.Parser } @@ -66,12 +67,15 @@ func (s *testSuite) SetUpSuite(c *C) { flag.Lookup("mockTikv") useMockTikv := *mockTikv if useMockTikv { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) s.store = store @@ -947,6 +951,17 @@ func (s *testSuite) TestHintsSetID(c *C) { c.Assert(len(bindData.Bindings), Equals, 1) bind = bindData.Bindings[0] c.Assert(bind.ID, Equals, "use_index(@`sel_1` `test`.`t` `idx_a`)") + + s.cleanBindingEnv(tk) + err := tk.ExecToErr("create global binding for select * from t using select /*+ non_exist_hint() */ * from t") + c.Assert(terror.ErrorEqual(err, parser.ErrWarnOptimizerHintParseError), IsTrue) + tk.MustExec("create global binding for select * from t where a > 10 using select * from t where a > 10") + bindData = bindHandle.GetBindRecord(hash, sql, "test") + c.Check(bindData, NotNil) + c.Check(bindData.OriginalSQL, Equals, "select * from t where a > ?") + c.Assert(len(bindData.Bindings), Equals, 1) + bind = bindData.Bindings[0] + c.Assert(bind.ID, Equals, "") } func (s *testSuite) TestCapturePlanBaselineIgnoreTiFlash(c *C) { diff --git a/bindinfo/cache.go b/bindinfo/cache.go index f5ba9693673aa..7fd44b934e892 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -117,7 +117,7 @@ func (br *BindRecord) prepareHints(sctx sessionctx.Context) error { return err } } - hintsSet, err := hint.ParseHintsSet(p, bind.BindSQL, bind.Charset, bind.Collation, br.Db) + hintsSet, warns, err := hint.ParseHintsSet(p, bind.BindSQL, bind.Charset, bind.Collation, br.Db) if err != nil { return err } @@ -125,6 +125,12 @@ func (br *BindRecord) prepareHints(sctx sessionctx.Context) error { if err != nil { return err } + // For `create global binding for select * from t using select * from t`, we allow it though hintsStr is empty. + // For `create global binding for select * from t using select /*+ non_exist_hint() */ * from t`, + // the hint is totally invaild, we escalate warning to error. + if hintsStr == "" && len(warns) > 0 { + return warns[0] + } br.Bindings[i].Hint = hintsSet br.Bindings[i].ID = hintsStr } diff --git a/cmd/ddltest/ddl_test.go b/cmd/ddltest/ddl_test.go index 818d641be1ee2..f78112b7e5c27 100644 --- a/cmd/ddltest/ddl_test.go +++ b/cmd/ddltest/ddl_test.go @@ -745,7 +745,7 @@ func (s *TestDDLSuite) TestSimpleConflictUpdate(c *C) { k := randomNum(rowCount) s.mustExec(c, fmt.Sprintf("update test_conflict_update set c2 = %d where c1 = %d", defaultValue, k)) mu.Lock() - keysMap[int64(k)] = int64(defaultValue) + keysMap[int64(k)] = defaultValue mu.Unlock() } }() diff --git a/cmd/ddltest/index_test.go b/cmd/ddltest/index_test.go index 112527242c2f2..53f89a286b747 100644 --- a/cmd/ddltest/index_test.go +++ b/cmd/ddltest/index_test.go @@ -78,8 +78,8 @@ func (s *TestDDLSuite) checkAddIndex(c *C, indexInfo *model.IndexInfo) { } c.Assert(err, IsNil) - c.Assert(handles, HasKey, h) - delete(handles, h) + c.Assert(handles, HasKey, h.IntValue()) + delete(handles, h.IntValue()) } c.Assert(handles, HasLen, 0) @@ -116,7 +116,7 @@ func (s *TestDDLSuite) checkDropIndex(c *C, indexInfo *model.IndexInfo) { } c.Assert(err, IsNil) - handles[h] = struct{}{} + handles[h.IntValue()] = struct{}{} } // TODO: Uncomment this after apply pool is finished diff --git a/cmd/explaintest/r/black_list.result b/cmd/explaintest/r/black_list.result index 9408bc9f44230..86306cabfd399 100644 --- a/cmd/explaintest/r/black_list.result +++ b/cmd/explaintest/r/black_list.result @@ -22,7 +22,7 @@ id estRows task access object operator info TableReader_7 3323.33 root data:Selection_6 └─Selection_6 3323.33 cop[tikv] lt(test.t.a, 1) └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo -insert into mysql.expr_pushdown_blacklist values('<'); +insert into mysql.expr_pushdown_blacklist values('<', 'tikv,tiflash,tidb', 'for test'); admin reload expr_pushdown_blacklist; explain select * from t where a < 1; @@ -30,7 +30,7 @@ id estRows task access object operator info Selection_5 8000.00 root lt(test.t.a, 1) └─TableReader_7 10000.00 root data:TableFullScan_6 └─TableFullScan_6 10000.00 cop[tikv] table:t keep order:false, stats:pseudo -delete from mysql.expr_pushdown_blacklist where name='<'; +delete from mysql.expr_pushdown_blacklist where name='<' and store_type = 'tikv,tiflash,tidb' and reason = 'for test'; admin reload expr_pushdown_blacklist; explain select * from t where a < 1; @@ -38,7 +38,7 @@ id estRows task access object operator info TableReader_7 3323.33 root data:Selection_6 └─Selection_6 3323.33 cop[tikv] lt(test.t.a, 1) └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo -insert into mysql.expr_pushdown_blacklist values('lt'); +insert into mysql.expr_pushdown_blacklist values('lt', 'tikv,tiflash,tidb', 'for test'); admin reload expr_pushdown_blacklist; explain select * from t where a < 1; @@ -46,7 +46,7 @@ id estRows task access object operator info Selection_5 8000.00 root lt(test.t.a, 1) └─TableReader_7 10000.00 root data:TableFullScan_6 └─TableFullScan_6 10000.00 cop[tikv] table:t keep order:false, stats:pseudo -delete from mysql.expr_pushdown_blacklist where name='lt'; +delete from mysql.expr_pushdown_blacklist where name='lt' and store_type = 'tikv,tiflash,tidb' and reason = 'for test'; admin reload expr_pushdown_blacklist; explain select * from t where a < 1; diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 231d5782c3e7c..6d2cacb8402e6 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -164,10 +164,10 @@ id estRows task access object operator info Union_17 26000.00 root ├─HashAgg_21 16000.00 root group by:Column#10, funcs:firstrow(Column#12)->Column#10 │ └─Union_22 16000.00 root -│ ├─StreamAgg_27 8000.00 root group by:test.t2.c1, funcs:firstrow(test.t2.c1)->Column#12, funcs:firstrow(test.t2.c1)->Column#10 +│ ├─StreamAgg_27 8000.00 root group by:test.t2.c1, funcs:firstrow(test.t2.c1)->Column#12 │ │ └─IndexReader_40 10000.00 root index:IndexFullScan_39 │ │ └─IndexFullScan_39 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:true, stats:pseudo -│ └─StreamAgg_45 8000.00 root group by:test.t2.c1, funcs:firstrow(test.t2.c1)->Column#12, funcs:firstrow(test.t2.c1)->Column#10 +│ └─StreamAgg_45 8000.00 root group by:test.t2.c1, funcs:firstrow(test.t2.c1)->Column#12 │ └─IndexReader_58 10000.00 root index:IndexFullScan_57 │ └─IndexFullScan_57 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:true, stats:pseudo └─IndexReader_63 10000.00 root index:IndexFullScan_62 @@ -176,13 +176,13 @@ explain select c1 from t2 union all select c1 from t2 union select c1 from t2; id estRows task access object operator info HashAgg_18 24000.00 root group by:Column#10, funcs:firstrow(Column#11)->Column#10 └─Union_19 24000.00 root - ├─StreamAgg_24 8000.00 root group by:test.t2.c1, funcs:firstrow(test.t2.c1)->Column#11, funcs:firstrow(test.t2.c1)->Column#10 + ├─StreamAgg_24 8000.00 root group by:test.t2.c1, funcs:firstrow(test.t2.c1)->Column#11 │ └─IndexReader_37 10000.00 root index:IndexFullScan_36 │ └─IndexFullScan_36 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:true, stats:pseudo - ├─StreamAgg_42 8000.00 root group by:test.t2.c1, funcs:firstrow(test.t2.c1)->Column#11, funcs:firstrow(test.t2.c1)->Column#10 + ├─StreamAgg_42 8000.00 root group by:test.t2.c1, funcs:firstrow(test.t2.c1)->Column#11 │ └─IndexReader_55 10000.00 root index:IndexFullScan_54 │ └─IndexFullScan_54 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:true, stats:pseudo - └─StreamAgg_60 8000.00 root group by:test.t2.c1, funcs:firstrow(test.t2.c1)->Column#11, funcs:firstrow(test.t2.c1)->Column#10 + └─StreamAgg_60 8000.00 root group by:test.t2.c1, funcs:firstrow(test.t2.c1)->Column#11 └─IndexReader_73 10000.00 root index:IndexFullScan_72 └─IndexFullScan_72 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:true, stats:pseudo select * from information_schema.tidb_indexes where table_name='t4'; @@ -669,17 +669,17 @@ id estRows task access object operator info Sort_13 2.00 root Column#3:asc └─HashAgg_17 2.00 root group by:Column#3, funcs:firstrow(Column#6)->Column#3 └─Union_18 2.00 root - ├─HashAgg_19 1.00 root group by:1, funcs:firstrow(0)->Column#6, funcs:firstrow(0)->Column#3 + ├─HashAgg_19 1.00 root group by:1, funcs:firstrow(0)->Column#6 │ └─TableDual_22 1.00 root rows:1 - └─HashAgg_25 1.00 root group by:1, funcs:firstrow(1)->Column#6, funcs:firstrow(1)->Column#3 + └─HashAgg_25 1.00 root group by:1, funcs:firstrow(1)->Column#6 └─TableDual_28 1.00 root rows:1 explain SELECT 0 AS a FROM dual UNION (SELECT 1 AS a FROM dual ORDER BY a); id estRows task access object operator info HashAgg_15 2.00 root group by:Column#3, funcs:firstrow(Column#6)->Column#3 └─Union_16 2.00 root - ├─HashAgg_17 1.00 root group by:1, funcs:firstrow(0)->Column#6, funcs:firstrow(0)->Column#3 + ├─HashAgg_17 1.00 root group by:1, funcs:firstrow(0)->Column#6 │ └─TableDual_20 1.00 root rows:1 - └─StreamAgg_27 1.00 root group by:Column#1, funcs:firstrow(Column#1)->Column#6, funcs:firstrow(Column#1)->Column#3 + └─StreamAgg_27 1.00 root group by:Column#1, funcs:firstrow(Column#1)->Column#6 └─Projection_32 1.00 root 1->Column#1 └─TableDual_33 1.00 root rows:1 create table t (i int key, j int, unique key (i, j)); diff --git a/cmd/explaintest/r/explain_generate_column_substitute.result b/cmd/explaintest/r/explain_generate_column_substitute.result index bd32f690d888b..f0e442aa24d4f 100644 --- a/cmd/explaintest/r/explain_generate_column_substitute.result +++ b/cmd/explaintest/r/explain_generate_column_substitute.result @@ -1,11 +1,11 @@ use test; drop table if exists t; -create table t(a int, b real, c int as ((a+1)) virtual, e real as ((b+a))); +create table t(a int, b real, c bigint as ((a+1)) virtual, e real as ((b+a))); insert into t values (1, 2.0, default, default), (2, 2.1, default, default), (5, 3.0, default, default), (5, -1.0, default, default), (0, 0.0, default, default), (-1, -2.0, default, default), (0, 0, default, default); alter table t add index idx_c(c); alter table t add index idx_e(e); -set @@sql_mode="" +set @@sql_mode=""; desc select * from t where a+1=3; id estRows task access object operator info IndexLookUp_10 10.00 root @@ -324,3 +324,58 @@ Delete_4 N/A root N/A └─IndexLookUp_11 10.00 root ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:expr_idx_e(_V$_expr_idx_e_0) range:[0,0], keep order:false, stats:pseudo └─TableRowIDScan_10(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo +drop table if exists t0; +create table t0(c0 float(24), c1 double as (c0) unique); +explain select c0 from t0; +id estRows task access object operator info +TableReader_5 10000.00 root data:TableFullScan_4 +└─TableFullScan_4 10000.00 cop[tikv] table:t0 keep order:false, stats:pseudo + -- TableRead +drop table if exists t0; +create table t0(c0 float(25), c1 double as (c0) unique); +explain select c0 from t0; +id estRows task access object operator info +IndexReader_7 10000.00 root index:IndexFullScan_6 +└─IndexFullScan_6 10000.00 cop[tikv] table:t0, index:c1(c1) keep order:false, stats:pseudo + -- IndexRead +drop table if exists t0; +create table t0(c0 double(24), c1 double as (c0) unique); +explain select c0 from t0; +id estRows task access object operator info +IndexReader_7 10000.00 root index:IndexFullScan_6 +└─IndexFullScan_6 10000.00 cop[tikv] table:t0, index:c1(c1) keep order:false, stats:pseudo + -- IndexRead +drop table if exists t0; +create table t0(c0 double(25), c1 double as (c0) unique); +explain select c0 from t0; +id estRows task access object operator info +IndexReader_7 10000.00 root index:IndexFullScan_6 +└─IndexFullScan_6 10000.00 cop[tikv] table:t0, index:c1(c1) keep order:false, stats:pseudo + -- IndexRead +drop table if exists t0; +create table t0(c0 float(24), c1 float as (c0) unique); +explain select c0 from t0; +id estRows task access object operator info +IndexReader_7 10000.00 root index:IndexFullScan_6 +└─IndexFullScan_6 10000.00 cop[tikv] table:t0, index:c1(c1) keep order:false, stats:pseudo + -- IndexRead +drop table if exists t0; +create table t0(c0 float(25), c1 float as (c0) unique); +explain select c0 from t0; +id estRows task access object operator info +TableReader_5 10000.00 root data:TableFullScan_4 +└─TableFullScan_4 10000.00 cop[tikv] table:t0 keep order:false, stats:pseudo + -- TableRead +drop table if exists t0; +create table t0(c0 double(24), c1 float as (c0) unique); +explain select c0 from t0; +id estRows task access object operator info +TableReader_5 10000.00 root data:TableFullScan_4 +└─TableFullScan_4 10000.00 cop[tikv] table:t0 keep order:false, stats:pseudo + -- TableRead +drop table if exists t0; +create table t0(c0 double(25), c1 float as (c0) unique); +explain select c0 from t0; +id estRows task access object operator info +TableReader_5 10000.00 root data:TableFullScan_4 +└─TableFullScan_4 10000.00 cop[tikv] table:t0 keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/partition_pruning.result b/cmd/explaintest/r/partition_pruning.result index c7a376bb87bbd..da4222bad3a22 100644 --- a/cmd/explaintest/r/partition_pruning.result +++ b/cmd/explaintest/r/partition_pruning.result @@ -1,5 +1,5 @@ set @@session.tidb_enable_table_partition=1; -drop table if exists t1,t2,t3,t4,t5,t6,t7,t8,t9; +drop table if exists t0,t1,t2,t3,t4,t5,t6,t7,t8,t9; # # Bug#53806: Wrong estimates for range query in partitioned MyISAM table # Bug#46754: 'rows' field doesn't reflect partition pruning @@ -4198,3 +4198,177 @@ id estRows task access object operator info TableReader_8 8000.00 root data:Selection_7 └─Selection_7 8000.00 cop[tikv] or(and(eq(test.t.a, 11), eq(test.t.b, 1)), and(eq(test.t.a, 12), eq(test.t.b, 1))) └─TableFullScan_6 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +drop table if exists t; +create table t (ts timestamp(3) not null default current_timestamp(3)) +partition by range (floor(unix_timestamp(ts))) ( +partition p0 values less than (unix_timestamp('2020-04-05 00:00:00')), +partition p1 values less than (unix_timestamp('2020-04-15 00:00:00')), +partition p2 values less than (unix_timestamp('2020-04-25 00:00:00')) +); +explain select * from t where ts = '2020-04-06 00:00:00' -- p1; +id estRows task access object operator info +TableReader_8 10.00 root data:Selection_7 +└─Selection_7 10.00 cop[tikv] eq(test.t.ts, 2020-04-06 00:00:00.000000) + └─TableFullScan_6 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +explain select * from t where ts = '2020-04-05 00:00:00.001' -- p1; +id estRows task access object operator info +TableReader_8 10.00 root data:Selection_7 +└─Selection_7 10.00 cop[tikv] eq(test.t.ts, 2020-04-05 00:00:00.001000) + └─TableFullScan_6 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +explain select * from t where ts > '2020-04-15 00:00:00' -- p2; +id estRows task access object operator info +TableReader_8 3333.33 root data:Selection_7 +└─Selection_7 3333.33 cop[tikv] gt(test.t.ts, 2020-04-15 00:00:00.000000) + └─TableFullScan_6 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +explain select * from t where ts > '2020-04-14 23:59:59.999' -- p1,p2; +id estRows task access object operator info +Union_8 6666.67 root +├─TableReader_11 3333.33 root data:Selection_10 +│ └─Selection_10 3333.33 cop[tikv] gt(test.t.ts, 2020-04-14 23:59:59.999000) +│ └─TableFullScan_9 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +└─TableReader_14 3333.33 root data:Selection_13 + └─Selection_13 3333.33 cop[tikv] gt(test.t.ts, 2020-04-14 23:59:59.999000) + └─TableFullScan_12 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +explain select * from t where ts > '2020-04-15 00:00:00.001' -- p2; +id estRows task access object operator info +TableReader_8 3333.33 root data:Selection_7 +└─Selection_7 3333.33 cop[tikv] gt(test.t.ts, 2020-04-15 00:00:00.001000) + └─TableFullScan_6 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +explain select * from t where ts > '2020-04-26 00:00:00.001' -- dual; +id estRows task access object operator info +TableDual_6 0.00 root rows:0 +explain select * from t where ts >= '2020-04-04 12:22:32' -- p0,p1,p2; +id estRows task access object operator info +Union_9 10000.00 root +├─TableReader_12 3333.33 root data:Selection_11 +│ └─Selection_11 3333.33 cop[tikv] ge(test.t.ts, 2020-04-04 12:22:32.000000) +│ └─TableFullScan_10 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +├─TableReader_15 3333.33 root data:Selection_14 +│ └─Selection_14 3333.33 cop[tikv] ge(test.t.ts, 2020-04-04 12:22:32.000000) +│ └─TableFullScan_13 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +└─TableReader_18 3333.33 root data:Selection_17 + └─Selection_17 3333.33 cop[tikv] ge(test.t.ts, 2020-04-04 12:22:32.000000) + └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +explain select * from t where ts >= '2020-04-05 00:00:00' -- p1,p2; +id estRows task access object operator info +Union_8 6666.67 root +├─TableReader_11 3333.33 root data:Selection_10 +│ └─Selection_10 3333.33 cop[tikv] ge(test.t.ts, 2020-04-05 00:00:00.000000) +│ └─TableFullScan_9 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +└─TableReader_14 3333.33 root data:Selection_13 + └─Selection_13 3333.33 cop[tikv] ge(test.t.ts, 2020-04-05 00:00:00.000000) + └─TableFullScan_12 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +explain select * from t where ts >= '2020-04-25 00:00:00' -- dual; +id estRows task access object operator info +TableDual_6 0.00 root rows:0 +explain select * from t where ts < '2020-04-25 00:00:00' -- p0,p1,p2; +id estRows task access object operator info +Union_9 9970.00 root +├─TableReader_12 3323.33 root data:Selection_11 +│ └─Selection_11 3323.33 cop[tikv] lt(test.t.ts, 2020-04-25 00:00:00.000000) +│ └─TableFullScan_10 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +├─TableReader_15 3323.33 root data:Selection_14 +│ └─Selection_14 3323.33 cop[tikv] lt(test.t.ts, 2020-04-25 00:00:00.000000) +│ └─TableFullScan_13 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +└─TableReader_18 3323.33 root data:Selection_17 + └─Selection_17 3323.33 cop[tikv] lt(test.t.ts, 2020-04-25 00:00:00.000000) + └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +explain select * from t where ts < '2020-04-15 00:00:00.001' -- p0,p1,p2; +id estRows task access object operator info +Union_9 9970.00 root +├─TableReader_12 3323.33 root data:Selection_11 +│ └─Selection_11 3323.33 cop[tikv] lt(test.t.ts, 2020-04-15 00:00:00.001000) +│ └─TableFullScan_10 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +├─TableReader_15 3323.33 root data:Selection_14 +│ └─Selection_14 3323.33 cop[tikv] lt(test.t.ts, 2020-04-15 00:00:00.001000) +│ └─TableFullScan_13 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +└─TableReader_18 3323.33 root data:Selection_17 + └─Selection_17 3323.33 cop[tikv] lt(test.t.ts, 2020-04-15 00:00:00.001000) + └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +explain select * from t where ts < '2020-04-15 00:00:00' -- expect perfect : p0,p1, obtain: p0,p1,p2; +id estRows task access object operator info +Union_9 9970.00 root +├─TableReader_12 3323.33 root data:Selection_11 +│ └─Selection_11 3323.33 cop[tikv] lt(test.t.ts, 2020-04-15 00:00:00.000000) +│ └─TableFullScan_10 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +├─TableReader_15 3323.33 root data:Selection_14 +│ └─Selection_14 3323.33 cop[tikv] lt(test.t.ts, 2020-04-15 00:00:00.000000) +│ └─TableFullScan_13 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +└─TableReader_18 3323.33 root data:Selection_17 + └─Selection_17 3323.33 cop[tikv] lt(test.t.ts, 2020-04-15 00:00:00.000000) + └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +explain select * from t where ts < '2020-04-14 23:59:59.999' -- p0,p1; +id estRows task access object operator info +Union_8 6646.67 root +├─TableReader_11 3323.33 root data:Selection_10 +│ └─Selection_10 3323.33 cop[tikv] lt(test.t.ts, 2020-04-14 23:59:59.999000) +│ └─TableFullScan_9 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +└─TableReader_14 3323.33 root data:Selection_13 + └─Selection_13 3323.33 cop[tikv] lt(test.t.ts, 2020-04-14 23:59:59.999000) + └─TableFullScan_12 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +explain select * from t where ts < '2020-04-03 00:00:00' -- p0; +id estRows task access object operator info +TableReader_8 3323.33 root data:Selection_7 +└─Selection_7 3323.33 cop[tikv] lt(test.t.ts, 2020-04-03 00:00:00.000000) + └─TableFullScan_6 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +explain select * from t where ts < '2021-05-03 00:00:00' -- p0,p1,p2; +id estRows task access object operator info +Union_9 9970.00 root +├─TableReader_12 3323.33 root data:Selection_11 +│ └─Selection_11 3323.33 cop[tikv] lt(test.t.ts, 2021-05-03 00:00:00.000000) +│ └─TableFullScan_10 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +├─TableReader_15 3323.33 root data:Selection_14 +│ └─Selection_14 3323.33 cop[tikv] lt(test.t.ts, 2021-05-03 00:00:00.000000) +│ └─TableFullScan_13 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +└─TableReader_18 3323.33 root data:Selection_17 + └─Selection_17 3323.33 cop[tikv] lt(test.t.ts, 2021-05-03 00:00:00.000000) + └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +explain select * from t where ts <= '2020-04-05 00:00:00' -- p0,p1; +id estRows task access object operator info +Union_8 6646.67 root +├─TableReader_11 3323.33 root data:Selection_10 +│ └─Selection_10 3323.33 cop[tikv] le(test.t.ts, 2020-04-05 00:00:00.000000) +│ └─TableFullScan_9 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +└─TableReader_14 3323.33 root data:Selection_13 + └─Selection_13 3323.33 cop[tikv] le(test.t.ts, 2020-04-05 00:00:00.000000) + └─TableFullScan_12 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +explain select * from t where ts <= '2020-04-03 00:00:00' -- p0; +id estRows task access object operator info +TableReader_8 3323.33 root data:Selection_7 +└─Selection_7 3323.33 cop[tikv] le(test.t.ts, 2020-04-03 00:00:00.000000) + └─TableFullScan_6 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +explain select * from t where ts <= '2020-04-14 23:59:59.123' -- p0,p1; +id estRows task access object operator info +Union_8 6646.67 root +├─TableReader_11 3323.33 root data:Selection_10 +│ └─Selection_10 3323.33 cop[tikv] le(test.t.ts, 2020-04-14 23:59:59.123000) +│ └─TableFullScan_9 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +└─TableReader_14 3323.33 root data:Selection_13 + └─Selection_13 3323.33 cop[tikv] le(test.t.ts, 2020-04-14 23:59:59.123000) + └─TableFullScan_12 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +explain select * from t where ts <= '2020-04-25 00:00:00' -- p0,p1,p2; +id estRows task access object operator info +Union_9 9970.00 root +├─TableReader_12 3323.33 root data:Selection_11 +│ └─Selection_11 3323.33 cop[tikv] le(test.t.ts, 2020-04-25 00:00:00.000000) +│ └─TableFullScan_10 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +├─TableReader_15 3323.33 root data:Selection_14 +│ └─Selection_14 3323.33 cop[tikv] le(test.t.ts, 2020-04-25 00:00:00.000000) +│ └─TableFullScan_13 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo +└─TableReader_18 3323.33 root data:Selection_17 + └─Selection_17 3323.33 cop[tikv] le(test.t.ts, 2020-04-25 00:00:00.000000) + └─TableFullScan_16 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo +explain select * from t where ts > '2020-04-25 00:00:00' or ts < '2020-01-02 00:00:00' -- p0; +id estRows task access object operator info +TableReader_8 6656.67 root data:Selection_7 +└─Selection_7 6656.67 cop[tikv] or(gt(test.t.ts, 2020-04-25 00:00:00.000000), lt(test.t.ts, 2020-01-02 00:00:00.000000)) + └─TableFullScan_6 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +explain select * from t where ts > '2020-04-02 00:00:00' and ts < '2020-04-07 00:00:00' -- p0,p1; +id estRows task access object operator info +Union_8 500.00 root +├─TableReader_11 250.00 root data:Selection_10 +│ └─Selection_10 250.00 cop[tikv] gt(test.t.ts, 2020-04-02 00:00:00.000000), lt(test.t.ts, 2020-04-07 00:00:00.000000) +│ └─TableFullScan_9 10000.00 cop[tikv] table:t, partition:p0 keep order:false, stats:pseudo +└─TableReader_14 250.00 root data:Selection_13 + └─Selection_13 250.00 cop[tikv] gt(test.t.ts, 2020-04-02 00:00:00.000000), lt(test.t.ts, 2020-04-07 00:00:00.000000) + └─TableFullScan_12 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo diff --git a/cmd/explaintest/t/black_list.test b/cmd/explaintest/t/black_list.test index 6e93cb4943046..ef03c3f72519d 100644 --- a/cmd/explaintest/t/black_list.test +++ b/cmd/explaintest/t/black_list.test @@ -16,25 +16,25 @@ admin reload opt_rule_blacklist; explain select * from t where a < 1; -insert into mysql.expr_pushdown_blacklist values('<'); +insert into mysql.expr_pushdown_blacklist values('<', 'tikv,tiflash,tidb', 'for test'); admin reload expr_pushdown_blacklist; explain select * from t where a < 1; -delete from mysql.expr_pushdown_blacklist where name='<'; +delete from mysql.expr_pushdown_blacklist where name='<' and store_type = 'tikv,tiflash,tidb' and reason = 'for test'; admin reload expr_pushdown_blacklist; explain select * from t where a < 1; -insert into mysql.expr_pushdown_blacklist values('lt'); +insert into mysql.expr_pushdown_blacklist values('lt', 'tikv,tiflash,tidb', 'for test'); admin reload expr_pushdown_blacklist; explain select * from t where a < 1; -delete from mysql.expr_pushdown_blacklist where name='lt'; +delete from mysql.expr_pushdown_blacklist where name='lt' and store_type = 'tikv,tiflash,tidb' and reason = 'for test'; admin reload expr_pushdown_blacklist; diff --git a/cmd/explaintest/t/explain_generate_column_substitute.test b/cmd/explaintest/t/explain_generate_column_substitute.test index 140cab1cc4056..ee9008bf32351 100644 --- a/cmd/explaintest/t/explain_generate_column_substitute.test +++ b/cmd/explaintest/t/explain_generate_column_substitute.test @@ -1,13 +1,13 @@ use test; drop table if exists t; -create table t(a int, b real, c int as ((a+1)) virtual, e real as ((b+a))); +create table t(a int, b real, c bigint as ((a+1)) virtual, e real as ((b+a))); insert into t values (1, 2.0, default, default), (2, 2.1, default, default), (5, 3.0, default, default), (5, -1.0, default, default), (0, 0.0, default, default), (-1, -2.0, default, default), (0, 0, default, default); alter table t add index idx_c(c); alter table t add index idx_e(e); -set @@sql_mode="" +set @@sql_mode=""; # test generate column substitution # substitute where @@ -132,3 +132,36 @@ desc update t set a=2, b = 3 where b+a = 3; # test delete desc delete from t where a+1 = 3; desc delete from t where b+a = 0; + +# test flen for float and double +drop table if exists t0; +create table t0(c0 float(24), c1 double as (c0) unique); +explain select c0 from t0; -- TableRead + +drop table if exists t0; +create table t0(c0 float(25), c1 double as (c0) unique); +explain select c0 from t0; -- IndexRead + +drop table if exists t0; +create table t0(c0 double(24), c1 double as (c0) unique); +explain select c0 from t0; -- IndexRead + +drop table if exists t0; +create table t0(c0 double(25), c1 double as (c0) unique); +explain select c0 from t0; -- IndexRead + +drop table if exists t0; +create table t0(c0 float(24), c1 float as (c0) unique); +explain select c0 from t0; -- IndexRead + +drop table if exists t0; +create table t0(c0 float(25), c1 float as (c0) unique); +explain select c0 from t0; -- TableRead + +drop table if exists t0; +create table t0(c0 double(24), c1 float as (c0) unique); +explain select c0 from t0; -- TableRead + +drop table if exists t0; +create table t0(c0 double(25), c1 float as (c0) unique); +explain select c0 from t0; -- TableRead diff --git a/cmd/explaintest/t/partition_pruning.test b/cmd/explaintest/t/partition_pruning.test index 867e80da13d19..e7741b3196a0c 100644 --- a/cmd/explaintest/t/partition_pruning.test +++ b/cmd/explaintest/t/partition_pruning.test @@ -6,7 +6,7 @@ set @@session.tidb_enable_table_partition=1; -- source include/have_partition.inc --disable_warnings -drop table if exists t1,t2,t3,t4,t5,t6,t7,t8,t9; +drop table if exists t0,t1,t2,t3,t4,t5,t6,t7,t8,t9; --enable_warnings --echo # @@ -976,3 +976,38 @@ PARTITION BY RANGE COLUMNS(a) ( ); desc select * from t where a = 11 and b = 1 or a = 12 and b = 1; + +# +# MySQL doesn't support partition pruning for 'floor(unix_timestamp(ts))' but it works on TiDB +# https://github.com/pingcap/tidb/issues/16354 +# +drop table if exists t; +create table t (ts timestamp(3) not null default current_timestamp(3)) +partition by range (floor(unix_timestamp(ts))) ( + partition p0 values less than (unix_timestamp('2020-04-05 00:00:00')), + partition p1 values less than (unix_timestamp('2020-04-15 00:00:00')), + partition p2 values less than (unix_timestamp('2020-04-25 00:00:00')) +); + +explain select * from t where ts = '2020-04-06 00:00:00' -- p1; +explain select * from t where ts = '2020-04-05 00:00:00.001' -- p1; +explain select * from t where ts > '2020-04-15 00:00:00' -- p2; +explain select * from t where ts > '2020-04-14 23:59:59.999' -- p1,p2; +explain select * from t where ts > '2020-04-15 00:00:00.001' -- p2; +explain select * from t where ts > '2020-04-26 00:00:00.001' -- dual; +explain select * from t where ts >= '2020-04-04 12:22:32' -- p0,p1,p2; +explain select * from t where ts >= '2020-04-05 00:00:00' -- p1,p2; +explain select * from t where ts >= '2020-04-25 00:00:00' -- dual; +explain select * from t where ts < '2020-04-25 00:00:00' -- p0,p1,p2; +explain select * from t where ts < '2020-04-15 00:00:00.001' -- p0,p1,p2; +explain select * from t where ts < '2020-04-15 00:00:00' -- expect perfect : p0,p1, obtain: p0,p1,p2; +explain select * from t where ts < '2020-04-14 23:59:59.999' -- p0,p1; +explain select * from t where ts < '2020-04-03 00:00:00' -- p0; +explain select * from t where ts < '2021-05-03 00:00:00' -- p0,p1,p2; +explain select * from t where ts <= '2020-04-05 00:00:00' -- p0,p1; +explain select * from t where ts <= '2020-04-03 00:00:00' -- p0; +explain select * from t where ts <= '2020-04-14 23:59:59.123' -- p0,p1; +explain select * from t where ts <= '2020-04-25 00:00:00' -- p0,p1,p2; +explain select * from t where ts > '2020-04-25 00:00:00' or ts < '2020-01-02 00:00:00' -- p0; +explain select * from t where ts > '2020-04-02 00:00:00' and ts < '2020-04-07 00:00:00' -- p0,p1; + diff --git a/cmd/importcheck/importcheck.go b/cmd/importcheck/importcheck.go new file mode 100644 index 0000000000000..45e9cc2ab6365 --- /dev/null +++ b/cmd/importcheck/importcheck.go @@ -0,0 +1,119 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "errors" + "flag" + "fmt" + "go/ast" + "go/parser" + "go/token" + "io/ioutil" + "os" + "path/filepath" + "strings" + + "github.com/pingcap/tidb/util/hack" +) + +func main() { + err := run() + if err != nil { + fmt.Fprintf(os.Stderr, "import check fail: %s\n", err) + os.Exit(1) + } +} + +func run() error { + flag.Parse() + + if flag.NArg() != 1 { + return errors.New("need given root folder param") + } + + root, err := filepath.EvalSymlinks(flag.Arg(0)) + if err != nil { + return fmt.Errorf("eval symlinks error: %s", err) + } + + return filepath.Walk(root, func(path string, info os.FileInfo, err error) error { + if info.IsDir() { + return nil + } + if !strings.HasSuffix(path, ".go") { + return nil + } + return checkFile(path) + }) +} + +func checkFile(path string) error { + src, err := ioutil.ReadFile(path) + if err != nil { + return err + } + file, err := parser.ParseFile(token.NewFileSet(), path, src, parser.AllErrors|parser.ParseComments) + if err != nil { + return err + } + + var importSpecs []*ast.ImportSpec + for _, d := range file.Decls { + if genDecl, ok := d.(*ast.GenDecl); ok { + if genDecl.Tok != token.IMPORT { + continue + } + for _, spec := range genDecl.Specs { + if importSpec, ok := spec.(*ast.ImportSpec); ok { + importSpecs = append(importSpecs, importSpec) + } + } + } + } + + var preIsStd bool + for i, im := range importSpecs { + stdImport := !strings.Contains(im.Path.Value, ".") + if stdImport { + // std import + if i == 0 { + preIsStd = true + continue + } + if !preIsStd { + return errors.New(fmt.Sprintf("stdlib %s need be group together and before non-stdlib group in %s", im.Path.Value, path)) + } + continue + } + // non-std import + if i != 0 { + if !preIsStd { + continue + } + if !checkSepWithNewline(src, importSpecs[i-1].Path.Pos(), im.Path.Pos()) { + return errors.New(fmt.Sprintf("non-stdlib %s need be group together and after stdlib group in %s", im.Path.Value, path)) + } + preIsStd = false + } + } + + return nil +} + +func checkSepWithNewline(src []byte, pre token.Pos, cur token.Pos) bool { + preSrc := src[pre:cur] + newLine := strings.Count(string(hack.String(preSrc)), "\n") + return newLine == 2 +} diff --git a/cmd/importer/stats.go b/cmd/importer/stats.go index 0be8ab37903f7..2f258462b93dc 100644 --- a/cmd/importer/stats.go +++ b/cmd/importer/stats.go @@ -83,7 +83,7 @@ func getValidPrefix(lower, upper string) string { randCh := uint8(rand.Intn(int(upper[i]-lower[i]))) + lower[i] newBytes := make([]byte, i, i+1) copy(newBytes, lower[:i]) - newBytes = append(newBytes, byte(randCh)) + newBytes = append(newBytes, randCh) return string(newBytes) } } diff --git a/config/config.go b/config/config.go index acf0936719b59..59a2d047f853f 100644 --- a/config/config.go +++ b/config/config.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/sys/storage" tracing "github.com/uber/jaeger-client-go/config" "go.uber.org/zap" ) @@ -51,6 +50,8 @@ const ( DefPort = 4000 // DefStatusPort is the default status port of TiBD DefStatusPort = 10080 + // DefStoreLivenessTimeout is the default value for store liveness timeout. + DefStoreLivenessTimeout = "120s" ) // Valid config maps @@ -86,7 +87,7 @@ type Config struct { MemQuotaQuery int64 `toml:"mem-quota-query" json:"mem-quota-query"` // TempStorageQuota describe the temporary storage Quota during query exector when OOMUseTmpStorage is enabled // If the quota exceed the capacity of the TempStoragePath, the tidb-server would exit with fatal error - TempStorageQuota int64 `toml:"temp-storage-quota" json:"temp-storage-quota"` // Bytes + TempStorageQuota int64 `toml:"tmp-storage-quota" json:"tmp-storage-quota"` // Bytes EnableStreaming bool `toml:"enable-streaming" json:"enable-streaming"` EnableBatchDML bool `toml:"enable-batch-dml" json:"enable-batch-dml"` TxnLocalLatches TxnLocalLatches `toml:"-" json:"-"` @@ -130,9 +131,8 @@ type Config struct { NewCollationsEnabledOnFirstBootstrap bool `toml:"new_collations_enabled_on_first_bootstrap" json:"new_collations_enabled_on_first_bootstrap"` // Experimental contains parameters for experimental features. Experimental Experimental `toml:"experimental" json:"experimental"` - // EnableDynamicConfig enables the TiDB to fetch configs from PD and update itself during runtime. - // see https://github.com/pingcap/tidb/pull/13660 for more details. - EnableDynamicConfig bool `toml:"enable-dynamic-config" json:"enable-dynamic-config"` + // EnableCollectExecutionInfo enables the TiDB to collect execution info. + EnableCollectExecutionInfo bool `toml:"enable-collect-execution-info" json:"enable-collect-execution-info"` } // UpdateTempStoragePath is to update the `TempStoragePath` if port/statusPort was changed @@ -348,19 +348,22 @@ type Status struct { // Performance is the performance section of the config. type Performance struct { - MaxProcs uint `toml:"max-procs" json:"max-procs"` - MaxMemory uint64 `toml:"max-memory" json:"max-memory"` - StatsLease string `toml:"stats-lease" json:"stats-lease"` - StmtCountLimit uint `toml:"stmt-count-limit" json:"stmt-count-limit"` - FeedbackProbability float64 `toml:"feedback-probability" json:"feedback-probability"` - QueryFeedbackLimit uint `toml:"query-feedback-limit" json:"query-feedback-limit"` - PseudoEstimateRatio float64 `toml:"pseudo-estimate-ratio" json:"pseudo-estimate-ratio"` - ForcePriority string `toml:"force-priority" json:"force-priority"` - BindInfoLease string `toml:"bind-info-lease" json:"bind-info-lease"` - TxnTotalSizeLimit uint64 `toml:"txn-total-size-limit" json:"txn-total-size-limit"` - TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"` - CrossJoin bool `toml:"cross-join" json:"cross-join"` - RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"` + MaxProcs uint `toml:"max-procs" json:"max-procs"` + MaxMemory uint64 `toml:"max-memory" json:"max-memory"` + StatsLease string `toml:"stats-lease" json:"stats-lease"` + StmtCountLimit uint `toml:"stmt-count-limit" json:"stmt-count-limit"` + FeedbackProbability float64 `toml:"feedback-probability" json:"feedback-probability"` + QueryFeedbackLimit uint `toml:"query-feedback-limit" json:"query-feedback-limit"` + PseudoEstimateRatio float64 `toml:"pseudo-estimate-ratio" json:"pseudo-estimate-ratio"` + ForcePriority string `toml:"force-priority" json:"force-priority"` + BindInfoLease string `toml:"bind-info-lease" json:"bind-info-lease"` + TxnTotalSizeLimit uint64 `toml:"txn-total-size-limit" json:"txn-total-size-limit"` + TCPKeepAlive bool `toml:"tcp-keep-alive" json:"tcp-keep-alive"` + CrossJoin bool `toml:"cross-join" json:"cross-join"` + RunAutoAnalyze bool `toml:"run-auto-analyze" json:"run-auto-analyze"` + DistinctAggPushDown bool `toml:"distinct-agg-push-down" json:"agg-push-down-join"` + CommitterConcurrency int `toml:"committer-concurrency" json:"committer-concurrency"` + MaxTxnTTL uint64 `toml:"max-txn-ttl" json:"max-txn-ttl"` } // PlanCache is the PlanCache section of the config. @@ -450,6 +453,8 @@ type TiKVClient struct { // If a store has been up to the limit, it will return error for successive request to // prevent the store occupying too much token in dispatching level. StoreLimit int64 `toml:"store-limit" json:"store-limit"` + // StoreLivenessTimeout is the timeout for store liveness check request. + StoreLivenessTimeout string `toml:"store-liveness-timeout" json:"store-liveness-timeout"` CoprCache CoprocessorCache `toml:"copr-cache" json:"copr-cache"` } @@ -582,18 +587,21 @@ var defaultConf = Config{ RecordQPSbyDB: false, }, Performance: Performance{ - MaxMemory: 0, - TCPKeepAlive: true, - CrossJoin: true, - StatsLease: "3s", - RunAutoAnalyze: true, - StmtCountLimit: 5000, - FeedbackProbability: 0.05, - QueryFeedbackLimit: 1024, - PseudoEstimateRatio: 0.8, - ForcePriority: "NO_PRIORITY", - BindInfoLease: "3s", - TxnTotalSizeLimit: DefTxnTotalSizeLimit, + MaxMemory: 0, + TCPKeepAlive: true, + CrossJoin: true, + StatsLease: "3s", + RunAutoAnalyze: true, + StmtCountLimit: 5000, + FeedbackProbability: 0.05, + QueryFeedbackLimit: 1024, + PseudoEstimateRatio: 0.8, + ForcePriority: "NO_PRIORITY", + BindInfoLease: "3s", + TxnTotalSizeLimit: DefTxnTotalSizeLimit, + DistinctAggPushDown: false, + CommitterConcurrency: 16, + MaxTxnTTL: 10 * 60 * 1000, // 10min }, ProxyProtocol: ProxyProtocol{ Networks: "", @@ -625,22 +633,15 @@ var defaultConf = Config{ EnableChunkRPC: true, - RegionCacheTTL: 600, - StoreLimit: 0, + RegionCacheTTL: 600, + StoreLimit: 0, + StoreLivenessTimeout: DefStoreLivenessTimeout, CoprCache: CoprocessorCache{ - // WARNING: Currently Coprocessor Cache may lead to inconsistent result. Do not open it. - // These config items are hidden from user, so that fill them with zero value instead of default value. - Enabled: false, - CapacityMB: 0, - AdmissionMaxResultMB: 0, - AdmissionMinProcessMs: 0, - - // If you still want to use Coprocessor Cache, here are some recommended configurations: - // Enabled: true, - // CapacityMB: 1000, - // AdmissionMaxResultMB: 10, - // AdmissionMinProcessMs: 5, + Enabled: true, + CapacityMB: 1000, + AdmissionMaxResultMB: 10, + AdmissionMinProcessMs: 5, }, }, Binlog: Binlog{ @@ -666,7 +667,7 @@ var defaultConf = Config{ AllowAutoRandom: false, AllowsExpressionIndex: false, }, - EnableDynamicConfig: false, + EnableCollectExecutionInfo: false, } var ( @@ -832,9 +833,6 @@ func (c *Config) Valid() error { return fmt.Errorf("grpc-connection-count should be greater than 0") } - if c.Performance.TxnTotalSizeLimit > 100<<20 && c.Binlog.Enable { - return fmt.Errorf("txn-total-size-limit should be less than %d with binlog enabled", 100<<20) - } if c.Performance.TxnTotalSizeLimit > 10<<30 { return fmt.Errorf("txn-total-size-limit should be less than %d", 10<<30) } @@ -855,6 +853,9 @@ func (c *Config) Valid() error { if c.PreparedPlanCache.Capacity < 1 { return fmt.Errorf("capacity in [prepared-plan-cache] should be at least 1") } + if c.PreparedPlanCache.MemoryGuardRatio < 0 || c.PreparedPlanCache.MemoryGuardRatio > 1 { + return fmt.Errorf("memory-guard-ratio in [prepared-plan-cache] must be NOT less than 0 and more than 1") + } if len(c.IsolationRead.Engines) < 1 { return fmt.Errorf("the number of [isolation-read]engines for isolation read should be at least 1") } @@ -864,20 +865,6 @@ func (c *Config) Valid() error { } } - // check capacity and the quota when OOMUseTmpStorage is enabled - if c.OOMUseTmpStorage { - if c.TempStorageQuota < 0 { - // means unlimited, do nothing - } else { - capacityByte, err := storage.GetTargetDirectoryCapacity(c.TempStoragePath) - if err != nil { - return err - } else if capacityByte > uint64(c.TempStorageQuota) { - return fmt.Errorf("value of [temp-storage-quota](%d byte) exceeds the capacity(%d byte) of the [%s] directory", c.TempStorageQuota, capacityByte, c.TempStoragePath) - } - } - } - // test log level l := zap.NewAtomicLevel() return l.UnmarshalText([]byte(c.Log.Level)) diff --git a/config/config.toml.example b/config/config.toml.example index 9bead23834dee..52f99090a0749 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -43,9 +43,9 @@ oom-use-tmp-storage = true # tmp-storage-path = "/tmp/tidb/NDAwMC8xMDA4MA==/tmp-storage" # Specifies the maximum use of temporary storage (bytes) for all active queries when `oom-use-tmp-storage` is enabled. -# If the `temp-storage-quota` exceeds the capacity of the temporary storage directory, tidb-server would return an error and exit. -# The default value of temp-storage-quota is under 0 which means tidb-server wouldn't check the capacity. -temp-storage-quota = -1 +# If the `tmp-storage-quota` exceeds the capacity of the temporary storage directory, tidb-server would return an error and exit. +# The default value of tmp-storage-quota is under 0 which means tidb-server wouldn't check the capacity. +tmp-storage-quota = -1 # Specifies what operation TiDB performs when a single SQL statement exceeds the memory quota specified by mem-quota-query and cannot be spilled over to disk. # Valid options: ["log", "cancel"] @@ -232,12 +232,22 @@ force-priority = "NO_PRIORITY" # Bind info lease duration, which influences the duration of loading bind info and handling invalid bind. bind-info-lease = "3s" +# Whether support pushing down aggregation with distinct to cop task +distinct-agg-push-down = false + # The limitation of the size in byte for the entries in one transaction. # If using TiKV as the storage, the entry represents a key/value pair. -# NOTE: If binlog is enabled, this value should be less than 104857600(10M) because this is the maximum size that can be handled by Pumper. -# If binlog is not enabled, this value should be less than 10737418240(10G). +# NOTE: If binlog is enabled with Kafka (e.g. arbiter cluster), +# this value should be less than 1073741824(1G) because this is the maximum size that can be handled by Kafka. +# If binlog is disabled or binlog is enabled without Kafka, this value should be less than 10737418240(10G). txn-total-size-limit = 104857600 +# The max number of running concurrency two phase committer request for an SQL. +committer-concurrency = 16 + +# max lifetime of transaction ttl manager. +max-txn-ttl = 600000 + [proxy-protocol] # PROXY protocol acceptable client networks. # Empty string means disable PROXY protocol, * means all networks. @@ -338,6 +348,22 @@ region-cache-ttl = 600 # default 0 means shutting off store limit. store-limit = 0 +# store-liveness-timeout is used to control timeout for store liveness after sending request failed. +store-liveness-timeout = "120s" + +[tikv-client.copr-cache] +# Whether to enable the copr cache. The copr cache saves the result from TiKV Coprocessor in the memory and +# reuses the result when corresponding data in TiKV is unchanged, on a region basis. +enabled = true + +# The capacity in MB of the cache. +capacity-mb = 1000.0 + +# Only cache requests whose result set is small. +admission-max-result-mb = 10.0 +# Only cache requests takes notable time to process. +admission-min-process-ms = 5 + [binlog] # enable to write binlog. # NOTE: If binlog is enabled, txn-total-size-limit should be less than 104857600(10M). diff --git a/config/config_test.go b/config/config_test.go index 58be42bd0b568..52dc1dd12fca5 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -385,10 +385,6 @@ func (s *testConfigSuite) TestTxnTotalSizeLimitValid(c *C) { conf.Performance.TxnTotalSizeLimit = tt.limit c.Assert(conf.Valid() == nil, Equals, tt.valid) } - - conf.Binlog.Enable = true - conf.Performance.TxnTotalSizeLimit = 100<<20 + 1 - c.Assert(conf.Valid(), NotNil) } func (s *testConfigSuite) TestAllowAutoRandomValid(c *C) { @@ -404,6 +400,21 @@ func (s *testConfigSuite) TestAllowAutoRandomValid(c *C) { checkValid(false, false, true) } +func (s *testConfigSuite) TestPreparePlanCacheValid(c *C) { + conf := NewConfig() + tests := map[PreparedPlanCache]bool{ + {Enabled: true, Capacity: 0}: false, + {Enabled: true, Capacity: 2}: true, + {Enabled: true, MemoryGuardRatio: -0.1}: false, + {Enabled: true, MemoryGuardRatio: 2.2}: false, + {Enabled: true, Capacity: 2, MemoryGuardRatio: 0.5}: true, + } + for testCase, res := range tests { + conf.PreparedPlanCache = testCase + c.Assert(conf.Valid() == nil, Equals, res) + } +} + func (s *testConfigSuite) TestMaxIndexLength(c *C) { conf := NewConfig() checkValid := func(indexLen int, shouldBeValid bool) { diff --git a/config/config_util.go b/config/config_util.go index 737b50446b19f..6922437cf1bcb 100644 --- a/config/config_util.go +++ b/config/config_util.go @@ -132,3 +132,25 @@ func decodeConfig(content string) (*Config, error) { _, err := toml.Decode(content, c) return c, err } + +// FlattenConfigItems flatten this config, see more cases in the test. +func FlattenConfigItems(nestedConfig map[string]interface{}) map[string]interface{} { + flatMap := make(map[string]interface{}) + flatten(flatMap, nestedConfig, "") + return flatMap +} + +func flatten(flatMap map[string]interface{}, nested interface{}, prefix string) { + switch nested.(type) { + case map[string]interface{}: + for k, v := range nested.(map[string]interface{}) { + path := k + if prefix != "" { + path = prefix + "." + k + } + flatten(flatMap, v, path) + } + default: // don't flatten arrays + flatMap[prefix] = nested + } +} diff --git a/config/config_util_test.go b/config/config_util_test.go index 0eb8ec940fe89..7972fcf706000 100644 --- a/config/config_util_test.go +++ b/config/config_util_test.go @@ -14,12 +14,14 @@ package config import ( + "encoding/json" "fmt" "io/ioutil" "os" "path/filepath" "reflect" + "github.com/BurntSushi/toml" . "github.com/pingcap/check" ) @@ -119,3 +121,50 @@ func (s *testConfigSuite) TestAtomicWriteConfig(c *C) { c.Assert(dconf.Performance.MaxProcs, Equals, uint(432)) c.Assert(dconf.Performance.PseudoEstimateRatio, Equals, 54.3) } + +func (s *testConfigSuite) TestFlattenConfig(c *C) { + toJSONStr := func(v interface{}) string { + str, err := json.Marshal(v) + c.Assert(err, IsNil) + return string(str) + } + + jsonConf := `{ + "k0": 233333, + "k1": "v1", + "k2": ["v2-1", "v2-2", "v2-3"], + "k3": [{"k3-1":"v3-1"}, {"k3-2":"v3-2"}, {"k3-3":"v3-3"}], + "k4": { + "k4-1": [1, 2, 3, 4], + "k4-2": [5, 6, 7, 8], + "k4-3": [666] + }}` + nested := make(map[string]interface{}) + c.Assert(json.Unmarshal([]byte(jsonConf), &nested), IsNil) + flatMap := FlattenConfigItems(nested) + c.Assert(len(flatMap), Equals, 7) + c.Assert(toJSONStr(flatMap["k0"]), Equals, "233333") + c.Assert(flatMap["k1"], Equals, "v1") + c.Assert(toJSONStr(flatMap["k2"]), Equals, `["v2-1","v2-2","v2-3"]`) + c.Assert(toJSONStr(flatMap["k3"]), Equals, `[{"k3-1":"v3-1"},{"k3-2":"v3-2"},{"k3-3":"v3-3"}]`) + c.Assert(toJSONStr(flatMap["k4.k4-1"]), Equals, `[1,2,3,4]`) + c.Assert(toJSONStr(flatMap["k4.k4-2"]), Equals, `[5,6,7,8]`) + c.Assert(toJSONStr(flatMap["k4.k4-3"]), Equals, `[666]`) + + tomlConf := ` +port=4000 +[log] +level='info' +format='text' +[isolation-read] +engines = ["tikv", "tiflash", "tidb"] +` + nested = make(map[string]interface{}) + c.Assert(toml.Unmarshal([]byte(tomlConf), &nested), IsNil) + flatMap = FlattenConfigItems(nested) + c.Assert(len(flatMap), Equals, 4) + c.Assert(toJSONStr(flatMap["port"]), Equals, "4000") + c.Assert(toJSONStr(flatMap["log.level"]), Equals, `"info"`) + c.Assert(toJSONStr(flatMap["log.format"]), Equals, `"text"`) + c.Assert(toJSONStr(flatMap["isolation-read.engines"]), Equals, `["tikv","tiflash","tidb"]`) +} diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index 0de3d2c8b3d92..bd4c240cb45fc 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -224,7 +224,7 @@ func (s *testColumnChangeSuite) testColumnDrop(c *C, ctx sessionctx.Context, d * testDropColumn(c, ctx, d, s.dbInfo, tbl.Meta(), dropCol.Name.L, false) } -func (s *testColumnChangeSuite) checkAddWriteOnly(ctx sessionctx.Context, d *ddl, deleteOnlyTable, writeOnlyTable table.Table, h int64) error { +func (s *testColumnChangeSuite) checkAddWriteOnly(ctx sessionctx.Context, d *ddl, deleteOnlyTable, writeOnlyTable table.Table, h kv.Handle) error { // WriteOnlyTable: insert t values (2, 3) err := ctx.NewTxn(context.Background()) if err != nil { @@ -259,7 +259,7 @@ func (s *testColumnChangeSuite) checkAddWriteOnly(ctx sessionctx.Context, d *ddl return errors.Trace(err) } // WriteOnlyTable: update t set c1 = 2 where c1 = 1 - h, _, err = writeOnlyTable.Seek(ctx, 0) + h, _, err = writeOnlyTable.Seek(ctx, kv.IntHandle(0)) if err != nil { return errors.Trace(err) } diff --git a/ddl/column_test.go b/ddl/column_test.go index fd325417e0bfa..dad5eca4b7f95 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -338,7 +339,7 @@ func (s *testColumnSuite) TestColumn(c *C) { testDropTable(c, ctx, d, s.dbInfo, tblInfo) } -func (s *testColumnSuite) checkColumnKVExist(ctx sessionctx.Context, t table.Table, handle int64, col *table.Column, columnValue interface{}, isExist bool) error { +func (s *testColumnSuite) checkColumnKVExist(ctx sessionctx.Context, t table.Table, handle kv.Handle, col *table.Column, columnValue interface{}, isExist bool) error { err := ctx.NewTxn(context.Background()) if err != nil { return errors.Trace(err) @@ -381,7 +382,7 @@ func (s *testColumnSuite) checkColumnKVExist(ctx sessionctx.Context, t table.Tab return nil } -func (s *testColumnSuite) checkNoneColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle int64, col *table.Column, columnValue interface{}) error { +func (s *testColumnSuite) checkNoneColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, col *table.Column, columnValue interface{}) error { t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID) if err != nil { return errors.Trace(err) @@ -397,7 +398,7 @@ func (s *testColumnSuite) checkNoneColumn(ctx sessionctx.Context, d *ddl, tblInf return nil } -func (s *testColumnSuite) checkDeleteOnlyColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle int64, col *table.Column, row []types.Datum, columnValue interface{}) error { +func (s *testColumnSuite) checkDeleteOnlyColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, col *table.Column, row []types.Datum, columnValue interface{}) error { t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID) if err != nil { return errors.Trace(err) @@ -498,7 +499,7 @@ func (s *testColumnSuite) checkDeleteOnlyColumn(ctx sessionctx.Context, d *ddl, return nil } -func (s *testColumnSuite) checkWriteOnlyColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle int64, col *table.Column, row []types.Datum, columnValue interface{}) error { +func (s *testColumnSuite) checkWriteOnlyColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, col *table.Column, row []types.Datum, columnValue interface{}) error { t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID) if err != nil { return errors.Trace(err) @@ -603,7 +604,7 @@ func (s *testColumnSuite) checkWriteOnlyColumn(ctx sessionctx.Context, d *ddl, t return nil } -func (s *testColumnSuite) checkReorganizationColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, handle int64, col *table.Column, row []types.Datum, columnValue interface{}) error { +func (s *testColumnSuite) checkReorganizationColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, col *table.Column, row []types.Datum, columnValue interface{}) error { t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID) if err != nil { return errors.Trace(err) @@ -699,7 +700,7 @@ func (s *testColumnSuite) checkReorganizationColumn(ctx sessionctx.Context, d *d return nil } -func (s *testColumnSuite) checkPublicColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, _ int64, newCol *table.Column, oldRow []types.Datum, columnValue interface{}) error { +func (s *testColumnSuite) checkPublicColumn(ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo, newCol *table.Column, oldRow []types.Datum, columnValue interface{}) error { t, err := testGetTableWithError(d, s.dbInfo.ID, tblInfo.ID) if err != nil { return errors.Trace(err) @@ -793,7 +794,7 @@ func (s *testColumnSuite) checkPublicColumn(ctx sessionctx.Context, d *ddl, tblI return nil } -func (s *testColumnSuite) checkAddColumn(state model.SchemaState, d *ddl, tblInfo *model.TableInfo, handle int64, newCol *table.Column, oldRow []types.Datum, columnValue interface{}) error { +func (s *testColumnSuite) checkAddColumn(state model.SchemaState, d *ddl, tblInfo *model.TableInfo, handle kv.Handle, newCol *table.Column, oldRow []types.Datum, columnValue interface{}) error { ctx := testNewContext(d) var err error switch state { @@ -804,9 +805,9 @@ func (s *testColumnSuite) checkAddColumn(state model.SchemaState, d *ddl, tblInf case model.StateWriteOnly: err = errors.Trace(s.checkWriteOnlyColumn(ctx, d, tblInfo, handle, newCol, oldRow, columnValue)) case model.StateWriteReorganization, model.StateDeleteReorganization: - err = errors.Trace(s.checkReorganizationColumn(ctx, d, tblInfo, handle, newCol, oldRow, columnValue)) + err = errors.Trace(s.checkReorganizationColumn(ctx, d, tblInfo, newCol, oldRow, columnValue)) case model.StatePublic: - err = errors.Trace(s.checkPublicColumn(ctx, d, tblInfo, handle, newCol, oldRow, columnValue)) + err = errors.Trace(s.checkPublicColumn(ctx, d, tblInfo, newCol, oldRow, columnValue)) } return err } @@ -1176,7 +1177,8 @@ func (s *testColumnSuite) colDefStrToFieldType(c *C, str string) *types.FieldTyp stmt, err := parser.New().ParseOneStmt(sqlA, "", "") c.Assert(err, IsNil) colDef := stmt.(*ast.AlterTableStmt).Specs[0].NewColumns[0] - col, _, err := buildColumnAndConstraint(nil, 0, colDef, nil, mysql.DefaultCharset, "", mysql.DefaultCharset, "") + chs, coll := charset.GetDefaultCharsetAndCollate() + col, _, err := buildColumnAndConstraint(nil, 0, colDef, nil, chs, coll) c.Assert(err, IsNil) return &col.FieldType } diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 98c911b69ade4..0928bf75afa85 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -841,7 +841,7 @@ func (s *testStateChangeSuite) TestParallelChangeColumnName(c *C) { oneErr = err2 } } - c.Assert(oneErr.Error(), Equals, "[schema:1060]current error msg: Cancelled DDL job, original error msg: Duplicate column name 'aa'") + c.Assert(oneErr.Error(), Equals, "[schema:1060]Duplicate column name 'aa'") } s.testControlParallelExecSQL(c, sql1, sql2, f) } @@ -851,7 +851,7 @@ func (s *testStateChangeSuite) TestParallelAlterAddIndex(c *C) { sql2 := "CREATE INDEX index_b ON t (c);" f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[ddl:1061]current error msg: Cancelled DDL job, original error msg: index already exist index_b") + c.Assert(err2.Error(), Equals, "[ddl:1061]index already exist index_b") } s.testControlParallelExecSQL(c, sql1, sql2, f) } @@ -862,7 +862,7 @@ func (s *serialTestStateChangeSuite) TestParallelAlterAddExpressionIndex(c *C) { sql2 := "CREATE INDEX expr_index_b ON t ((c+1));" f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[ddl:1061]current error msg: Cancelled DDL job, original error msg: index already exist expr_index_b") + c.Assert(err2.Error(), Equals, "[ddl:1061]index already exist expr_index_b") } s.testControlParallelExecSQL(c, sql1, sql2, f) } @@ -872,7 +872,7 @@ func (s *testStateChangeSuite) TestParallelAddPrimaryKey(c *C) { sql2 := "ALTER TABLE t add primary key index_b(c);" f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[schema:1068]current error msg: Cancelled DDL job, original error msg: Multiple primary key defined") + c.Assert(err2.Error(), Equals, "[schema:1068]Multiple primary key defined") } s.testControlParallelExecSQL(c, sql1, sql2, f) } @@ -883,7 +883,7 @@ func (s *testStateChangeSuite) TestParallelAlterAddPartition(c *C) { );` f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[ddl:1493]current error msg: Cancelled DDL job, original error msg: VALUES LESS THAN value must be strictly increasing for each partition") + c.Assert(err2.Error(), Equals, "[ddl:1493]VALUES LESS THAN value must be strictly increasing for each partition") } s.testControlParallelExecSQL(c, sql1, sql1, f) } @@ -892,7 +892,7 @@ func (s *testStateChangeSuite) TestParallelDropColumn(c *C) { sql := "ALTER TABLE t drop COLUMN c ;" f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[ddl:1091]current error msg: Cancelled DDL job, original error msg: column c doesn't exist") + c.Assert(err2.Error(), Equals, "[ddl:1091]column c doesn't exist") } s.testControlParallelExecSQL(c, sql, sql, f) } @@ -901,7 +901,7 @@ func (s *testStateChangeSuite) TestParallelDropColumns(c *C) { sql := "ALTER TABLE t drop COLUMN b, drop COLUMN c;" f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[ddl:1091]current error msg: Cancelled DDL job, original error msg: column b doesn't exist") + c.Assert(err2.Error(), Equals, "[ddl:1091]column b doesn't exist") } s.testControlParallelExecSQL(c, sql, sql, f) } @@ -920,7 +920,7 @@ func (s *testStateChangeSuite) TestParallelDropIndex(c *C) { sql2 := "alter table t drop index idx2 ;" f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[autoid:1075]current error msg: Cancelled DDL job, original error msg: Incorrect table definition; there can be only one auto column and it must be defined as a key") + c.Assert(err2.Error(), Equals, "[autoid:1075]Incorrect table definition; there can be only one auto column and it must be defined as a key") } s.testControlParallelExecSQL(c, sql1, sql2, f) } @@ -934,7 +934,7 @@ func (s *testStateChangeSuite) TestParallelDropPrimaryKey(c *C) { sql2 := "alter table t drop primary key;" f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[ddl:1091]current error msg: Cancelled DDL job, original error msg: index PRIMARY doesn't exist") + c.Assert(err2.Error(), Equals, "[ddl:1091]index PRIMARY doesn't exist") } s.testControlParallelExecSQL(c, sql1, sql2, f) } @@ -945,7 +945,7 @@ func (s *testStateChangeSuite) TestParallelCreateAndRename(c *C) { defer s.se.Execute(context.Background(), "drop table t_exists") f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[schema:1050]current error msg: Cancelled DDL job, original error msg: Table 't_exists' already exists") + c.Assert(err2.Error(), Equals, "[schema:1050]Table 't_exists' already exists") } s.testControlParallelExecSQL(c, sql1, sql2, f) } @@ -1085,7 +1085,7 @@ func (s *testStateChangeSuite) TestParallelUpdateTableReplica(c *C) { }() wg.Wait() c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[ddl:-1]current error msg: Cancelled DDL job, original error msg: the replica available status of table t1 is already updated") + c.Assert(err2.Error(), Equals, "[ddl:-1]the replica available status of table t1 is already updated") } func (s *testStateChangeSuite) testParallelExecSQL(c *C, sql string) { @@ -1336,7 +1336,7 @@ func (s *serialTestStateChangeSuite) TestParallelFlashbackTable(c *C) { // Disable emulator GC, otherwise, emulator GC will delete table record as soon as possible after executing drop table DDL. ddl.EmulatorGCDisable() gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - time.Duration(48*60*60*time.Second)).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` @@ -1359,8 +1359,13 @@ func (s *serialTestStateChangeSuite) TestParallelFlashbackTable(c *C) { f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) c.Assert(err2, NotNil) - c.Assert(err2.Error(), Equals, "[schema:1050]current error msg: Cancelled DDL job, original error msg: Table 't_flashback' already exists") - + c.Assert(err2.Error(), Equals, "[schema:1050]Table 't_flashback' already exists") } s.testControlParallelExecSQL(c, sql1, sql1, f) + + // Test parallel flashback table with different name + tk.MustExec("drop table t_flashback") + sql1 = "flashback table t_flashback" + sql2 := "flashback table t_flashback to t_flashback2" + s.testControlParallelExecSQL(c, sql1, sql2, f) } diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 14a5622644fa6..c1ad1b9a6ce8f 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -16,6 +16,7 @@ package ddl_test import ( "context" "fmt" + "strconv" "strings" "sync/atomic" "time" @@ -38,6 +39,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -56,13 +58,12 @@ var _ = Suite(&testIntegrationSuite6{&testIntegrationSuite{}}) var _ = SerialSuites(&testIntegrationSuite7{&testIntegrationSuite{}}) type testIntegrationSuite struct { - lease time.Duration - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - dom *domain.Domain - ctx sessionctx.Context - tk *testkit.TestKit + lease time.Duration + cluster cluster.Cluster + store kv.Storage + dom *domain.Domain + ctx sessionctx.Context + tk *testkit.TestKit } func setupIntegrationSuite(s *testIntegrationSuite, c *C) { @@ -70,12 +71,15 @@ func setupIntegrationSuite(s *testIntegrationSuite, c *C) { s.lease = 50 * time.Millisecond ddl.SetWaitTimeWhenErrorOccurred(0) - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) s.store, err = mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) session.SetSchemaLease(s.lease) @@ -409,13 +413,13 @@ func (s *testIntegrationSuite1) TestIssue5092(c *C) { tk.MustExec("alter table t_issue_5092 add column d int default 4 after c1, add column aa int default 0 first") tk.MustQuery("select * from t_issue_5092").Check(testkit.Rows("0 1 2 22 3 33 4")) tk.MustQuery("show create table t_issue_5092").Check(testkit.Rows("t_issue_5092 CREATE TABLE `t_issue_5092` (\n" + - " `aa` int(11) DEFAULT '0',\n" + - " `a` int(11) DEFAULT '1',\n" + - " `b` int(11) DEFAULT '2',\n" + - " `b1` int(11) DEFAULT '22',\n" + - " `c` int(11) DEFAULT '3',\n" + - " `c1` int(11) DEFAULT '33',\n" + - " `d` int(11) DEFAULT '4'\n" + + " `aa` int(11) DEFAULT 0,\n" + + " `a` int(11) DEFAULT 1,\n" + + " `b` int(11) DEFAULT 2,\n" + + " `b1` int(11) DEFAULT 22,\n" + + " `c` int(11) DEFAULT 3,\n" + + " `c1` int(11) DEFAULT 33,\n" + + " `d` int(11) DEFAULT 4\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) tk.MustExec("drop table t_issue_5092") @@ -578,7 +582,7 @@ func (s *testIntegrationSuite5) TestErrnoErrorCode(c *C) { sql = "alter table test_error_code_succ drop index idx_not_exist" tk.MustGetErrCode(sql, errno.ErrCantDropFieldOrKey) sql = "alter table test_error_code_succ drop column c3" - tk.MustGetErrCode(sql, int(errno.ErrUnsupportedDDLOperation)) + tk.MustGetErrCode(sql, errno.ErrUnsupportedDDLOperation) // modify column sql = "alter table test_error_code_succ modify testx.test_error_code_succ.c1 bigint" tk.MustGetErrCode(sql, errno.ErrWrongDBName) @@ -1153,7 +1157,7 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { c.Assert(err, IsNil) // Split the table. - s.cluster.SplitTable(s.mvccStore, tbl.Meta().ID, 100) + s.cluster.SplitTable(tbl.Meta().ID, 100) unique := false indexName := model.NewCIStr("idx_b") @@ -1231,7 +1235,7 @@ func (s *testIntegrationSuite3) TestMultiRegionGetTableEndHandle(c *C) { testCtx := newTestMaxTableRowIDContext(c, d, tbl) // Split the table. - s.cluster.SplitTable(s.mvccStore, tblID, 100) + s.cluster.SplitTable(tblID, 100) maxID, emptyTable := getMaxTableRowID(testCtx, s.store) c.Assert(emptyTable, IsFalse) @@ -2152,3 +2156,107 @@ func (s *testIntegrationSuite7) TestAddExpressionIndexOnPartition(c *C) { tk.MustQuery("select * from t;").Check(testkit.Rows("1 'test' 2", "12 'test' 3", "15 'test' 10", "20 'test' 20")) } + +// TestCreateTableWithAutoIdCache test the auto_id_cache table option. +// `auto_id_cache` take effects on handle too when `PKIshandle` is false, +// or even there is no auto_increment column at all. +func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("USE test;") + tk.MustExec("drop table if exists t;") + tk.MustExec("drop table if exists t1;") + + // Test primary key is handle. + tk.MustExec("create table t(a int auto_increment key) auto_id_cache 100") + tblInfo, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + c.Assert(tblInfo.Meta().AutoIdCache, Equals, int64(100)) + tk.MustExec("insert into t values()") + tk.MustQuery("select * from t").Check(testkit.Rows("1")) + tk.MustExec("delete from t") + + // Invalid the allocator cache, insert will trigger a new cache + tk.MustExec("rename table t to t1;") + tk.MustExec("insert into t1 values()") + tk.MustQuery("select * from t1").Check(testkit.Rows("101")) + + // Test primary key is not handle. + tk.MustExec("drop table if exists t;") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t(a int) auto_id_cache 100") + tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + + tk.MustExec("insert into t values()") + tk.MustQuery("select _tidb_rowid from t").Check(testkit.Rows("1")) + tk.MustExec("delete from t") + + // Invalid the allocator cache, insert will trigger a new cache + tk.MustExec("rename table t to t1;") + tk.MustExec("insert into t1 values()") + tk.MustQuery("select _tidb_rowid from t1").Check(testkit.Rows("101")) + + // Test both auto_increment and rowid exist. + tk.MustExec("drop table if exists t;") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t(a int null, b int auto_increment unique) auto_id_cache 100") + tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + + tk.MustExec("insert into t(b) values(NULL)") + tk.MustQuery("select b, _tidb_rowid from t").Check(testkit.Rows("1 2")) + tk.MustExec("delete from t") + + // Invalid the allocator cache, insert will trigger a new cache. + tk.MustExec("rename table t to t1;") + tk.MustExec("insert into t1(b) values(NULL)") + tk.MustQuery("select b, _tidb_rowid from t1").Check(testkit.Rows("101 102")) + tk.MustExec("delete from t1") + + // Test alter auto_id_cache. + tk.MustExec("alter table t1 auto_id_cache 200") + tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + c.Assert(err, IsNil) + c.Assert(tblInfo.Meta().AutoIdCache, Equals, int64(200)) + + tk.MustExec("insert into t1(b) values(NULL)") + tk.MustQuery("select b, _tidb_rowid from t1").Check(testkit.Rows("201 202")) + tk.MustExec("delete from t1") + + // Invalid the allocator cache, insert will trigger a new cache. + tk.MustExec("rename table t1 to t;") + tk.MustExec("insert into t(b) values(NULL)") + tk.MustQuery("select b, _tidb_rowid from t").Check(testkit.Rows("401 402")) + tk.MustExec("delete from t") + + tk.MustExec("drop table if exists t;") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t(a int auto_increment key) auto_id_cache 3") + tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + c.Assert(tblInfo.Meta().AutoIdCache, Equals, int64(3)) + + // Test insert batch size(4 here) greater than the customized autoid step(3 here). + tk.MustExec("insert into t(a) values(NULL),(NULL),(NULL),(NULL)") + tk.MustQuery("select a from t").Check(testkit.Rows("1", "2", "3", "4")) + tk.MustExec("delete from t") + + // Invalid the allocator cache, insert will trigger a new cache. + tk.MustExec("rename table t to t1;") + tk.MustExec("insert into t1(a) values(NULL)") + next := tk.MustQuery("select a from t1").Rows()[0][0].(string) + nextInt, err := strconv.Atoi(next) + c.Assert(err, IsNil) + c.Assert(nextInt, Greater, 5) + + // Test auto_id_cache overflows int64. + tk.MustExec("drop table if exists t;") + _, err = tk.Exec("create table t(a int) auto_id_cache = 9223372036854775808") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "table option auto_id_cache overflows int64") + + tk.MustExec("create table t(a int) auto_id_cache = 9223372036854775807") + _, err = tk.Exec("alter table t auto_id_cache = 9223372036854775808") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "table option auto_id_cache overflows int64") +} diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index fad386d01c71b..76ccd79a170f0 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -233,13 +233,14 @@ func (s *testIntegrationSuite3) TestCreateTableWithPartition(c *C) { );`) tk.MustExec("set @@tidb_enable_table_partition = 1") - _, err = tk.Exec(`create table t30 ( + tk.MustExec("set @@tidb_enable_table_partition = 1") + tk.MustExec(`create table t30 ( a int, b float, c varchar(30)) partition by range columns (a, b) (partition p0 values less than (10, 10.0))`) - c.Assert(ddl.ErrNotAllowedTypeInPartition.Equal(err), IsTrue) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 8200 Unsupported partition type, treat as normal table")) tk.MustGetErrCode(`create table t31 (a int not null) partition by range( a );`, tmysql.ErrPartitionsMustBeDefined) tk.MustGetErrCode(`create table t32 (a int not null) partition by range columns( a );`, tmysql.ErrPartitionsMustBeDefined) @@ -260,6 +261,23 @@ func (s *testIntegrationSuite3) TestCreateTableWithPartition(c *C) { PARTITION p2 VALUES LESS THAN (2000), PARTITION p3 VALUES LESS THAN (2005) );`, tmysql.ErrBadField) + + // Fix a timezone dependent check bug introduced in https://github.com/pingcap/tidb/pull/10655 + tk.MustExec(`create table t34 (dt timestamp(3)) partition by range (floor(unix_timestamp(dt))) ( + partition p0 values less than (unix_timestamp('2020-04-04 00:00:00')), + partition p1 values less than (unix_timestamp('2020-04-05 00:00:00')));`) + + tk.MustGetErrCode(`create table t34 (dt timestamp(3)) partition by range (unix_timestamp(date(dt))) ( + partition p0 values less than (unix_timestamp('2020-04-04 00:00:00')), + partition p1 values less than (unix_timestamp('2020-04-05 00:00:00')));`, tmysql.ErrWrongExprInPartitionFunc) + + tk.MustGetErrCode(`create table t34 (dt datetime) partition by range (unix_timestamp(dt)) ( + partition p0 values less than (unix_timestamp('2020-04-04 00:00:00')), + partition p1 values less than (unix_timestamp('2020-04-05 00:00:00')));`, tmysql.ErrWrongExprInPartitionFunc) + + // Fix https://github.com/pingcap/tidb/issues/16333 + tk.MustExec(`create table t35 (dt timestamp) partition by range (unix_timestamp(dt)) +(partition p0 values less than (unix_timestamp('2020-04-15 00:00:00')));`) } func (s *testIntegrationSuite2) TestCreateTableWithHashPartition(c *C) { @@ -379,22 +397,43 @@ create table log_message_1 ( "create table t (id text) partition by range columns (id) (partition p0 values less than ('abc'));", ddl.ErrNotAllowedTypeInPartition, }, + // create as normal table, warning. + // { + // "create table t (a int, b varchar(64)) partition by range columns (a, b) (" + + // "partition p0 values less than (1, 'a')," + + // "partition p1 values less than (1, 'a'))", + // ddl.ErrRangeNotIncreasing, + // }, { - "create table t (a int, b varchar(64)) partition by range columns (a, b) (" + - "partition p0 values less than (1, 'a')," + - "partition p1 values less than (1, 'a'))", + "create table t (a int, b varchar(64)) partition by range columns ( b) (" + + "partition p0 values less than ( 'a')," + + "partition p1 values less than ('a'))", ddl.ErrRangeNotIncreasing, }, + // create as normal table, warning. + // { + // "create table t (a int, b varchar(64)) partition by range columns (a, b) (" + + // "partition p0 values less than (1, 'b')," + + // "partition p1 values less than (1, 'a'))", + // ddl.ErrRangeNotIncreasing, + // }, { - "create table t (a int, b varchar(64)) partition by range columns (a, b) (" + - "partition p0 values less than (1, 'b')," + - "partition p1 values less than (1, 'a'))", + "create table t (a int, b varchar(64)) partition by range columns (b) (" + + "partition p0 values less than ('b')," + + "partition p1 values less than ('a'))", ddl.ErrRangeNotIncreasing, }, + // create as normal table, warning. + // { + // "create table t (a int, b varchar(64)) partition by range columns (a, b) (" + + // "partition p0 values less than (1, maxvalue)," + + // "partition p1 values less than (1, 'a'))", + // ddl.ErrRangeNotIncreasing, + // }, { - "create table t (a int, b varchar(64)) partition by range columns (a, b) (" + - "partition p0 values less than (1, maxvalue)," + - "partition p1 values less than (1, 'a'))", + "create table t (a int, b varchar(64)) partition by range columns ( b) (" + + "partition p0 values less than ( maxvalue)," + + "partition p1 values less than ('a'))", ddl.ErrRangeNotIncreasing, }, { @@ -416,6 +455,10 @@ create table log_message_1 ( tk.MustExec("create table t1 (a int, b char(3)) partition by range columns (a, b) (" + "partition p0 values less than (1, 'a')," + "partition p1 values less than (2, maxvalue))") + + tk.MustExec("create table t2 (a int, b char(3)) partition by range columns (b) (" + + "partition p0 values less than ( 'a')," + + "partition p1 values less than (maxvalue))") } func (s *testIntegrationSuite3) TestCreateTableWithKeyPartition(c *C) { @@ -427,6 +470,9 @@ func (s *testIntegrationSuite3) TestCreateTableWithKeyPartition(c *C) { s1 char(32) primary key ) partition by key(s1) partitions 10;`) + + tk.MustExec(`drop table if exists tm2`) + tk.MustExec(`create table tm2 (a char(5), unique key(a(5))) partition by key() partitions 5;`) } func (s *testIntegrationSuite5) TestAlterTableAddPartition(c *C) { @@ -1570,7 +1616,7 @@ func getPartitionTableRecordsNum(c *C, ctx sessionctx.Context, tbl table.Partiti for _, def := range info.Definitions { pid := def.ID partition := tbl.(table.PartitionedTable).GetPartition(pid) - startKey := partition.RecordKey(math.MinInt64) + startKey := partition.RecordKey(kv.IntHandle(math.MinInt64)) c.Assert(ctx.NewTxn(context.Background()), IsNil) err := partition.IterRecords(ctx, startKey, partition.Cols(), func(h int64, data []types.Datum, cols []*table.Column) (bool, error) { diff --git a/ddl/db_test.go b/ddl/db_test.go index 848351ecd48c2..97d2f5f2e8289 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -74,8 +75,7 @@ var _ = SerialSuites(&testSerialDBSuite{&testDBSuite{}}) const defaultBatchSize = 1024 type testDBSuite struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore + cluster cluster.Cluster store kv.Storage dom *domain.Domain schemaName string @@ -88,19 +88,22 @@ type testDBSuite struct { func setUpSuite(s *testDBSuite, c *C) { var err error - s.lease = 100 * time.Millisecond + s.lease = 600 * time.Millisecond session.SetSchemaLease(s.lease) session.DisableStats4Test() s.schemaName = "test_db" s.autoIDStep = autoid.GetStep() ddl.SetWaitTimeWhenErrorOccurred(0) - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) s.store, err = mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) @@ -233,7 +236,7 @@ func (s *testDBSuite5) TestAddPrimaryKeyRollback1(c *C) { hasNullValsInKey := false idxName := "PRIMARY" addIdxSQL := "alter table t1 add primary key c3_index (c3);" - errMsg := "[kv:1062]current error msg: Cancelled DDL job, original error msg: Duplicate entry '' for key 'PRIMARY'" + errMsg := "[kv:1062]Duplicate entry '' for key 'PRIMARY'" testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey) } @@ -242,7 +245,7 @@ func (s *testDBSuite1) TestAddPrimaryKeyRollback2(c *C) { hasNullValsInKey := true idxName := "PRIMARY" addIdxSQL := "alter table t1 add primary key c3_index (c3);" - errMsg := "[ddl:1138]current error msg: Cancelled DDL job, original error msg: Invalid use of NULL value" + errMsg := "[ddl:1138]Invalid use of NULL value" testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey) } @@ -250,7 +253,7 @@ func (s *testDBSuite2) TestAddUniqueIndexRollback(c *C) { hasNullValsInKey := false idxName := "c3_index" addIdxSQL := "create unique index c3_index on t1 (c3)" - errMsg := "[kv:1062]current error msg: Cancelled DDL job, original error msg: Duplicate entry '' for key 'c3_index'" + errMsg := "[kv:1062]Duplicate entry '' for key 'c3_index'" testAddIndexRollback(c, s.store, s.lease, idxName, addIdxSQL, errMsg, hasNullValsInKey) } @@ -278,7 +281,7 @@ func (s *testSerialDBSuite) TestAddExpressionIndexRollback(c *C) { } d.(ddl.DDLForTest).SetHook(hook) - tk.MustGetErrMsg("alter table t1 add index expr_idx ((pow(c1, c2)));", "[ddl:8202]current error msg: Cancelled DDL job, original error msg: Cannot decode index value, because [types:1690]DOUBLE value is out of range in 'pow(160, 160)'") + tk.MustGetErrMsg("alter table t1 add index expr_idx ((pow(c1, c2)));", "[ddl:8202]Cannot decode index value, because [types:1690]DOUBLE value is out of range in 'pow(160, 160)'") c.Assert(checkErr, IsNil) tk.MustQuery("select * from t1;").Check(testkit.Rows("20 20 20", "80 80 80", "160 160 160")) } @@ -1082,7 +1085,7 @@ LOOP: c.Assert(ctx.NewTxn(context.Background()), IsNil) t := testGetTableByName(c, ctx, "test_db", "test_add_index") handles := make(map[int64]struct{}) - startKey := t.RecordKey(math.MinInt64) + startKey := t.RecordKey(kv.IntHandle(math.MinInt64)) err := t.IterRecords(ctx, startKey, t.Cols(), func(h int64, data []types.Datum, cols []*table.Column) (bool, error) { handles[h] = struct{}{} @@ -1122,9 +1125,9 @@ LOOP: } c.Assert(err, IsNil) - _, ok := handles[h] + _, ok := handles[h.IntValue()] c.Assert(ok, IsTrue) - delete(handles, h) + delete(handles, h.IntValue()) } c.Assert(handles, HasLen, 0) tk.MustExec("drop table test_add_index") @@ -1491,7 +1494,7 @@ func checkDelRangeDone(c *C, ctx sessionctx.Context, idx table.Index) { } c.Assert(err, IsNil) - handles[h] = struct{}{} + handles[h.IntValue()] = struct{}{} } return handles } @@ -1887,7 +1890,7 @@ func (s *testDBSuite6) TestDropColumn(c *C) { s.tk.MustExec("create table t1 (a int,b int) partition by hash(a) partitions 4;") _, err := s.tk.Exec("alter table t1 drop column a") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[expression:1054]current error msg: Cancelled DDL job, original error msg: Unknown column 'a' in 'expression'") + c.Assert(err.Error(), Equals, "[expression:1054]Unknown column 'a' in 'expression'") s.tk.MustExec("drop database drop_col_db") } @@ -3292,19 +3295,11 @@ func (s *testDBSuite5) TestModifyColumnRollBack(c *C) { s.dom.DDL().(ddl.DDLForTest).SetHook(hook) done := make(chan error, 1) go backgroundExec(s.store, "alter table t1 change c2 c2 bigint not null;", done) - ticker := time.NewTicker(s.lease / 2) - defer ticker.Stop() -LOOP: - for { - select { - case err := <-done: - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") - break LOOP - case <-ticker.C: - s.mustExec(c, "insert into t1(c2) values (null);") - } - } + + err := <-done + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") + s.mustExec(c, "insert into t1(c2) values (null);") t := s.testGetTable(c, "t1") for _, col := range t.Cols() { @@ -3357,7 +3352,7 @@ func (s *testDBSuite1) TestModifyColumnNullToNotNull(c *C) { _, err := s.tk.Exec("alter table t1 change c2 c2 int not null;") c.Assert(checkErr, IsNil) c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:1138]current error msg: Cancelled DDL job, original error msg: Invalid use of NULL value") + c.Assert(err.Error(), Equals, "[ddl:1138]Invalid use of NULL value") s.tk.MustQuery("select * from t1").Check(testkit.Rows(" ")) // Check insert error when column has PreventNullInsertFlag. @@ -3410,7 +3405,11 @@ func (s *testDBSuite2) TestTransactionOnAddDropColumn(c *C) { originHook := s.dom.DDL().GetHook() defer s.dom.DDL().(ddl.DDLForTest).SetHook(originHook) hook := &ddl.TestDDLCallback{} + var checkErr error hook.OnJobRunBeforeExported = func(job *model.Job) { + if checkErr != nil { + return + } switch job.SchemaState { case model.StateWriteOnly, model.StateWriteReorganization, model.StateDeleteOnly, model.StateDeleteReorganization: default: @@ -3419,7 +3418,10 @@ func (s *testDBSuite2) TestTransactionOnAddDropColumn(c *C) { // do transaction. for _, transaction := range transactions { for _, sql := range transaction { - s.mustExec(c, sql) + if _, checkErr = s.tk.Exec(sql); checkErr != nil { + checkErr = errors.Errorf("err: %s, sql: %s, job schema state: %s", checkErr.Error(), sql, job.SchemaState) + return + } } } } @@ -3429,6 +3431,7 @@ func (s *testDBSuite2) TestTransactionOnAddDropColumn(c *C) { go backgroundExec(s.store, "alter table t1 add column c int not null after a", done) err := <-done c.Assert(err, IsNil) + c.Assert(checkErr, IsNil) s.tk.MustQuery("select a,b from t1 order by a").Check(testkit.Rows("1 1", "1 1", "1 1", "2 2", "2 2", "2 2")) s.mustExec(c, "delete from t1") @@ -3436,6 +3439,7 @@ func (s *testDBSuite2) TestTransactionOnAddDropColumn(c *C) { go backgroundExec(s.store, "alter table t1 drop column c", done) err = <-done c.Assert(err, IsNil) + c.Assert(checkErr, IsNil) s.tk.MustQuery("select a,b from t1 order by a").Check(testkit.Rows("1 1", "1 1", "1 1", "2 2", "2 2", "2 2")) } @@ -3457,7 +3461,11 @@ func (s *testDBSuite3) TestTransactionWithWriteOnlyColumn(c *C) { originHook := s.dom.DDL().GetHook() defer s.dom.DDL().(ddl.DDLForTest).SetHook(originHook) hook := &ddl.TestDDLCallback{} + var checkErr error hook.OnJobRunBeforeExported = func(job *model.Job) { + if checkErr != nil { + return + } switch job.SchemaState { case model.StateWriteOnly: default: @@ -3466,7 +3474,10 @@ func (s *testDBSuite3) TestTransactionWithWriteOnlyColumn(c *C) { // do transaction. for _, transaction := range transactions { for _, sql := range transaction { - s.mustExec(c, sql) + if _, checkErr = s.tk.Exec(sql); checkErr != nil { + checkErr = errors.Errorf("err: %s, sql: %s, job schema state: %s", checkErr.Error(), sql, job.SchemaState) + return + } } } } @@ -3476,6 +3487,7 @@ func (s *testDBSuite3) TestTransactionWithWriteOnlyColumn(c *C) { go backgroundExec(s.store, "alter table t1 add column c int not null", done) err := <-done c.Assert(err, IsNil) + c.Assert(checkErr, IsNil) s.tk.MustQuery("select a from t1").Check(testkit.Rows("2")) s.mustExec(c, "delete from t1") @@ -3483,6 +3495,7 @@ func (s *testDBSuite3) TestTransactionWithWriteOnlyColumn(c *C) { go backgroundExec(s.store, "alter table t1 drop column c", done) err = <-done c.Assert(err, IsNil) + c.Assert(checkErr, IsNil) s.tk.MustQuery("select a from t1").Check(testkit.Rows("2")) } @@ -3517,10 +3530,10 @@ func (s *testDBSuite4) TestAddColumn2(c *C) { ctx := context.Background() err = s.tk.Se.NewTxn(ctx) c.Assert(err, IsNil) - oldRow, err := writeOnlyTable.RowWithCols(s.tk.Se, 1, writeOnlyTable.WritableCols()) + oldRow, err := writeOnlyTable.RowWithCols(s.tk.Se, kv.IntHandle(1), writeOnlyTable.WritableCols()) c.Assert(err, IsNil) c.Assert(len(oldRow), Equals, 3) - err = writeOnlyTable.RemoveRecord(s.tk.Se, 1, oldRow) + err = writeOnlyTable.RemoveRecord(s.tk.Se, kv.IntHandle(1), oldRow) c.Assert(err, IsNil) _, err = writeOnlyTable.AddRecord(s.tk.Se, types.MakeDatums(oldRow[0].GetInt64(), 2, oldRow[2].GetInt64()), table.IsUpdate) c.Assert(err, IsNil) @@ -3656,7 +3669,7 @@ func (s *testDBSuite5) TestAddIndexForGeneratedColumn(c *C) { s.tk.MustExec("insert into t values()") s.tk.MustExec("ALTER TABLE t ADD COLUMN y1 year as (y + 2)") _, err := s.tk.Exec("ALTER TABLE t ADD INDEX idx_y(y1)") - c.Assert(err.Error(), Equals, "[ddl:8202]current error msg: Cancelled DDL job, original error msg: Cannot decode index value, because cannot convert datum from unsigned bigint to type year.") + c.Assert(err.Error(), Equals, "[ddl:8202]Cannot decode index value, because cannot convert datum from unsigned bigint to type year.") t := s.testGetTable(c, "t") for _, idx := range t.Indices() { @@ -4549,7 +4562,7 @@ func (s *testDBSuite2) TestDDLWithInvalidTableInfo(c *C) { // Test drop partition column. _, err = tk.Exec("alter table t drop column a;") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[expression:1054]current error msg: Cancelled DDL job, original error msg: Unknown column 'a' in 'expression'") + c.Assert(err.Error(), Equals, "[expression:1054]Unknown column 'a' in 'expression'") // Test modify column with invalid expression. _, err = tk.Exec("alter table t modify column c int GENERATED ALWAYS AS ((case when (a = 0) then 0when (a > 0) then (b / a) end));") c.Assert(err, NotNil) diff --git a/ddl/ddl.go b/ddl/ddl.go index 16663e289de0f..4d6fcbe477919 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -534,11 +534,6 @@ func (d *ddl) doDDLJob(ctx sessionctx.Context, job *model.Job) error { } if historyJob.Error != nil { - if (historyJob.State == model.JobStateRollbackDone || historyJob.State == model.JobStateCancelled) && !historyJob.Error.Equal(errCancelledDDLJob) { - historyJob.Error = historyJob.Error.Class().Synthesize(historyJob.Error.Code(), - fmt.Sprintf("current error msg: %s, original error msg: %s", - errCancelledDDLJob.ToSQLError().Message, historyJob.Error.ToSQLError().Message)) - } return errors.Trace(historyJob.Error) } // Only for JobStateCancelled job which is adding columns or drop columns. diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 17e1b19fea0f2..f77d70ff26adb 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -61,8 +61,12 @@ const expressionIndexPrefix = "_V$" func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt) error { dbInfo := &model.DBInfo{Name: schema} if charsetInfo != nil { - dbInfo.Charset = charsetInfo.Chs - dbInfo.Collate = charsetInfo.Col + chs, coll, err := ResolveCharsetCollation(ast.CharsetOpt{Chs: charsetInfo.Chs, Col: charsetInfo.Col}) + if err != nil { + return errors.Trace(err) + } + dbInfo.Charset = chs + dbInfo.Collate = coll } else { dbInfo.Charset, dbInfo.Collate = charset.GetDefaultCharsetAndCollate() } @@ -158,11 +162,6 @@ func (d *ddl) AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) ( return nil } - // Check the current TiDB limitations. - if err = checkModifyCharsetAndCollation(toCharset, toCollate, dbInfo.Charset, dbInfo.Collate, false); err != nil { - return errors.Trace(err) - } - // Do the DDL job. job := &model.Job{ SchemaID: dbInfo.ID, @@ -274,8 +273,13 @@ func setColumnFlagWithConstraint(colMap map[string]*table.Column, v *ast.Constra } } -func buildColumnsAndConstraints(ctx sessionctx.Context, colDefs []*ast.ColumnDef, - constraints []*ast.Constraint, tblCharset, tblCollate, dbCharset, dbCollate string) ([]*table.Column, []*ast.Constraint, error) { +func buildColumnsAndConstraints( + ctx sessionctx.Context, + colDefs []*ast.ColumnDef, + constraints []*ast.Constraint, + tblCharset string, + tblCollate string, +) ([]*table.Column, []*ast.Constraint, error) { colMap := map[string]*table.Column{} // outPriKeyConstraint is the primary key constraint out of column definition. such as: create table t1 (id int , age int, primary key(id)); var outPriKeyConstraint *ast.Constraint @@ -287,7 +291,7 @@ func buildColumnsAndConstraints(ctx sessionctx.Context, colDefs []*ast.ColumnDef } cols := make([]*table.Column, 0, len(colDefs)) for i, colDef := range colDefs { - col, cts, err := buildColumnAndConstraint(ctx, i, colDef, outPriKeyConstraint, tblCharset, tblCollate, dbCharset, dbCollate) + col, cts, err := buildColumnAndConstraint(ctx, i, colDef, outPriKeyConstraint, tblCharset, tblCollate) if err != nil { return nil, nil, errors.Trace(err) } @@ -303,36 +307,31 @@ func buildColumnsAndConstraints(ctx sessionctx.Context, colDefs []*ast.ColumnDef return cols, constraints, nil } -// ResolveCharsetCollation will resolve the charset by the order: table charset > database charset > server default charset, -// and it will also resolve the collate by the order: table collate > database collate > server default collate. -func ResolveCharsetCollation(tblCharset, tblCollate, dbCharset, dbCollate string) (string, string, error) { - if len(tblCharset) != 0 { - // tblCollate is not specified by user. - if len(tblCollate) == 0 { - defCollate, err := charset.GetDefaultCollation(tblCharset) +// ResolveCharsetCollation will resolve the charset and collate by the order of parameters: +// * If any given ast.CharsetOpt is not empty, the resolved charset and collate will be returned. +// * If all ast.CharsetOpts are empty, the default charset and collate will be returned. +func ResolveCharsetCollation(charsetOpts ...ast.CharsetOpt) (string, string, error) { + for _, v := range charsetOpts { + if v.Col != "" { + collation, err := collate.GetCollationByName(v.Col) if err != nil { - // return terror is better. - return "", "", ErrUnknownCharacterSet.GenWithStackByArgs(tblCharset) + return "", "", errors.Trace(err) } - return tblCharset, defCollate, nil + if v.Chs != "" && collation.CharsetName != v.Chs { + return "", "", charset.ErrCollationCharsetMismatch.GenWithStackByArgs(v.Col, v.Chs) + } + return collation.CharsetName, v.Col, nil } - return tblCharset, tblCollate, nil - } - - if len(dbCharset) != 0 { - // dbCollate is not specified by user. - if len(dbCollate) == 0 { - defCollate, err := charset.GetDefaultCollation(dbCharset) + if v.Chs != "" { + coll, err := charset.GetDefaultCollation(v.Chs) if err != nil { - return "", "", ErrUnknownCharacterSet.GenWithStackByArgs(dbCharset) + return "", "", errors.Trace(err) } - return dbCharset, defCollate, nil + return v.Chs, coll, err } - return dbCharset, dbCollate, nil } - - charset, collate := charset.GetDefaultCharsetAndCollate() - return charset, collate, nil + chs, coll := charset.GetDefaultCharsetAndCollate() + return chs, coll, nil } func typesNeedCharset(tp byte) bool { @@ -345,65 +344,13 @@ func typesNeedCharset(tp byte) bool { return false } -func setCharsetCollationFlenDecimal(tp *types.FieldType, specifiedCollates []string, tblCharset, tblCollate, dbCharset, dbCollate string) error { - tp.Charset = strings.ToLower(tp.Charset) - tp.Collate = strings.ToLower(tp.Collate) - if len(tp.Charset) == 0 { - if typesNeedCharset(tp.Tp) { - if len(specifiedCollates) == 0 { - // Both the charset and collate are not specified. - var err error - tp.Charset, tp.Collate, err = ResolveCharsetCollation(tblCharset, tblCollate, dbCharset, dbCollate) - if err != nil { - return errors.Trace(err) - } - } else { - // The charset is not specified but the collate is. - // We should derive charset from it's collate specified rather than getting from table and db. - // It is handled like mysql's logic, use derived charset to judge conflict with next collate. - for _, spc := range specifiedCollates { - derivedCollation, err := collate.GetCollationByName(spc) - if err != nil { - return errors.Trace(err) - } - if len(tp.Charset) == 0 { - tp.Charset = derivedCollation.CharsetName - } else if tp.Charset != derivedCollation.CharsetName { - return ErrCollationCharsetMismatch.GenWithStackByArgs(derivedCollation.Name, tp.Charset) - } - tp.Collate = derivedCollation.Name - } - } - } else { - tp.Charset = charset.CharsetBin - tp.Collate = charset.CharsetBin - } +func setCharsetCollationFlenDecimal(tp *types.FieldType, colCharset, colCollate string) error { + if typesNeedCharset(tp.Tp) { + tp.Charset = colCharset + tp.Collate = colCollate } else { - if !charset.ValidCharsetAndCollation(tp.Charset, tp.Collate) { - return errUnsupportedCharset.GenWithStackByArgs(tp.Charset, tp.Collate) - } - if len(tp.Collate) == 0 { - if len(specifiedCollates) == 0 { - // The charset is specified, but the collate is not. - var err error - tp.Collate, err = charset.GetDefaultCollation(tp.Charset) - if err != nil { - return errors.Trace(err) - } - } else { - // Both the charset and collate are specified. - for _, spc := range specifiedCollates { - derivedCollation, err := collate.GetCollationByName(spc) - if err != nil { - return errors.Trace(err) - } - if tp.Charset != derivedCollation.CharsetName { - return ErrCollationCharsetMismatch.GenWithStackByArgs(derivedCollation.Name, tp.Charset) - } - tp.Collate = derivedCollation.Name - } - } - } + tp.Charset = charset.CharsetBin + tp.Collate = charset.CharsetBin } // Use default value for flen or decimal when they are unspecified. @@ -422,13 +369,31 @@ func setCharsetCollationFlenDecimal(tp *types.FieldType, specifiedCollates []str return nil } -// outPriKeyConstraint is the primary key constraint out of column definition. such as: create table t1 (id int , age int, primary key(id)); -func buildColumnAndConstraint(ctx sessionctx.Context, offset int, - colDef *ast.ColumnDef, outPriKeyConstraint *ast.Constraint, tblCharset, tblCollate, dbCharset, dbCollate string) (*table.Column, []*ast.Constraint, error) { - // specifiedCollates refers to collates in colDef.Options, should handle them together. - specifiedCollates := extractCollateFromOption(colDef) +// buildColumnAndConstraint builds table.Column and ast.Constraint from the parameters. +// outPriKeyConstraint is the primary key constraint out of column definition. For example: +// `create table t1 (id int , age int, primary key(id));` +func buildColumnAndConstraint( + ctx sessionctx.Context, + offset int, + colDef *ast.ColumnDef, + outPriKeyConstraint *ast.Constraint, + tblCharset string, + tblCollate string, +) (*table.Column, []*ast.Constraint, error) { + // specifiedCollate refers to the last collate specified in colDef.Options. + chs, coll, err := getCharsetAndCollateInColumnDef(colDef) + if err != nil { + return nil, nil, errors.Trace(err) + } + chs, coll, err = ResolveCharsetCollation( + ast.CharsetOpt{Chs: chs, Col: coll}, + ast.CharsetOpt{Chs: tblCharset, Col: tblCollate}, + ) + if err != nil { + return nil, nil, errors.Trace(err) + } - if err := setCharsetCollationFlenDecimal(colDef.Tp, specifiedCollates, tblCharset, tblCollate, dbCharset, dbCollate); err != nil { + if err := setCharsetCollationFlenDecimal(colDef.Tp, chs, coll); err != nil { return nil, nil, errors.Trace(err) } col, cts, err := columnDefToCol(ctx, offset, colDef, outPriKeyConstraint) @@ -870,7 +835,10 @@ func checkDefaultValue(ctx sessionctx.Context, c *table.Column, hasDefaultValue return nil } - if c.GetDefaultValue() != nil && !c.DefaultIsExpr { + if c.GetDefaultValue() != nil { + if c.DefaultIsExpr { + return nil + } if _, err := table.GetColDefaultValue(ctx, c.ToInfo()); err != nil { return types.ErrInvalidDefault.GenWithStackByArgs(c.Name) } @@ -1192,10 +1160,18 @@ func convertAutoRandomBitsToUnsigned(autoRandomBits int) (uint64, error) { return uint64(autoRandomBits), nil } -func buildTableInfo(ctx sessionctx.Context, tableName model.CIStr, cols []*table.Column, constraints []*ast.Constraint) (tbInfo *model.TableInfo, err error) { +func buildTableInfo( + ctx sessionctx.Context, + tableName model.CIStr, + cols []*table.Column, + constraints []*ast.Constraint, + charset string, + collate string) (tbInfo *model.TableInfo, err error) { tbInfo = &model.TableInfo{ Name: tableName, Version: model.CurrLatestTableInfoVersion, + Charset: charset, + Collate: collate, } for _, v := range cols { v.ID = allocateColumnID(tbInfo) @@ -1337,7 +1313,7 @@ func checkTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo } } - if err = checkRangePartitioningKeysConstraints(ctx, s, tbInfo); err != nil { + if err = checkPartitioningKeysConstraints(ctx, s, tbInfo); err != nil { return errors.Trace(err) } } @@ -1350,7 +1326,11 @@ func checkTableInfoValid(tblInfo *model.TableInfo) error { return err } -func buildTableInfoWithLike(ident ast.Ident, referTblInfo *model.TableInfo) *model.TableInfo { +func buildTableInfoWithLike(ident ast.Ident, referTblInfo *model.TableInfo) (*model.TableInfo, error) { + // Check the referred table is a real table object. + if referTblInfo.IsSequence() || referTblInfo.IsView() { + return nil, ErrWrongObject.GenWithStackByArgs(ident.Schema, referTblInfo.Name, "BASE TABLE") + } tblInfo := *referTblInfo // Check non-public column and adjust column offset. newColumns := referTblInfo.Cols() @@ -1378,7 +1358,7 @@ func buildTableInfoWithLike(ident ast.Ident, referTblInfo *model.TableInfo) *mod copy(pi.Definitions, referTblInfo.Partition.Definitions) tblInfo.Partition = &pi } - return &tblInfo + return &tblInfo, nil } // BuildTableInfoFromAST builds model.TableInfo from a SQL statement. @@ -1408,11 +1388,18 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh colDefs := s.Cols tableCharset, tableCollate, err := getCharsetAndCollateInTableOption(0, s.Options) if err != nil { - return nil, err + return nil, errors.Trace(err) + } + tableCharset, tableCollate, err = ResolveCharsetCollation( + ast.CharsetOpt{Chs: tableCharset, Col: tableCollate}, + ast.CharsetOpt{Chs: dbCharset, Col: dbCollate}, + ) + if err != nil { + return nil, errors.Trace(err) } // The column charset haven't been resolved here. - cols, newConstraints, err := buildColumnsAndConstraints(ctx, colDefs, s.Constraints, tableCharset, tableCollate, dbCharset, dbCollate) + cols, newConstraints, err := buildColumnsAndConstraints(ctx, colDefs, s.Constraints, tableCharset, tableCollate) if err != nil { return nil, errors.Trace(err) } @@ -1422,12 +1409,10 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh } var tbInfo *model.TableInfo - tbInfo, err = buildTableInfo(ctx, s.Table.Name, cols, newConstraints) + tbInfo, err = buildTableInfo(ctx, s.Table.Name, cols, newConstraints, tableCharset, tableCollate) if err != nil { return nil, errors.Trace(err) } - tbInfo.Collate = tableCollate - tbInfo.Charset = tableCharset if err = setTableAutoRandomBits(ctx, tbInfo, colDefs); err != nil { return nil, errors.Trace(err) @@ -1442,10 +1427,6 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh return nil, errors.Trace(err) } - if err = resolveDefaultTableCharsetAndCollation(tbInfo, dbCharset, dbCollate); err != nil { - return nil, errors.Trace(err) - } - return tbInfo, nil } @@ -1497,7 +1478,7 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e // build tableInfo var tbInfo *model.TableInfo if s.ReferTable != nil { - tbInfo = buildTableInfoWithLike(ident, referTbl.Meta()) + tbInfo, err = buildTableInfoWithLike(ident, referTbl.Meta()) } else { tbInfo, err = buildTableInfoWithStmt(ctx, s, schema.Charset, schema.Collate) } @@ -1594,11 +1575,18 @@ func (d *ddl) CreateTableWithInfo( err = nil } } else if actionType == model.ActionCreateTable { - d.preSplitAndScatter(ctx, tbInfo) + d.preSplitAndScatter(ctx, tbInfo, tbInfo.GetPartitionInfo()) if tbInfo.AutoIncID > 1 { // Default tableAutoIncID base is 0. // If the first ID is expected to greater than 1, we need to do rebase. - err = d.handleAutoIncID(tbInfo, schema.ID) + if err = d.handleAutoIncID(tbInfo, schema.ID, autoid.RowIDAllocType); err != nil { + return errors.Trace(err) + } + } + if tbInfo.AutoRandID > 1 { + // Default tableAutoRandID base is 0. + // If the first ID is expected to greater than 1, we need to do rebase. + err = d.handleAutoIncID(tbInfo, schema.ID, autoid.AutoRandomType) } } @@ -1607,7 +1595,8 @@ func (d *ddl) CreateTableWithInfo( } // preSplitAndScatter performs pre-split and scatter of the table's regions. -func (d *ddl) preSplitAndScatter(ctx sessionctx.Context, tbInfo *model.TableInfo) { +// If `pi` is not nil, will only split region for `pi`, this is used when add partition. +func (d *ddl) preSplitAndScatter(ctx sessionctx.Context, tbInfo *model.TableInfo, pi *model.PartitionInfo) { sp, ok := d.store.(kv.SplittableStore) if !ok || atomic.LoadUint32(&EnableSplitTableRegion) == 0 { return @@ -1622,7 +1611,6 @@ func (d *ddl) preSplitAndScatter(ctx sessionctx.Context, tbInfo *model.TableInfo } else { scatterRegion = variable.TiDBOptOn(val) } - pi := tbInfo.GetPartitionInfo() if pi != nil { preSplit = func() { splitPartitionTableRegion(sp, pi, scatterRegion) } } else { @@ -1681,19 +1669,21 @@ func (d *ddl) CreateView(ctx sessionctx.Context, s *ast.CreateViewStmt) (err err }) } - tbInfo, err := buildTableInfo(ctx, s.ViewName.Name, cols, nil) - if err != nil { - return err - } - tbInfo.View = viewInfo - + tblCharset := "" + tblCollate := "" if v, ok := ctx.GetSessionVars().GetSystemVar("character_set_client"); ok { - tbInfo.Charset = v + tblCharset = v } if v, ok := ctx.GetSessionVars().GetSystemVar("collation_connection"); ok { - tbInfo.Collate = v + tblCollate = v } + tbInfo, err := buildTableInfo(ctx, s.ViewName.Name, cols, nil, tblCharset, tblCollate) + if err != nil { + return err + } + tbInfo.View = viewInfo + onExist := OnExistError if s.OrReplace { onExist = OnExistReplace @@ -1886,7 +1876,7 @@ func checkCharsetAndCollation(cs string, co string) error { // handleAutoIncID handles auto_increment option in DDL. It creates a ID counter for the table and initiates the counter to a proper value. // For example if the option sets auto_increment to 10. The counter will be set to 9. So the next allocated ID will be 10. -func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64) error { +func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64, tp autoid.AllocatorType) error { allocs := autoid.NewAllocatorsFromTblInfo(d.store, schemaID, tbInfo) tbInfo.State = model.StatePublic tb, err := table.TableFromMeta(allocs, tbInfo) @@ -1896,33 +1886,32 @@ func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64) error { // The operation of the minus 1 to make sure that the current value doesn't be used, // the next Alloc operation will get this value. // Its behavior is consistent with MySQL. - if err = tb.RebaseAutoID(nil, tbInfo.AutoIncID-1, false); err != nil { - return errors.Trace(err) + if tp == autoid.RowIDAllocType { + if err = tb.RebaseAutoID(nil, tbInfo.AutoIncID-1, false, tp); err != nil { + return errors.Trace(err) + } + } else { + if err = tb.RebaseAutoID(nil, tbInfo.AutoRandID-1, false, tp); err != nil { + return errors.Trace(err) + } } return nil } -func resolveDefaultTableCharsetAndCollation(tbInfo *model.TableInfo, dbCharset, dbCollate string) (err error) { - chr, collate, err := ResolveCharsetCollation(tbInfo.Charset, tbInfo.Collate, dbCharset, dbCollate) - if err != nil { - return errors.Trace(err) - } - if len(tbInfo.Charset) == 0 { - tbInfo.Charset = chr - } - - if len(tbInfo.Collate) == 0 { - tbInfo.Collate = collate - } - return -} - // handleTableOptions updates tableInfo according to table options. func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) error { for _, op := range options { switch op.Tp { case ast.TableOptionAutoIncrement: tbInfo.AutoIncID = int64(op.UintValue) + case ast.TableOptionAutoIdCache: + if op.UintValue > uint64(math.MaxInt64) { + // TODO: Refine this error. + return errors.New("table option auto_id_cache overflows int64") + } + tbInfo.AutoIdCache = int64(op.UintValue) + case ast.TableOptionAutoRandomBase: + tbInfo.AutoRandID = int64(op.UintValue) case ast.TableOptionComment: tbInfo.Comment = op.StrValue case ast.TableOptionCompression: @@ -1938,6 +1927,8 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err tbInfo.MaxShardRowIDBits = tbInfo.ShardRowIDBits case ast.TableOptionPreSplitRegion: tbInfo.PreSplitRegions = op.UintValue + case ast.TableOptionCharset, ast.TableOptionCollate: + // We don't handle charset and collate here since they're handled in `getCharsetAndCollateInTableOption`. } } if tbInfo.PreSplitRegions > tbInfo.ShardRowIDBits { @@ -1953,6 +1944,33 @@ func isIgnorableSpec(tp ast.AlterTableType) bool { return tp == ast.AlterTableLock || tp == ast.AlterTableAlgorithm } +// getCharsetAndCollateInColumnDef will iterate collate in the options, validate it by checking the charset +// of column definition. If there's no collate in the option, the default collate of column's charset will be used. +func getCharsetAndCollateInColumnDef(def *ast.ColumnDef) (chs, coll string, err error) { + chs = def.Tp.Charset + coll = def.Tp.Collate + if chs != "" && coll == "" { + if coll, err = charset.GetDefaultCollation(chs); err != nil { + return "", "", errors.Trace(err) + } + } + for _, opt := range def.Options { + if opt.Tp == ast.ColumnOptionCollate { + info, err := collate.GetCollationByName(opt.StrValue) + if err != nil { + return "", "", errors.Trace(err) + } + if chs == "" { + chs = info.CharsetName + } else if chs != info.CharsetName { + return "", "", ErrCollationCharsetMismatch.GenWithStackByArgs(info.Name, chs) + } + coll = info.Name + } + } + return +} + // getCharsetAndCollateInTableOption will iterate the charset and collate in the options, // and returns the last charset and collate in options. If there is no charset in the options, // the returns charset will be "", the same as collate. @@ -2055,7 +2073,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A } is := d.infoHandle.Get() - if is.TableIsView(ident.Schema, ident.Name) { + if is.TableIsView(ident.Schema, ident.Name) || is.TableIsSequence(ident.Schema, ident.Name) { return ErrWrongObject.GenWithStackByArgs(ident.Schema, ident.Name, "BASE TABLE") } @@ -2149,7 +2167,15 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A } err = d.ShardRowID(ctx, ident, opt.UintValue) case ast.TableOptionAutoIncrement: - err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue)) + err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.RowIDAllocType) + case ast.TableOptionAutoIdCache: + if opt.UintValue > uint64(math.MaxInt64) { + // TODO: Refine this error. + return errors.New("table option auto_id_cache overflows int64") + } + err = d.AlterTableAutoIDCache(ctx, ident, int64(opt.UintValue)) + case ast.TableOptionAutoRandomBase: + err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.AutoRandomType) case ast.TableOptionComment: spec.Comment = opt.StrValue err = d.AlterTableComment(ctx, ident, spec) @@ -2189,12 +2215,12 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A return nil } -func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int64) error { +func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int64, tp autoid.AllocatorType) error { schema, t, err := d.getSchemaAndTableByIdent(ctx, ident) if err != nil { return errors.Trace(err) } - autoIncID, err := t.Allocators(ctx).Get(autoid.RowIDAllocType).NextGlobalAutoID(t.Meta().ID) + autoIncID, err := t.Allocators(ctx).Get(tp).NextGlobalAutoID(t.Meta().ID) if err != nil { return errors.Trace(err) } @@ -2204,11 +2230,15 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 // and TiDB-B finds 100 < 30001 but returns without any handling, // then TiDB-A may still allocate 99 for auto_increment column. This doesn't make sense for the user. newBase = mathutil.MaxInt64(newBase, autoIncID) + actionType := model.ActionRebaseAutoID + if tp == autoid.AutoRandomType { + actionType = model.ActionRebaseAutoRandomBase + } job := &model.Job{ SchemaID: schema.ID, TableID: t.Meta().ID, SchemaName: schema.Name.L, - Type: model.ActionRebaseAutoID, + Type: actionType, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{newBase}, } @@ -2332,10 +2362,23 @@ func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model } } - // Ignore table constraints now, maybe return error later. - // We use length(t.Cols()) as the default offset firstly, we will change the - // column's offset later. - col, _, err = buildColumnAndConstraint(ctx, len(t.Cols()), specNewColumn, nil, t.Meta().Charset, t.Meta().Collate, schema.Charset, schema.Collate) + tableCharset, tableCollate, err := ResolveCharsetCollation( + ast.CharsetOpt{Chs: t.Meta().Charset, Col: t.Meta().Collate}, + ast.CharsetOpt{Chs: schema.Charset, Col: schema.Collate}, + ) + if err != nil { + return nil, errors.Trace(err) + } + // Ignore table constraints now, they will be checked later. + // We use length(t.Cols()) as the default offset firstly, we will change the column's offset later. + col, _, err = buildColumnAndConstraint( + ctx, + len(t.Cols()), + specNewColumn, + nil, + tableCharset, + tableCollate, + ) if err != nil { return nil, errors.Trace(err) } @@ -2511,6 +2554,9 @@ func (d *ddl) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, spec * ctx.GetSessionVars().StmtCtx.AppendNote(err) return nil } + if err == nil { + d.preSplitAndScatter(ctx, meta, partInfo) + } err = d.callHookOnChanged(err) return errors.Trace(err) } @@ -3042,19 +3088,26 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or Version: col.Version, }) + var chs, coll string // TODO: Remove it when all table versions are greater than or equal to TableInfoVersion1. // If newCol's charset is empty and the table's version less than TableInfoVersion1, // we will not modify the charset of the column. This behavior is not compatible with MySQL. if len(newCol.FieldType.Charset) == 0 && t.Meta().Version < model.TableInfoVersion1 { - newCol.FieldType.Charset = col.FieldType.Charset - newCol.FieldType.Collate = col.FieldType.Collate + chs = col.FieldType.Charset + coll = col.FieldType.Collate + } else { + chs, coll, err = getCharsetAndCollateInColumnDef(specNewColumn) + if err != nil { + return nil, errors.Trace(err) + } + chs, coll, err = ResolveCharsetCollation( + ast.CharsetOpt{Chs: chs, Col: coll}, + ast.CharsetOpt{Chs: t.Meta().Charset, Col: t.Meta().Collate}, + ast.CharsetOpt{Chs: schema.Charset, Col: schema.Collate}, + ) } - // specifiedCollates refers to collates in colDef.Option. When setting charset and collate here we - // should take the collate in colDef.Option into consideration rather than handling it separately - specifiedCollates := extractCollateFromOption(specNewColumn) - err = setCharsetCollationFlenDecimal(&newCol.FieldType, specifiedCollates, t.Meta().Charset, t.Meta().Collate, schema.Charset, schema.Collate) - if err != nil { + if err = setCharsetCollationFlenDecimal(&newCol.FieldType, chs, coll); err != nil { return nil, errors.Trace(err) } @@ -3394,6 +3447,27 @@ func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *a return errors.Trace(err) } +// AlterTableAutoIDCache updates the table comment information. +func (d *ddl) AlterTableAutoIDCache(ctx sessionctx.Context, ident ast.Ident, newCache int64) error { + schema, tb, err := d.getSchemaAndTableByIdent(ctx, ident) + if err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: tb.Meta().ID, + SchemaName: schema.Name.L, + Type: model.ActionModifyTableAutoIdCache, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{newCache}, + } + + err = d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + return errors.Trace(err) +} + // AlterTableCharset changes the table charset and collate. func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Ident, toCharset, toCollate string, needsOverwriteCols bool) error { // use the last one. @@ -3540,14 +3614,12 @@ func checkAlterTableCharset(tblInfo *model.TableInfo, dbInfo *model.DBInfo, toCh } } - if len(origCharset) == 0 { - // The table charset may be "", if the table is create in old TiDB version, such as v2.0.8. - // This DDL will update the table charset to default charset. - origCharset, origCollate, err = ResolveCharsetCollation("", "", dbInfo.Charset, dbInfo.Collate) - if err != nil { - return doNothing, err - } - } + // The table charset may be "", if the table is create in old TiDB version, such as v2.0.8. + // This DDL will update the table charset to default charset. + origCharset, origCollate, err = ResolveCharsetCollation( + ast.CharsetOpt{Chs: origCharset, Col: origCollate}, + ast.CharsetOpt{Chs: dbInfo.Charset, Col: dbInfo.Collate}, + ) if err = checkModifyCharsetAndCollation(toCharset, toCollate, origCharset, origCollate, false); err != nil { return doNothing, err @@ -3709,6 +3781,13 @@ func (d *ddl) TruncateTable(ctx sessionctx.Context, ti ast.Ident) error { } return errors.Trace(err) } + oldTblInfo := tb.Meta() + if oldTblInfo.PreSplitRegions > 0 { + if _, tb, err := d.getSchemaAndTableByIdent(ctx, ti); err == nil { + d.preSplitAndScatter(ctx, tb.Meta(), tb.Meta().GetPartitionInfo()) + } + } + if !config.TableLockEnabled() { return nil } @@ -4508,22 +4587,6 @@ type lockTablesArg struct { IsCleanup bool } -// extractCollateFromOption take collates(may multiple) in option into consideration -// when handle charset and collate of a column, rather than handling it separately. -func extractCollateFromOption(def *ast.ColumnDef) []string { - var specifiedCollates []string - for i := 0; i < len(def.Options); i++ { - op := def.Options[i] - if op.Tp == ast.ColumnOptionCollate { - specifiedCollates = append(specifiedCollates, op.StrValue) - def.Options = append(def.Options[:i], def.Options[i+1:]...) - // maintain the correct index - i-- - } - } - return specifiedCollates -} - func (d *ddl) RepairTable(ctx sessionctx.Context, table *ast.TableName, createStmt *ast.CreateTableStmt) error { // Existence of DB and table has been checked in the preprocessor. oldTableInfo, ok := (ctx.Value(domainutil.RepairedTable)).(*model.TableInfo) @@ -4619,7 +4682,7 @@ func (d *ddl) CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStm return err } // TiDB describe the sequence within a tableInfo, as a same-level object of a table and view. - tbInfo, err := buildTableInfo(ctx, ident.Name, nil, nil) + tbInfo, err := buildTableInfo(ctx, ident.Name, nil, nil, "", "") if err != nil { return err } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index a2973efc6f475..2eb17fb6aad52 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -628,13 +628,17 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionTruncateTable: ver, err = onTruncateTable(d, t, job) case model.ActionRebaseAutoID: - ver, err = onRebaseAutoID(d.store, t, job) + ver, err = onRebaseRowIDType(d.store, t, job) + case model.ActionRebaseAutoRandomBase: + ver, err = onRebaseAutoRandomType(d.store, t, job) case model.ActionRenameTable: ver, err = onRenameTable(d, t, job) case model.ActionShardRowID: ver, err = w.onShardRowID(d, t, job) case model.ActionModifyTableComment: ver, err = onModifyTableComment(t, job) + case model.ActionModifyTableAutoIdCache: + ver, err = onModifyTableAutoIDCache(t, job) case model.ActionAddTablePartition: ver, err = onAddTablePartition(d, t, job) case model.ActionModifyTableCharsetAndCollate: diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 19073b56200e9..5a3630173bbbb 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -21,6 +21,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -205,7 +206,7 @@ func (s *testDDLSuite) TestInvalidDDLJob(c *C) { Args: []interface{}{}, } err := d.doDDLJob(ctx, job) - c.Assert(err.Error(), Equals, "[ddl:8204]current error msg: Cancelled DDL job, original error msg: invalid ddl job type: none") + c.Assert(err.Error(), Equals, "[ddl:8204]invalid ddl job type: none") } func (s *testDDLSuite) TestForeignKeyError(c *C) { @@ -653,7 +654,8 @@ func (s *testDDLSuite) TestCancelJob(c *C) { Tp: &types.FieldType{Tp: mysql.TypeLonglong}, Options: []*ast.ColumnOption{}, } - col, _, err := buildColumnAndConstraint(ctx, 2, newColumnDef, nil, mysql.DefaultCharset, "", mysql.DefaultCharset, "") + chs, coll := charset.GetDefaultCharsetAndCollate() + col, _, err := buildColumnAndConstraint(ctx, 2, newColumnDef, nil, chs, coll) c.Assert(err, IsNil) addColumnArgs := []interface{}{col, &ast.ColumnPosition{Tp: ast.ColumnPositionNone}, 0} @@ -885,7 +887,7 @@ func (s *testDDLSuite) TestCancelJob(c *C) { Tp: &types.FieldType{Tp: mysql.TypeLonglong}, Options: []*ast.ColumnOption{}, } - col, _, err := buildColumnAndConstraint(ctx, 0, newColumnDef, nil, mysql.DefaultCharset, "", mysql.DefaultCharset, "") + col, _, err := buildColumnAndConstraint(ctx, 0, newColumnDef, nil, mysql.DefaultCharset, "") c.Assert(err, IsNil) cols[i] = col } diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index abf3640751bfe..14c5af7d60784 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/testkit" @@ -53,25 +54,27 @@ func TestT(t *testing.T) { var _ = Suite(&testFailDBSuite{}) type testFailDBSuite struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - lease time.Duration - store kv.Storage - dom *domain.Domain - se session.Session - p *parser.Parser + cluster cluster.Cluster + lease time.Duration + store kv.Storage + dom *domain.Domain + se session.Session + p *parser.Parser } func (s *testFailDBSuite) SetUpSuite(c *C) { s.lease = 200 * time.Millisecond ddl.SetWaitTimeWhenErrorOccurred(1 * time.Microsecond) var err error - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) s.store, err = mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) session.SetSchemaLease(s.lease) @@ -225,7 +228,7 @@ func (s *testFailDBSuite) TestAddIndexFailed(c *C) { tblID := tbl.Meta().ID // Split the table. - s.cluster.SplitTable(s.mvccStore, tblID, 100) + s.cluster.SplitTable(tblID, 100) tk.MustExec("alter table t add index idx_b(b)") tk.MustExec("admin check index t idx_b") @@ -358,7 +361,7 @@ func (s *testFailDBSuite) TestAddIndexWorkerNum(c *C) { splitCount := 100 // Split table to multi region. - s.cluster.SplitTable(s.mvccStore, tbl.Meta().ID, splitCount) + s.cluster.SplitTable(tbl.Meta().ID, splitCount) err = ddlutil.LoadDDLReorgVars(tk.Se) c.Assert(err, IsNil) diff --git a/ddl/index.go b/ddl/index.go index a6ee6c7d1fd3d..56f47dd5df7d7 100755 --- a/ddl/index.go +++ b/ddl/index.go @@ -820,7 +820,7 @@ func (w *addIndexWorker) getIndexRecord(handle int64, recordKey []byte, rawRecor cols := t.Cols() idxInfo := w.index.Meta() sysZone := timeutil.SystemLocation() - _, err := w.rowDecoder.DecodeAndEvalRowWithMap(w.sessCtx, handle, rawRecord, time.UTC, sysZone, w.rowMap) + _, err := w.rowDecoder.DecodeAndEvalRowWithMap(w.sessCtx, kv.IntHandle(handle), rawRecord, time.UTC, sysZone, w.rowMap) if err != nil { return nil, errors.Trace(errCantDecodeIndex.GenWithStackByArgs(err)) } @@ -972,7 +972,7 @@ func (w *addIndexWorker) batchCheckUniqueKey(txn kv.Transaction, idxRecords []*i w.initBatchCheckBufs(len(idxRecords)) stmtCtx := w.sessCtx.GetSessionVars().StmtCtx for i, record := range idxRecords { - idxKey, distinct, err := w.index.GenIndexKey(stmtCtx, record.vals, record.handle, w.idxKeyBufs[i]) + idxKey, distinct, err := w.index.GenIndexKey(stmtCtx, record.vals, kv.IntHandle(record.handle), w.idxKeyBufs[i]) if err != nil { return errors.Trace(err) } @@ -994,7 +994,7 @@ func (w *addIndexWorker) batchCheckUniqueKey(txn kv.Transaction, idxRecords []*i for i, key := range w.batchCheckKeys { if val, found := batchVals[string(key)]; found { if w.distinctCheckFlags[i] { - handle, err1 := tables.DecodeHandle(val) + handle, err1 := tables.DecodeHandleInUniqueIndexValue(val) if err1 != nil { return errors.Trace(err1) } @@ -1008,7 +1008,7 @@ func (w *addIndexWorker) batchCheckUniqueKey(txn kv.Transaction, idxRecords []*i // The keys in w.batchCheckKeys also maybe duplicate, // so we need to backfill the not found key into `batchVals` map. if w.distinctCheckFlags[i] { - batchVals[string(key)] = tables.EncodeHandle(idxRecords[i].handle) + batchVals[string(key)] = tables.EncodeHandleInUniqueIndexValue(idxRecords[i].handle) } } } @@ -1062,9 +1062,9 @@ func (w *addIndexWorker) backfillIndexInTxn(handleRange reorgIndexTask) (taskCtx } // Create the index. - handle, err := w.index.Create(w.sessCtx, txn, idxRecord.vals, idxRecord.handle) + handle, err := w.index.Create(w.sessCtx, txn, idxRecord.vals, kv.IntHandle(idxRecord.handle)) if err != nil { - if kv.ErrKeyExists.Equal(err) && idxRecord.handle == handle { + if kv.ErrKeyExists.Equal(err) && idxRecord.handle == handle.IntValue() { // Index already exists, skip it. continue } @@ -1190,8 +1190,8 @@ func makeupDecodeColMap(sessCtx sessionctx.Context, t table.Table, indexInfo *mo // to speed up adding index in table with disperse handle. // The `t` should be a non-partitioned table or a partition. func splitTableRanges(t table.PhysicalTable, store kv.Storage, startHandle, endHandle int64) ([]kv.KeyRange, error) { - startRecordKey := t.RecordKey(startHandle) - endRecordKey := t.RecordKey(endHandle).Next() + startRecordKey := t.RecordKey(kv.IntHandle(startHandle)) + endRecordKey := t.RecordKey(kv.IntHandle(endHandle)).Next() logutil.BgLogger().Info("[ddl] split table range from PD", zap.Int64("physicalTableID", t.GetPhysicalID()), zap.Int64("startHandle", startHandle), zap.Int64("endHandle", endHandle)) kvRange := kv.KeyRange{StartKey: startRecordKey, EndKey: endRecordKey} @@ -1223,7 +1223,7 @@ func decodeHandleRange(keyRange kv.KeyRange) (int64, int64, error) { return 0, 0, errors.Trace(err) } - return startHandle, endHandle, nil + return startHandle.IntValue(), endHandle.IntValue(), nil } func closeAddIndexWorkers(workers []*addIndexWorker) { @@ -1309,7 +1309,7 @@ func (w *worker) sendRangeTaskToWorkers(t table.Table, workers []*addIndexWorker return nil, errors.Trace(err) } - endKey := t.RecordKey(endHandle) + endKey := t.RecordKey(kv.IntHandle(endHandle)) endIncluded := false if endKey.Cmp(keyRange.EndKey) < 0 { endIncluded = true @@ -1558,19 +1558,19 @@ func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version if err != nil { return errors.Trace(err) } - firstKey := t.RecordKey(startHandle) + firstKey := t.RecordKey(kv.IntHandle(startHandle)) // Calculate the exclusive upper bound var upperBound kv.Key if endIncluded { if endHandle == math.MaxInt64 { - upperBound = t.RecordKey(endHandle).PrefixNext() + upperBound = t.RecordKey(kv.IntHandle(endHandle)).PrefixNext() } else { // PrefixNext is time costing. Try to avoid it if possible. - upperBound = t.RecordKey(endHandle + 1) + upperBound = t.RecordKey(kv.IntHandle(endHandle + 1)) } } else { - upperBound = t.RecordKey(endHandle) + upperBound = t.RecordKey(kv.IntHandle(endHandle)) } it, err := snap.Iter(firstKey, upperBound) @@ -1584,14 +1584,14 @@ func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version break } - var handle int64 + var handle kv.Handle handle, err = tablecodec.DecodeRowKey(it.Key()) if err != nil { return errors.Trace(err) } rk := t.RecordKey(handle) - more, err := fn(handle, rk, it.Value()) + more, err := fn(handle.IntValue(), rk, it.Value()) if !more || err != nil { return errors.Trace(err) } diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index 14e0ba704e0c3..a9d84301c0434 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -179,7 +179,7 @@ func checkIndexExists(ctx sessionctx.Context, tbl table.Table, indexValue interf if err != nil { return errors.Trace(err) } - doesExist, _, err := idx.Exist(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(indexValue), handle) + doesExist, _, err := idx.Exist(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(indexValue), kv.IntHandle(handle)) if err != nil { return errors.Trace(err) } @@ -218,7 +218,7 @@ func (s *testIndexChangeSuite) checkAddWriteOnly(d *ddl, ctx sessionctx.Context, } // WriteOnlyTable: update t set c2 = 1 where c1 = 4 and c2 = 4 - err = writeOnlyTbl.UpdateRecord(ctx, 4, types.MakeDatums(4, 4), types.MakeDatums(4, 1), touchedSlice(writeOnlyTbl)) + err = writeOnlyTbl.UpdateRecord(ctx, kv.IntHandle(4), types.MakeDatums(4, 4), types.MakeDatums(4, 1), touchedSlice(writeOnlyTbl)) if err != nil { return errors.Trace(err) } @@ -228,7 +228,7 @@ func (s *testIndexChangeSuite) checkAddWriteOnly(d *ddl, ctx sessionctx.Context, } // DeleteOnlyTable: update t set c2 = 3 where c1 = 4 and c2 = 1 - err = delOnlyTbl.UpdateRecord(ctx, 4, types.MakeDatums(4, 1), types.MakeDatums(4, 3), touchedSlice(writeOnlyTbl)) + err = delOnlyTbl.UpdateRecord(ctx, kv.IntHandle(4), types.MakeDatums(4, 1), types.MakeDatums(4, 3), touchedSlice(writeOnlyTbl)) if err != nil { return errors.Trace(err) } @@ -244,7 +244,7 @@ func (s *testIndexChangeSuite) checkAddWriteOnly(d *ddl, ctx sessionctx.Context, } // WriteOnlyTable: delete t where c1 = 4 and c2 = 3 - err = writeOnlyTbl.RemoveRecord(ctx, 4, types.MakeDatums(4, 3)) + err = writeOnlyTbl.RemoveRecord(ctx, kv.IntHandle(4), types.MakeDatums(4, 3)) if err != nil { return errors.Trace(err) } @@ -254,7 +254,7 @@ func (s *testIndexChangeSuite) checkAddWriteOnly(d *ddl, ctx sessionctx.Context, } // DeleteOnlyTable: delete t where c1 = 5 - err = delOnlyTbl.RemoveRecord(ctx, 5, types.MakeDatums(5, 5)) + err = delOnlyTbl.RemoveRecord(ctx, kv.IntHandle(5), types.MakeDatums(5, 5)) if err != nil { return errors.Trace(err) } @@ -290,7 +290,7 @@ func (s *testIndexChangeSuite) checkAddPublic(d *ddl, ctx sessionctx.Context, wr } // WriteOnlyTable: update t set c2 = 5 where c1 = 7 and c2 = 7 - err = writeTbl.UpdateRecord(ctx, 7, types.MakeDatums(7, 7), types.MakeDatums(7, 5), touchedSlice(writeTbl)) + err = writeTbl.UpdateRecord(ctx, kv.IntHandle(7), types.MakeDatums(7, 7), types.MakeDatums(7, 5), touchedSlice(writeTbl)) if err != nil { return errors.Trace(err) } @@ -303,7 +303,7 @@ func (s *testIndexChangeSuite) checkAddPublic(d *ddl, ctx sessionctx.Context, wr return errors.Trace(err) } // WriteOnlyTable: delete t where c1 = 6 - err = writeTbl.RemoveRecord(ctx, 6, types.MakeDatums(6, 6)) + err = writeTbl.RemoveRecord(ctx, kv.IntHandle(6), types.MakeDatums(6, 6)) if err != nil { return errors.Trace(err) } @@ -353,7 +353,7 @@ func (s *testIndexChangeSuite) checkDropWriteOnly(d *ddl, ctx sessionctx.Context } // WriteOnlyTable update t set c2 = 7 where c1 = 8 and c2 = 8 - err = writeTbl.UpdateRecord(ctx, 8, types.MakeDatums(8, 8), types.MakeDatums(8, 7), touchedSlice(writeTbl)) + err = writeTbl.UpdateRecord(ctx, kv.IntHandle(8), types.MakeDatums(8, 8), types.MakeDatums(8, 7), touchedSlice(writeTbl)) if err != nil { return errors.Trace(err) } @@ -364,7 +364,7 @@ func (s *testIndexChangeSuite) checkDropWriteOnly(d *ddl, ctx sessionctx.Context } // WriteOnlyTable delete t where c1 = 8 - err = writeTbl.RemoveRecord(ctx, 8, types.MakeDatums(8, 7)) + err = writeTbl.RemoveRecord(ctx, kv.IntHandle(8), types.MakeDatums(8, 7)) if err != nil { return errors.Trace(err) } @@ -408,7 +408,7 @@ func (s *testIndexChangeSuite) checkDropDeleteOnly(d *ddl, ctx sessionctx.Contex } // DeleteOnlyTable update t set c2 = 10 where c1 = 9 - err = delTbl.UpdateRecord(ctx, 9, types.MakeDatums(9, 9), types.MakeDatums(9, 10), touchedSlice(delTbl)) + err = delTbl.UpdateRecord(ctx, kv.IntHandle(9), types.MakeDatums(9, 9), types.MakeDatums(9, 10), touchedSlice(delTbl)) if err != nil { return errors.Trace(err) } diff --git a/ddl/mock.go b/ddl/mock.go index 78df71db25e29..72cc1f5a8d95f 100644 --- a/ddl/mock.go +++ b/ddl/mock.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/sessionctx" @@ -149,11 +150,12 @@ func (dr *mockDelRange) clear() {} // MockTableInfo mocks a table info by create table stmt ast and a specified table id. func MockTableInfo(ctx sessionctx.Context, stmt *ast.CreateTableStmt, tableID int64) (*model.TableInfo, error) { - cols, newConstraints, err := buildColumnsAndConstraints(ctx, stmt.Cols, stmt.Constraints, "", "", "", "") + chs, coll := charset.GetDefaultCharsetAndCollate() + cols, newConstraints, err := buildColumnsAndConstraints(ctx, stmt.Cols, stmt.Constraints, chs, coll) if err != nil { return nil, errors.Trace(err) } - tbl, err := buildTableInfo(ctx, stmt.Table.Name, cols, newConstraints) + tbl, err := buildTableInfo(ctx, stmt.Table.Name, cols, newConstraints, "", "") if err != nil { return nil, errors.Trace(err) } @@ -164,9 +166,5 @@ func MockTableInfo(ctx sessionctx.Context, stmt *ast.CreateTableStmt, tableID in return nil, errors.Trace(err) } - if err = resolveDefaultTableCharsetAndCollation(tbl, "", ""); err != nil { - return nil, errors.Trace(err) - } - return tbl, nil } diff --git a/ddl/partition.go b/ddl/partition.go index 6e0f0a6d74402..8292c3ccbc59e 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -46,42 +46,32 @@ func buildTablePartitionInfo(ctx sessionctx.Context, s *ast.CreateTableStmt) (*m return nil, nil } - // force-discard the unsupported types, even when @@tidb_enable_table_partition = 'on' - switch s.Partition.Tp { - case model.PartitionTypeKey: - // can't create a warning for KEY partition, it will fail an integration test :/ - return nil, nil - case model.PartitionTypeList, model.PartitionTypeSystemTime: - ctx.GetSessionVars().StmtCtx.AppendWarning(errUnsupportedCreatePartition) + if ctx.GetSessionVars().EnableTablePartition == "off" { + ctx.GetSessionVars().StmtCtx.AppendWarning(errTablePartitionDisabled) return nil, nil } var enable bool - switch ctx.GetSessionVars().EnableTablePartition { - case "on": - enable = true - case "off": - enable = false - default: - // When tidb_enable_table_partition = 'auto', - if s.Partition.Tp == model.PartitionTypeRange { - // Partition by range expression is enabled by default. - if s.Partition.ColumnNames == nil { - enable = true - } - // Partition by range columns and just one column. - if len(s.Partition.ColumnNames) == 1 { - enable = true - } + // When tidb_enable_table_partition is 'on' or 'auto'. + if s.Partition.Tp == model.PartitionTypeRange { + // Partition by range expression is enabled by default. + if s.Partition.ColumnNames == nil { + enable = true } - // Partition by hash is enabled by default. - // Note that linear hash is not enabled. - if s.Partition.Tp == model.PartitionTypeHash { + // Partition by range columns and just one column. + if len(s.Partition.ColumnNames) == 1 { enable = true } } + // Partition by hash is enabled by default. + // Note that linear hash is not enabled. + if s.Partition.Tp == model.PartitionTypeHash { + enable = true + } + if !enable { - ctx.GetSessionVars().StmtCtx.AppendWarning(errTablePartitionDisabled) + ctx.GetSessionVars().StmtCtx.AppendWarning(errUnsupportedCreatePartition) + return nil, nil } pi := &model.PartitionInfo{ @@ -303,32 +293,72 @@ func defaultTimezoneDependent(ctx sessionctx.Context, tblInfo *model.TableInfo, return !v, nil } +func checkPartitionFuncCallValid(ctx sessionctx.Context, tblInfo *model.TableInfo, expr *ast.FuncCallExpr) error { + // We assume the result of any function that has a TIMESTAMP argument to be + // timezone-dependent, since a TIMESTAMP value in both numeric and string + // contexts is interpreted according to the current timezone. + // The only exception is UNIX_TIMESTAMP() which returns the internal + // representation of a TIMESTAMP argument verbatim, and thus does not depend on + // the timezone. + // See https://github.com/mysql/mysql-server/blob/5.7/sql/item_func.h#L445 + if expr.FnName.L != ast.UnixTimestamp { + for _, arg := range expr.Args { + if colName, ok := arg.(*ast.ColumnNameExpr); ok { + col := findColumnByName(colName.Name.Name.L, tblInfo) + if col == nil { + return ErrBadField.GenWithStackByArgs(colName.Name.Name.O, "expression") + } + + if ok && col.FieldType.Tp == mysql.TypeTimestamp { + return errors.Trace(errWrongExprInPartitionFunc) + } + } + } + } + + // check function which allowed in partitioning expressions + // see https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-limitations-functions.html + switch expr.FnName.L { + // Mysql don't allow creating partitions with expressions with non matching + // arguments as a (sub)partitioning function, + // but we want to allow such expressions when opening existing tables for + // easier maintenance. This exception should be deprecated at some point in future so that we always throw an error. + // See https://github.com/mysql/mysql-server/blob/5.7/sql/sql_partition.cc#L1072 + case ast.Day, ast.DayOfMonth, ast.DayOfWeek, ast.DayOfYear, ast.Month, ast.Quarter, ast.ToDays, ast.ToSeconds, + ast.Weekday, ast.Year, ast.YearWeek: + return checkResultOK(hasDateField(ctx, tblInfo, expr)) + case ast.Hour, ast.MicroSecond, ast.Minute, ast.Second, ast.TimeToSec: + return checkResultOK(hasTimeField(ctx, tblInfo, expr)) + case ast.UnixTimestamp: + if len(expr.Args) != 1 { + return errors.Trace(errWrongExprInPartitionFunc) + } + col, err := expression.RewriteSimpleExprWithTableInfo(ctx, tblInfo, expr.Args[0]) + if err != nil { + return errors.Trace(err) + } + if col.GetType().Tp != mysql.TypeTimestamp { + return errors.Trace(errWrongExprInPartitionFunc) + } + return nil + case ast.Abs, ast.Ceiling, ast.DateDiff, ast.Extract, ast.Floor, ast.Mod: + for _, arg := range expr.Args { + if err := checkPartitionExprValid(ctx, tblInfo, arg); err != nil { + return err + } + } + return nil + } + return errors.Trace(ErrPartitionFunctionIsNotAllowed) +} + // checkPartitionExprValid checks partition expression validly. func checkPartitionExprValid(ctx sessionctx.Context, tblInfo *model.TableInfo, expr ast.ExprNode) error { switch v := expr.(type) { case *ast.FuncCastExpr, *ast.CaseExpr, *ast.SubqueryExpr, *ast.WindowFuncExpr, *ast.RowExpr, *ast.DefaultExpr, *ast.ValuesExpr: return errors.Trace(ErrPartitionFunctionIsNotAllowed) case *ast.FuncCallExpr: - // check function which allowed in partitioning expressions - // see https://dev.mysql.com/doc/mysql-partitioning-excerpt/5.7/en/partitioning-limitations-functions.html - switch v.FnName.L { - // Mysql don't allow creating partitions with expressions with non matching - // arguments as a (sub)partitioning function, - // but we want to allow such expressions when opening existing tables for - // easier maintenance. This exception should be deprecated at some point in future so that we always throw an error. - // See https://github.com/mysql/mysql-server/blob/5.7/sql/sql_partition.cc#L1072 - case ast.Day, ast.DayOfMonth, ast.DayOfWeek, ast.DayOfYear, ast.Month, ast.Quarter, ast.ToDays, ast.ToSeconds, - ast.Weekday, ast.Year, ast.YearWeek: - return checkPartitionFunc(hasDateField(ctx, tblInfo, expr)) - case ast.Hour, ast.MicroSecond, ast.Minute, ast.Second, ast.TimeToSec: - return checkPartitionFunc(hasTimeField(ctx, tblInfo, expr)) - case ast.UnixTimestamp: - return checkPartitionFunc(hasTimestampField(ctx, tblInfo, expr)) - case ast.Abs, ast.Ceiling, ast.DateDiff, ast.Extract, ast.Floor, ast.Mod: - return checkPartitionFunc(defaultTimezoneDependent(ctx, tblInfo, expr)) - default: - return errors.Trace(ErrPartitionFunctionIsNotAllowed) - } + return checkPartitionFuncCallValid(ctx, tblInfo, v) case *ast.BinaryOperationExpr: // The DIV operator (opcode.IntDiv) is also supported; the / operator ( opcode.Div ) is not permitted. // see https://dev.mysql.com/doc/refman/5.7/en/partitioning-limitations.html @@ -370,12 +400,12 @@ func checkPartitionFuncValid(ctx sessionctx.Context, tblInfo *model.TableInfo, e // For partition tables, mysql do not support Constant, random or timezone-dependent expressions // Based on mysql code to check whether field is valid, every time related type has check_valid_arguments_processor function. // See https://github.com/mysql/mysql-server/blob/5.7/sql/item_timefunc. -func checkPartitionFunc(isTimezoneDependent bool, err error) error { +func checkResultOK(ok bool, err error) error { if err != nil { return err } - if !isTimezoneDependent { + if !ok { return errors.Trace(errWrongExprInPartitionFunc) } @@ -439,7 +469,7 @@ func checkPartitionFuncType(ctx sessionctx.Context, s *ast.CreateTableStmt, tblI func checkCreatePartitionValue(ctx sessionctx.Context, tblInfo *model.TableInfo) error { pi := tblInfo.Partition defs := pi.Definitions - if len(defs) <= 1 { + if len(defs) == 0 { return nil } @@ -683,8 +713,8 @@ func getPartitionIDs(table *model.TableInfo) []int64 { return physicalTableIDs } -// checkRangePartitioningKeysConstraints checks that the range partitioning key is included in the table constraint. -func checkRangePartitioningKeysConstraints(sctx sessionctx.Context, s *ast.CreateTableStmt, tblInfo *model.TableInfo) error { +// checkPartitioningKeysConstraints checks that the range partitioning key is included in the table constraint. +func checkPartitioningKeysConstraints(sctx sessionctx.Context, s *ast.CreateTableStmt, tblInfo *model.TableInfo) error { // Returns directly if there are no unique keys in the table. if len(tblInfo.Indices) == 0 && !tblInfo.PKIsHandle { return nil @@ -702,6 +732,9 @@ func checkRangePartitioningKeysConstraints(sctx sessionctx.Context, s *ast.Creat partCols = columnInfoSlice(partColumns) } else if len(s.Partition.ColumnNames) > 0 { partCols = columnNameSlice(s.Partition.ColumnNames) + } else { + // TODO: Check keys constraints for list, key partition type and so on. + return nil } // Checks that the partitioning key is included in the constraint. @@ -776,11 +809,10 @@ func (cne *columnNameExtractor) Enter(node ast.Node) (ast.Node, bool) { func (cne *columnNameExtractor) Leave(node ast.Node) (ast.Node, bool) { if c, ok := node.(*ast.ColumnNameExpr); ok { - for _, info := range cne.tblInfo.Columns { - if info.Name.L == c.Name.Name.L { - cne.extractedColumns = append(cne.extractedColumns, info) - return node, true - } + info := findColumnByName(c.Name.Name.L, cne.tblInfo) + if info != nil { + cne.extractedColumns = append(cne.extractedColumns, info) + return node, true } cne.err = ErrBadField.GenWithStackByArgs(c.Name.Name.O, "expression") return nil, false @@ -788,6 +820,15 @@ func (cne *columnNameExtractor) Leave(node ast.Node) (ast.Node, bool) { return node, true } +func findColumnByName(colName string, tblInfo *model.TableInfo) *model.ColumnInfo { + for _, info := range tblInfo.Columns { + if info.Name.L == colName { + return info + } + } + return nil +} + func extractPartitionColumns(partExpr string, tblInfo *model.TableInfo) ([]*model.ColumnInfo, error) { partExpr = "select " + partExpr stmts, _, err := parser.New().Parse(partExpr, "", "") diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 167bc703accb4..f48449f53c2fe 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -14,6 +14,8 @@ package ddl import ( + "fmt" + "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" @@ -347,7 +349,7 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) model.ActionModifyColumn, model.ActionAddForeignKey, model.ActionDropForeignKey, model.ActionRenameTable, model.ActionModifyTableCharsetAndCollate, model.ActionTruncateTablePartition, - model.ActionModifySchemaCharsetAndCollate, model.ActionRepairTable: + model.ActionModifySchemaCharsetAndCollate, model.ActionRepairTable, model.ActionModifyTableAutoIdCache: ver, err = cancelOnlyNotHandledJob(job) default: job.State = model.JobStateCancelled @@ -355,16 +357,23 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) } if err != nil { + if job.Error == nil { + job.Error = toTError(err) + } + if !job.Error.Equal(errCancelledDDLJob) { + job.Error = job.Error.Class().Synthesize(job.Error.Code(), + fmt.Sprintf("DDL job rollback, error msg: %s", job.Error.ToSQLError().Message)) + } + job.ErrorCount++ + if job.State != model.JobStateRollingback && job.State != model.JobStateCancelled { logutil.Logger(w.logCtx).Error("[ddl] run DDL job failed", zap.String("job", job.String()), zap.Error(err)) } else { logutil.Logger(w.logCtx).Info("[ddl] the DDL job is cancelled normally", zap.String("job", job.String()), zap.Error(err)) + // If job is cancelled, we shouldn't return an error. + return ver, nil } - - if job.Error == nil { - job.Error = toTError(err) - } - job.ErrorCount++ } + return } diff --git a/ddl/sequence_test.go b/ddl/sequence_test.go index b52e3e7430c00..49920030c6f0e 100644 --- a/ddl/sequence_test.go +++ b/ddl/sequence_test.go @@ -255,6 +255,11 @@ func (s *testSequenceSuite) TestSequenceAsDefaultValue(c *C) { s.tk.MustExec("create sequence seq") // test the use sequence's nextval as default. + s.tk.MustExec("drop table if exists t") + s.tk.MustExec("create table t(a int not null default next value for seq key)") + s.tk.MustExec("drop table if exists t") + s.tk.MustExec("create table t(a int not null default nextval(seq), b int, primary key(a))") + s.tk.MustExec("create table t1 (a int default next value for seq)") s.tk.MustGetErrMsg("create table t2 (a char(1) default next value for seq)", "[ddl:8228]Unsupported sequence default value for column type 'a'") diff --git a/ddl/serial_test.go b/ddl/serial_test.go index b65cb23cad936..03f44135e3ae8 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/collate" @@ -52,8 +53,9 @@ import ( var _ = SerialSuites(&testSerialSuite{}) type testSerialSuite struct { - store kv.Storage - dom *domain.Domain + store kv.Storage + cluster cluster.Cluster + dom *domain.Domain } func (s *testSerialSuite) SetUpSuite(c *C) { @@ -67,8 +69,18 @@ func (s *testSerialSuite) SetUpSuite(c *C) { config.StoreGlobalConfig(&newCfg) ddl.SetWaitTimeWhenErrorOccurred(1 * time.Microsecond) + + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + var err error - s.store, err = mockstore.NewMockTikvStore() + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) + s.store, err = mockstore.NewMockTikvStore( + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), + ) c.Assert(err, IsNil) s.dom, err = session.BootstrapSession(s.store) @@ -186,10 +198,7 @@ func (s *testSerialSuite) TestMultiRegionGetTableEndHandle(c *C) { testCtx := newTestMaxTableRowIDContext(c, d, tbl) // Split the table. - cluster := mocktikv.NewCluster() - mvccStore := mocktikv.MustNewMVCCStore() - defer mvccStore.Close() - cluster.SplitTable(mvccStore, tblID, 100) + s.cluster.SplitTable(tblID, 100) maxID, emptyTable := getMaxTableRowID(testCtx, s.store) c.Assert(emptyTable, IsFalse) @@ -378,9 +387,19 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) { c.Assert(rows[1][1], Equals, fmt.Sprintf("t_%d_r_2305843009213693952", tbl.Meta().ID)) c.Assert(rows[2][1], Equals, fmt.Sprintf("t_%d_r_4611686018427387904", tbl.Meta().ID)) c.Assert(rows[3][1], Equals, fmt.Sprintf("t_%d_r_6917529027641081856", tbl.Meta().ID)) + // Test after truncate table the region is also splited. + tk.MustExec("truncate table t2") + re = tk.MustQuery("show table t2 regions") + rows = re.Rows() + c.Assert(len(rows), Equals, 4) + tbl = testGetTableByName(c, tk.Se, "test", "t2") + c.Assert(rows[1][1], Equals, fmt.Sprintf("t_%d_r_2305843009213693952", tbl.Meta().ID)) + c.Assert(rows[2][1], Equals, fmt.Sprintf("t_%d_r_4611686018427387904", tbl.Meta().ID)) + c.Assert(rows[3][1], Equals, fmt.Sprintf("t_%d_r_6917529027641081856", tbl.Meta().ID)) + defer atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0) - // for failure cases + // for failure table cases tk.MustExec("use ctwl_db") failSQL := fmt.Sprintf("create table t1 like test_not_exist.t") tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable) @@ -393,6 +412,14 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) { failSQL = fmt.Sprintf("create table t1 like ctwl_db.t") tk.MustGetErrCode(failSQL, mysql.ErrTableExists) + // test failure for wrong object cases + tk.MustExec("drop view if exists v") + tk.MustExec("create view v as select 1 from dual") + tk.MustGetErrCode("create table viewTable like v", mysql.ErrWrongObject) + tk.MustExec("drop sequence if exists seq") + tk.MustExec("create sequence seq") + tk.MustGetErrCode("create table sequenceTable like seq", mysql.ErrWrongObject) + tk.MustExec("drop database ctwl_db") tk.MustExec("drop database ctwl_db1") } @@ -478,8 +505,8 @@ func (s *testSerialSuite) TestRecoverTableByJobID(c *C) { // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. ddl.EmulatorGCDisable() gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - time.Duration(48*60*60*time.Second)).Format(gcTimeFormat) - timeAfterDrop := time.Now().Add(time.Duration(48 * 60 * 60 * time.Second)).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) + timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` @@ -514,7 +541,7 @@ func (s *testSerialSuite) TestRecoverTableByJobID(c *C) { // if GC enable is not exists in mysql.tidb _, err = tk.Exec(fmt.Sprintf("recover table by job %d", jobID)) c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:-1]current error msg: Cancelled DDL job, original error msg: can not get 'tikv_gc_enable'") + c.Assert(err.Error(), Equals, "[ddl:-1]can not get 'tikv_gc_enable'") err = gcutil.EnableGC(tk.Se) c.Assert(err, IsNil) @@ -595,7 +622,7 @@ func (s *testSerialSuite) TestRecoverTableByJobIDFail(c *C) { // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. ddl.EmulatorGCDisable() gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - time.Duration(48*60*60*time.Second)).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` @@ -664,7 +691,7 @@ func (s *testSerialSuite) TestRecoverTableByTableNameFail(c *C) { // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. ddl.EmulatorGCDisable() gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - time.Duration(48*60*60*time.Second)).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` @@ -723,7 +750,7 @@ func (s *testSerialSuite) TestCancelJobByErrorCountLimit(c *C) { _, err = tk.Exec("create table t (a int)") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:-1]current error msg: Cancelled DDL job, original error msg: mock do job error") + c.Assert(err.Error(), Equals, "[ddl:-1]DDL job rollback, error msg: mock do job error") } func (s *testSerialSuite) TestCanceledJobTakeTime(c *C) { diff --git a/ddl/split_region.go b/ddl/split_region.go index 9b88fd250988d..e62a1cff1d57a 100644 --- a/ddl/split_region.go +++ b/ddl/split_region.go @@ -73,10 +73,10 @@ func splitPreSplitedTable(store kv.SplittableStore, tbInfo *model.TableInfo, sca step := int64(1 << (tbInfo.ShardRowIDBits - tbInfo.PreSplitRegions)) max := int64(1 << tbInfo.ShardRowIDBits) splitTableKeys := make([][]byte, 0, 1<<(tbInfo.PreSplitRegions)) - for p := int64(step); p < max; p += step { + for p := step; p < max; p += step { recordID := p << (64 - tbInfo.ShardRowIDBits - 1) recordPrefix := tablecodec.GenTableRecordPrefix(tbInfo.ID) - key := tablecodec.EncodeRecordKey(recordPrefix, recordID) + key := tablecodec.EncodeRecordKey(recordPrefix, kv.IntHandle(recordID)) splitTableKeys = append(splitTableKeys, key) } var err error diff --git a/ddl/table.go b/ddl/table.go index 4e4a118985cb5..2802aeffa4ceb 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -231,6 +231,14 @@ func (w *worker) onRecoverTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in return ver, errors.Trace(err) } + err = checkTableIDNotExists(t, schemaID, tblInfo.ID) + if err != nil { + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableExists.Equal(err) { + job.State = model.JobStateCancelled + } + return ver, errors.Trace(err) + } + // Recover table divide into 2 steps: // 1. Check GC enable status, to decided whether enable GC after recover table. // a. Why not disable GC before put the job to DDL job queue? @@ -430,6 +438,10 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro if err != nil { return ver, errors.Trace(err) } + if tblInfo.IsView() { + job.State = model.JobStateCancelled + return ver, infoschema.ErrTableNotExists.GenWithStackByArgs(job.SchemaName, tblInfo.Name.O) + } err = t.DropTableOrView(schemaID, tblInfo.ID, true) if err != nil { @@ -477,7 +489,15 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro return ver, nil } -func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { +func onRebaseRowIDType(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { + return onRebaseAutoID(store, t, job, autoid.RowIDAllocType) +} + +func onRebaseAutoRandomType(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { + return onRebaseAutoID(store, t, job, autoid.AutoRandomType) +} + +func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job, tp autoid.AllocatorType) (ver int64, _ error) { schemaID := job.SchemaID var newBase int64 err := job.DecodeArgs(&newBase) @@ -491,7 +511,12 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, return ver, errors.Trace(err) } // No need to check `newBase` again, because `RebaseAutoID` will do this check. - tblInfo.AutoIncID = newBase + if tp == autoid.RowIDAllocType { + tblInfo.AutoIncID = newBase + } else { + tblInfo.AutoRandID = newBase + } + tbl, err := getTable(store, schemaID, tblInfo) if err != nil { job.State = model.JobStateCancelled @@ -500,7 +525,7 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, // The operation of the minus 1 to make sure that the current value doesn't be used, // the next Alloc operation will get this value. // Its behavior is consistent with MySQL. - err = tbl.RebaseAutoID(nil, tblInfo.AutoIncID-1, false) + err = tbl.RebaseAutoID(nil, newBase-1, false, tp) if err != nil { return ver, errors.Trace(err) } @@ -512,6 +537,27 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, return ver, nil } +func onModifyTableAutoIDCache(t *meta.Meta, job *model.Job) (int64, error) { + var cache int64 + if err := job.DecodeArgs(&cache); err != nil { + job.State = model.JobStateCancelled + return 0, errors.Trace(err) + } + + tblInfo, err := getTableInfoAndCancelFaultJob(t, job, job.SchemaID) + if err != nil { + return 0, errors.Trace(err) + } + + tblInfo.AutoIdCache = cache + ver, err := updateVersionAndTableInfo(t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + return ver, nil +} + func (w *worker) onShardRowID(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { var shardRowIDBits uint64 err := job.DecodeArgs(&shardRowIDBits) @@ -812,6 +858,20 @@ func checkTableNotExists(d *ddlCtx, t *meta.Meta, schemaID int64, tableName stri return checkTableNotExistsFromStore(t, schemaID, tableName) } +func checkTableIDNotExists(t *meta.Meta, schemaID, tableID int64) error { + tbl, err := t.GetTable(schemaID, tableID) + if err != nil { + if meta.ErrDBNotExists.Equal(err) { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs("") + } + return errors.Trace(err) + } + if tbl != nil { + return infoschema.ErrTableExists.GenWithStackByArgs(tbl.Name) + } + return nil +} + func checkTableNotExistsFromInfoSchema(is infoschema.InfoSchema, schemaID int64, tableName string) error { // Check this table's database. schema, ok := is.SchemaByID(schemaID) diff --git a/ddl/util/syncer_test.go b/ddl/util/syncer_test.go index 3c38ecd9f4a4c..a71ed0a13f79b 100644 --- a/ddl/util/syncer_test.go +++ b/ddl/util/syncer_test.go @@ -116,7 +116,7 @@ func TestSyncerSimple(t *testing.T) { t.Fatalf("get chan events count less than 1") } checkRespKV(t, 1, DDLGlobalSchemaVersion, fmt.Sprintf("%v", currentVer), resp.Events[0].Kv) - case <-time.After(100 * time.Millisecond): + case <-time.After(3 * time.Second): t.Fatalf("get udpate version failed") } }() diff --git a/distsql/distsql.go b/distsql/distsql.go index c9904f8d8fa9e..4ef861b43052e 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -188,7 +188,7 @@ func GetSystemEndian() tipb.Endian { } func init() { - var i int = 0x0100 + i := 0x0100 ptr := unsafe.Pointer(&i) if 0x01 == *(*byte)(ptr) { systemEndian = tipb.Endian_BigEndian diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 872bc831818a5..22a9c6fb082e2 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -68,7 +68,7 @@ func (builder *RequestBuilder) SetIndexRanges(sc *stmtctx.StatementContext, tid, // SetTableHandles sets "KeyRanges" for "kv.Request" by converting table handles // "handles" to "KeyRanges" firstly. -func (builder *RequestBuilder) SetTableHandles(tid int64, handles []int64) *RequestBuilder { +func (builder *RequestBuilder) SetTableHandles(tid int64, handles []kv.Handle) *RequestBuilder { builder.Request.KeyRanges = TableHandlesToKVRanges(tid, handles) return builder } @@ -203,7 +203,7 @@ func TableRangesToKVRanges(tid int64, ranges []*ranger.Range, fb *statistics.Que low := codec.EncodeInt(nil, ran.LowVal[0].GetInt64()) high := codec.EncodeInt(nil, ran.HighVal[0].GetInt64()) if ran.LowExclude { - low = []byte(kv.Key(low).PrefixNext()) + low = kv.Key(low).PrefixNext() } // If this range is split by histogram, then the high val will equal to one bucket's upper bound, // since we need to guarantee each range falls inside the exactly one bucket, `PerfixNext` will make the @@ -213,7 +213,7 @@ func TableRangesToKVRanges(tid int64, ranges []*ranger.Range, fb *statistics.Que feedbackRanges = append(feedbackRanges, r) if !ran.HighExclude { - high = []byte(kv.Key(high).PrefixNext()) + high = kv.Key(high).PrefixNext() } startKey := tablecodec.EncodeRowKey(tid, low) endKey := tablecodec.EncodeRowKey(tid, high) @@ -238,29 +238,29 @@ func encodeHandleKey(ran *ranger.Range) ([]byte, []byte) { low := codec.EncodeInt(nil, ran.LowVal[0].GetInt64()) high := codec.EncodeInt(nil, ran.HighVal[0].GetInt64()) if ran.LowExclude { - low = []byte(kv.Key(low).PrefixNext()) + low = kv.Key(low).PrefixNext() } if !ran.HighExclude { - high = []byte(kv.Key(high).PrefixNext()) + high = kv.Key(high).PrefixNext() } return low, high } // TableHandlesToKVRanges converts sorted handle to kv ranges. // For continuous handles, we should merge them to a single key range. -func TableHandlesToKVRanges(tid int64, handles []int64) []kv.KeyRange { +func TableHandlesToKVRanges(tid int64, handles []kv.Handle) []kv.KeyRange { krs := make([]kv.KeyRange, 0, len(handles)) i := 0 for i < len(handles) { j := i + 1 - for ; j < len(handles) && handles[j-1] != math.MaxInt64; j++ { - if handles[j] != handles[j-1]+1 { + for ; j < len(handles) && handles[j-1].IntValue() != math.MaxInt64; j++ { + if handles[j].IntValue() != handles[j-1].IntValue()+1 { break } } - low := codec.EncodeInt(nil, handles[i]) - high := codec.EncodeInt(nil, handles[j-1]) - high = []byte(kv.Key(high).PrefixNext()) + low := codec.EncodeInt(nil, handles[i].IntValue()) + high := codec.EncodeInt(nil, handles[j-1].IntValue()) + high = kv.Key(high).PrefixNext() startKey := tablecodec.EncodeRowKey(tid, low) endKey := tablecodec.EncodeRowKey(tid, high) krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey}) @@ -329,7 +329,7 @@ func encodeIndexKey(sc *stmtctx.StatementContext, ran *ranger.Range) ([]byte, [] return nil, nil, err } if ran.LowExclude { - low = []byte(kv.Key(low).PrefixNext()) + low = kv.Key(low).PrefixNext() } high, err := codec.EncodeKey(sc, nil, ran.HighVal...) if err != nil { @@ -337,7 +337,7 @@ func encodeIndexKey(sc *stmtctx.StatementContext, ran *ranger.Range) ([]byte, [] } if !ran.HighExclude { - high = []byte(kv.Key(high).PrefixNext()) + high = kv.Key(high).PrefixNext() } var hasNull bool @@ -350,7 +350,7 @@ func encodeIndexKey(sc *stmtctx.StatementContext, ran *ranger.Range) ([]byte, [] if hasNull { // Append 0 to make unique-key range [null, null] to be a scan rather than point-get. - high = []byte(kv.Key(high).Next()) + high = kv.Key(high).Next() } return low, high, nil } diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index ac8d4aabff447..f13949b2712e8 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -102,7 +102,7 @@ func (s *testSuite) getExpectedRanges(tid int64, hrs []*handleRange) []kv.KeyRan for _, hr := range hrs { low := codec.EncodeInt(nil, hr.start) high := codec.EncodeInt(nil, hr.end) - high = []byte(kv.Key(high).PrefixNext()) + high = kv.Key(high).PrefixNext() startKey := tablecodec.EncodeRowKey(tid, low) endKey := tablecodec.EncodeRowKey(tid, high) krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey}) @@ -111,7 +111,8 @@ func (s *testSuite) getExpectedRanges(tid int64, hrs []*handleRange) []kv.KeyRan } func (s *testSuite) TestTableHandlesToKVRanges(c *C) { - handles := []int64{0, 2, 3, 4, 5, 10, 11, 100, 9223372036854775806, 9223372036854775807} + handles := []kv.Handle{kv.IntHandle(0), kv.IntHandle(2), kv.IntHandle(3), kv.IntHandle(4), kv.IntHandle(5), + kv.IntHandle(10), kv.IntHandle(11), kv.IntHandle(100), kv.IntHandle(9223372036854775806), kv.IntHandle(9223372036854775807)} // Build expected key ranges. hrs := make([]*handleRange, 0, len(handles)) @@ -402,7 +403,8 @@ func (s *testSuite) TestRequestBuilder2(c *C) { } func (s *testSuite) TestRequestBuilder3(c *C) { - handles := []int64{0, 2, 3, 4, 5, 10, 11, 100} + handles := []kv.Handle{kv.IntHandle(0), kv.IntHandle(2), kv.IntHandle(3), kv.IntHandle(4), + kv.IntHandle(5), kv.IntHandle(10), kv.IntHandle(11), kv.IntHandle(100)} actual, err := (&RequestBuilder{}).SetTableHandles(15, handles). SetDAGRequest(&tipb.DAGRequest{}). diff --git a/distsql/select_result.go b/distsql/select_result.go index 6c1e72e2ff594..d37a6e5cb3a81 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -107,7 +107,7 @@ func (r *selectResult) fetchResp(ctx context.Context) error { // final round of fetch // TODO: Add a label to distinguish between success or failure. // https://github.com/pingcap/tidb/issues/11397 - metrics.DistSQLQueryHistgram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds()) + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds()) r.durationReported = true } return nil @@ -130,10 +130,14 @@ func (r *selectResult) fetchResp(ctx context.Context) error { for _, warning := range r.selectResp.Warnings { sc.AppendWarning(terror.ClassTiKV.Synthesize(terror.ErrCode(warning.Code), warning.Msg)) } - r.updateCopRuntimeStats(resultSubset.GetExecDetails(), resultSubset.RespTime()) + resultDetail := resultSubset.GetExecDetails() + r.updateCopRuntimeStats(resultDetail, resultSubset.RespTime()) r.feedback.Update(resultSubset.GetStartKey(), r.selectResp.OutputCounts) r.partialCount++ - sc.MergeExecDetails(resultSubset.GetExecDetails(), nil) + if resultDetail != nil { + resultDetail.CopTime = duration + } + sc.MergeExecDetails(resultDetail, nil) if len(r.selectResp.Chunks) != 0 { break } diff --git a/distsql/stream.go b/distsql/stream.go index 6a100c2d8cfe8..934eb28e16885 100644 --- a/distsql/stream.go +++ b/distsql/stream.go @@ -43,6 +43,9 @@ type streamResult struct { curr *tipb.Chunk partialCount int64 feedback *statistics.QueryFeedback + + fetchDuration time.Duration + durationReported bool } func (r *streamResult) Fetch(context.Context) {} @@ -70,13 +73,18 @@ func (r *streamResult) Next(ctx context.Context, chk *chunk.Chunk) error { func (r *streamResult) readDataFromResponse(ctx context.Context, resp kv.Response, result *tipb.Chunk) (bool, error) { startTime := time.Now() resultSubset, err := resp.Next(ctx) - // TODO: Add a label to distinguish between success or failure. - // https://github.com/pingcap/tidb/issues/11397 - metrics.DistSQLQueryHistgram.WithLabelValues(r.label, r.sqlType).Observe(time.Since(startTime).Seconds()) + duration := time.Since(startTime) + r.fetchDuration += duration if err != nil { return false, err } if resultSubset == nil { + if !r.durationReported { + // TODO: Add a label to distinguish between success or failure. + // https://github.com/pingcap/tidb/issues/11397 + metrics.DistSQLQueryHistogram.WithLabelValues(r.label, r.sqlType).Observe(r.fetchDuration.Seconds()) + r.durationReported = true + } return true, nil } @@ -98,7 +106,11 @@ func (r *streamResult) readDataFromResponse(ctx context.Context, resp kv.Respons } r.feedback.Update(resultSubset.GetStartKey(), stream.OutputCounts) r.partialCount++ - r.ctx.GetSessionVars().StmtCtx.MergeExecDetails(resultSubset.GetExecDetails(), nil) + resultDetail := resultSubset.GetExecDetails() + if resultDetail != nil { + resultDetail.CopTime = duration + } + r.ctx.GetSessionVars().StmtCtx.MergeExecDetails(resultDetail, nil) return false, nil } diff --git a/docs/design/2019-11-14-tidb-builtin-diagnostics-zh_CN.md b/docs/design/2019-11-14-tidb-builtin-diagnostics-zh_CN.md index 0d235da7f9496..e93d810a6cb1e 100644 --- a/docs/design/2019-11-14-tidb-builtin-diagnostics-zh_CN.md +++ b/docs/design/2019-11-14-tidb-builtin-diagnostics-zh_CN.md @@ -289,7 +289,7 @@ message ServerInfoResponse { - 终端用户:用户直接通过 SQL 查询获取集群信息排查问题 - 运维系统:TiDB 的使用环境比较多样,客户可以通过 SQL 获取集群信息将 TiDB 集成到自己的运维系统中 -- 生态工具:外部工具通过 SQL 拿到集群信息实现功能定制,比如 `[sqltop](https://github.com/ngaut/sqltop)` 可以直接通过集群 `events_statements_summary_by_digest` 获取整个集群的 SQL 采样信息 +- 生态工具:外部工具通过 SQL 拿到集群信息实现功能定制,比如 `[sqltop](https://github.com/ngaut/sqltop)` 可以直接通过集群 `statements_summary` 获取整个集群的 SQL 采样信息 #### 集群拓扑系统表 @@ -488,7 +488,7 @@ mysql> select address, type, value from pd_client_cmd_ops where start_time='2019 #### 内存表全局化 -目前 `slow_query`/`events_statements_summary_by_digest`/`processlist` 只包含单节点数据,本提案通过添加以下三张集群级别系统表使任何一个 TiDB 实例可以查看整个集群的信息: +目前 `slow_query`/`statements_summary`/`processlist` 只包含单节点数据,本提案通过添加以下三张集群级别系统表使任何一个 TiDB 实例可以查看整个集群的信息: | 表名 | 描述 | |------|-----| diff --git a/docs/design/2019-11-14-tidb-builtin-diagnostics.md b/docs/design/2019-11-14-tidb-builtin-diagnostics.md index 4223c17f08a09..d421c625a03b7 100644 --- a/docs/design/2019-11-14-tidb-builtin-diagnostics.md +++ b/docs/design/2019-11-14-tidb-builtin-diagnostics.md @@ -294,7 +294,7 @@ Each TiDB instance can access the information of other nodes through the HTTP AP - End User: Users can obtain cluster information directly through SQL query to troubleshooting problem - Operation and maintenance system: The ability to obtain cluster information through SQL will make it easier for users to integrate TiDB into their own operation and maintenance systems. -- Eco-system tools: External tools get the cluster information through SQL to realize function customization. For example, `[sqltop](https://github.com/ngaut/sqltop)` can directly obtain the SQL sampling information of the entire cluster through the `events_statements_summary_by_digest` table of the cluster. +- Eco-system tools: External tools get the cluster information through SQL to realize function customization. For example, `[sqltop](https://github.com/ngaut/sqltop)` can directly obtain the SQL sampling information of the entire cluster through the `statements_summary` table of the cluster. #### Cluster Topology System Table @@ -480,7 +480,7 @@ This proposal needs to implement the following performance profiling table: #### Globalized memory system table -Current the `slow_query`/`events_statements_summary_by_digest`/`processlist` memory tables only contain single-node data. This proposal allows any TiDB instance to view information about the entire cluster by adding the following three cluster-level system tables: +Current the `slow_query`/`statements_summary`/`processlist` memory tables only contain single-node data. This proposal allows any TiDB instance to view information about the entire cluster by adding the following three cluster-level system tables: | Table Name | Description | |------|-----| diff --git a/docs/design/2020-04-17-sql-sequence.md b/docs/design/2020-04-17-sql-sequence.md new file mode 100644 index 0000000000000..61baddb128b81 --- /dev/null +++ b/docs/design/2020-04-17-sql-sequence.md @@ -0,0 +1,115 @@ +# TiDB SEQUENCE GENERATOR + +- Author(s): [AilinKid](https://github.com/ailinkid) (Lingxiang Tai) +- Last updated: 2020-04-17 +- Discussion at: https://tidbcommunity.slack.com/archives/CQNKBDVM0 + +## Summary + +"Sequence Generator" (hereinafter referred to as "Sequence") is a database feature introduced in SQL 2003 Standard. It is defined as "a mechanism for generating successive exact numeric values, one at a time, which can be either internal or external objects". + +Based on the definition above, the conventional auto-increment columns of MySQL/TiDB can be considered as an internal implementation of Sequence Generator. This RFC describes the design of external Sequence Generator. Unless otherwise noted, the "Sequence" described below refers to external Sequence Generator. + +## Motivation + +- Standardization: Admittedly sequence is a well-defined syntax standard in SQL 2003. Although auto-increment can be regarded as an internal sequence implementation, it hasn't supported all the external features (such as CYCLE). + +- Usability: A sequence can be independent of tables, shared by multiple tables and be specified with CACHE details (either specify the number of CACHE or disable CACHE), which improves usability of the product. + +- Business Value: Sequence is a feature widely implemented in mainstream commercial databases. Users tend to expect TiDB can syntactically support sequence to reduce migration costs who migrate their business from other databases to TiDB. + +## Background + +The columns in TiDB with 'auto-increment' attribute will cache batch values by default in each instance to ensure better performance, which means auto-increment only guarantees the uniqueness under a multi-tidb-node cluster. This kind of cache mode is uncontrollable and implicit in auto-increment. + +But in sequence, we take it open. Users can specify whether to use cache mode or not, what the cache size should be, what's the min and max of the range like and so on. This factors make sequence more flexible, controllable and easy to use. + +## Syntax and Semantics + +Create sequence statement: + +``` +CREATE SEQUENCE [IF NOT EXISTS] sequence_name +[ INCREMENT [ BY | = ] increment ] +[ MINVALUE [=] minvalue | NO MINVALUE | NOMINVALUE ] +[ MAXVALUE [=] maxvalue | NO MAXVALUE | NOMAXVALUE ] +[ START [ WITH | = ] start ] +[ CACHE [=] cache | NOCACHE | NO CACHE ] +[ CYCLE | NOCYCLE | NO CYCLE] +``` + +Show create sequence statement: + +``` +SHOW CREATE SEQUENCE sequence_name +``` + +Drop sequence statement: + +``` +DROP SEQUENCE [IF EXISTS] sequence_name_list +``` + +Get the next value of sequence: + +``` +NEXT VALUE FOR sequence_name / NEXTVAL(sequence_name) +``` + +Get the previous value of sequence: + +``` +LASTVAL(sequence_name) +``` + +Move the current position in sequence number axis: + +``` +SETVAL(sequence_name, num) +``` + +## Rationale + +### Sequence metadata + +Sequence is a table-level object, sharing the same namespace with the base table and view objects, whose meta are recorded in `*model.tableInfo`. But actually sequence definition do not have columns in its meta, which means the sequence value cannot be operated by `DML` interface. The only way to do it is by `setval` function, after all, the sequence value is essentially a KV pair with some constraints. + +### Sequence values + +Each sequence value will be stored in its own range, with the key format below: + +``` +DB: SID: ——> int64 +``` + +Reads and writes to the sequence value (via the functions `nextval`, `setval`), will be implemented by direct calls to the `KV` layer's `Get`, `Inc`, and `Set` functions. Since sequence objects store the value at the range of its own, it will cause hotspot at sequence meta under insertion intensive cases. + +### Sequence Transaction + +Storing sequence values in their own ranges means that insertions to tables which use sequences will always touch two ranges. However, since the sequence update takes place outside of the SQL transaction, this should not trigger the `2PC` commit protocol. + +Sequence computation is under the expression framework, evaluating one value at a time. The values fetched by these functions won't be rolled back when the outer SQL transaction is aborted. + +### Sequence Binlog + +Distinguished from auto-increment's rebase functionality, columns with sequence as default won't rebase it's value automatically when the inserted value was specified explicitly. Besides, sequence can also be used directly as the inserted expression value. Thus, it's necessary to sync the sequence value down to the upstream cluster. + +Sequence binlog is triggered by both `nextval` and `setval` functions when the update reaches the storage directly rather than in the cache. + +Sequence binlog is not a sort of `DML` binlog, but mocked as a kind of `DDL` `job` by specifying the job id equal to -1 specially. Therefore, the drainer won't take it for granted that it's a real ddl job and appears in ddl history queue. + +### Sequence Performance + +The insertion performance of sequences is comparable to the `auto-increment` of the same cache size (now can be specified with `auto_id_cache` table option) in our recent tests on IDC cluster. The default cache size of sequence is 1000, which could achieve approximately 3 thousands TPS with 64 threads under our own SysBench. + +## Recommended scenarios for Sequence + +- Users have requirements for generating sequence value shared by multiple tables. + +- The user business tier prefers to encapsulate the unique value, for instance, combining the unique value with the date and the user ID to form a complete ID. And users wish that the unique value is maintained by the database. + +- Users are sensitive to the sequence space. To save the sequence space as much as possible, the trade-off is between the performance and the amount of `CACHE`, and even using the `NOCACHE` mode. + +## Compatibility + +Specific compatibility can be referred by [MySQL’s compatibility](https://pingcap.com/docs/stable/reference/mysql-compatibility/). diff --git a/docs/design/2020-04-20-brie.md b/docs/design/2020-04-20-brie.md new file mode 100644 index 0000000000000..7a20694b87dbf --- /dev/null +++ b/docs/design/2020-04-20-brie.md @@ -0,0 +1,331 @@ +# Proposal: Backup/Restore/Import/Export via SQL + +- Author(s): [kennytm](https://github.com/kennytm) +- Last updated: 2020-04-20 +- Discussion at: https://github.com/pingcap/tidb/pull/15274 + +## Abstract + +Support running backup, restore, import and export (BRIE) tasks directly on TiDB via SQL. + +## Background + +[BR] is a TiDB ecosystem tool providing fast, KV-level logical backup and restore in [SST] format. +[Lightning] is another tool for importing SQL and CSV data dump into TiDB, using SST as intermediate storage. + +We would like to integrate these tools directly into TiDB as SQL statements because + +* BR and Lightning requires direct connection with TiKV and PD. + Since direct operation on TiKV bypass TiDB's authentication, these ports must be closed from public. + In highly protected systems like DBaaS, this means BR and Lightning as separate tools cannot be used at all. + +* Supporting SQL access also improves usability, as you don't need to install new components to use these functions, + and SQL should be more familiar with database operators. + +[BR]: https://github.com/pingcap/br/ +[SST]: https://github.com/facebook/rocksdb/wiki/Creating-and-Ingesting-SST-files +[Lightning]: https://github.com/pingcap/tidb-lightning + +## Proposal + +### Syntax + +Introduce 4 kinds of statements for executing backup, restore, import and export: + +```sql +BACKUP DATABASE * TO 's3://storage/url'; +RESTORE DATABASE * FROM 's3://storage/url'; +IMPORT DATABASE * FROM 'file:///data/dump/'; +EXPORT DATABASE * TO 's3://storage/url'; +``` + +(Note: there is no plan to implement `EXPORT` as of TiDB 4.0.) + +`SCHEMA` can be used as alias of `DATABASE`, as usual. + +These can be restricted to selected databases or tables only: + +```sql +BACKUP DATABASE db1, db2, db3 TO 's3://storage/url'; +BACKUP TABLE db1.tbl1, db2.tbl2, db3.tbl3 TO 's3://storage/url'; +``` + +Use query parameters to describe about the data source. Other options can be supplied at the end of the statement. + +```sql +RESTORE DATABASE * + FROM 's3://storage/url?access-key=AAAAA&secret-access-key=SSSSS&force-path-style=1&endpoint=http://10.0.1.55:8888' + RATE_LIMIT = 512 MB/SECOND; +``` + +The `=` sign in the options can be omitted. + +### List of options + +
Data source parameters + +* s3:// + * endpoint=http://s3.example.com:8080 + * region=us-east-1 + * storage-class=STANDARD + * sse=AES256 + * acl=READ + * access-key=AKIAIOSFODNN7EXAMPLE + * secret-access-key=wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY + * provider=aws + * force-path-style=1 + * use-accelerate-endpoint=0 +* gcs:// + * endpoint=http://gcs.example.com:8080 + * storage-class=STANDARD + * predefined-acl=READER + * credentials-file=/home/tidb/gcp_creds.json + +
+ +
Common options + +* SEND_CREDENTIALS_TO_TIKV = 1 +* CONCURRENCY = 32 +* CHECKSUM = 1 +* RATE_LIMIT = 125 MB/SECOND +* CHECKPOINT = 1 + +
+ +
Backup/Export options + +* SNAPSHOT = 4 HOUR AGO +* SNAPSHOT = '2020-04-20 15:00:00' +* SNAPSHOT = 416110777682551 +* LAST_BACKUP = '2020-04-20 15:00:00' +* LAST_BACKUP = 416110777682551 + +
+ +
Restore/Import options + +* ONLINE = 1 + +
+ +
Import/Export CSV options + +* CSV_SEPARATOR = ',' +* CSV_DELIMITER = '"' +* CSV_HEADER = COLUMNS +* CSV_HEADER = 0 +* CSV_NOT_NULL = 1 +* CSV_NULL = '\\N' +* CSV_BACKSLASH_ESCAPE = 1 +* CSV_TRIM_LAST_SEPARATORS = 1 + +
+ +
Other Import options + +* ANALYZE = 1 +* SKIP_SCHEMA_FILES = 1 +* BACKEND = 'local' +* TIKV_IMPORTER = '127.0.0.1:8287' +* ON_DUPLICATE = 'replace' +* STRICT_FORMAT = 1 + +
+ +Some niche configuration (e.g. Lightning's `io-concurrency`) are not exposed currently. + +### Execution + +All 4 kinds of BRIE statements are **synchronous** — they block the current session until completion. +Nevertheless, these queries can be canceled using the standard `KILL TIDB QUERY n` statement. + +Additionally, because the BRIE tasks are not using transactions, they cannot be executed concurrently. +When one BRIE task is already running, another BRIE statement must not start until the existing task has completed. + +Because the BRIE statements interact with TiKV directly and can read from / write to arbitrary path, +all 4 statements require SUPER privilege to execute. + +If the statement is canceled or an error is encountered, it will report an error e.g. +``` +MySQL [tpcc]> backup table tpcc.stock to 'local:///tmp/br_tpcc_30'; +ERROR 8124 (HY000): Backup failed: backup meta exists, may be some backup files in the path already +``` + +If the statement completed successfully, it will report one row of summary e.g. +``` +MySQL [tpcc]> backup database tpcc to 'local:///tmp/br_tpcc_32'; ++-------------------------+-----------+-----------------+---------------------+---------------------+ +| Destination | Size | BackupTS | Queue Time | Execution Time | ++-------------------------+-----------+-----------------+---------------------+---------------------+ +| local:///tmp/br_tpcc_32 | 248665063 | 416099531454472 | 2020-04-12 23:09:48 | 2020-04-12 23:09:48 | ++-------------------------+-----------+-----------------+---------------------+---------------------+ +1 row in set (58.453 sec) +``` + +> **Note:** +> +> The `RESTORE` (BR) and `IMPORT` (Lightning) operations are not using transactions and are not ACID compliant. +> +> The data and index KV pairs are ingested separately, i.e. a table being restored would have in an intermediate state +> where some rows are missing index, and some index doesn't have corresponding rows. +> If the operation has failed or canceled, the table would be stuck in this inconsistent state. +> +> There is currently no plan to change this behavior. + +### Status query + +The status of the queued and running tasks can be queried using the `SHOW` statement: + +```sql +SHOW BACKUPS; +SHOW RESTORES; +SHOW IMPORTS; +SHOW EXPORTS; +``` + +Example: +``` +MySQL [(none)]> show backups; ++-------------------------+---------+-------------------+---------------------+---------------------+------------+ +| Destination | State | Progress | Queue_Time | Execution_Time | Connection | ++-------------------------+---------+-------------------+---------------------+---------------------+------------+ +| local:///tmp/br_tpcc_30 | Backup | 98.38709677419355 | 2020-04-12 23:09:03 | 2020-04-12 23:09:25 | 4 | +| s3://bucket/prefix | Wait | 0 | 2020-04-12 23:09:48 | 0000-00-00 00:00:00 | 9 | ++-------------------------+---------+-------------------+---------------------+---------------------+------------+ +``` + +The content can be filtered using `LIKE` and `WHERE` clauses, e.g. +```sql +SHOW BACKUP LIKE 'local:%'; -- same as `Destination` LIKE 'local:%' +SHOW BACKUP WHERE `State` = 'Wait'; +``` + +Similar to the BRIE statements, these `SHOW` statements also require SUPER privilege. + +To protect against leakage of secret keys, +all query parameters of S3 and GCS destinations are discarded in the SHOW statements. + +## Rationale + +### Synchronous execution + +We have chosen synchronous execution entirely due to usability. +Internal tests find that it is very confusing that running `BACKUP` returns immediately while nothing is written out, +and not knowing when the `BACKUP` has finished. + +This is also similar to the treatment of other long-running tasks in MySQL like `CREATE INDEX` and `ANALYZE`. + +In most existing RDBMS which supports BRIE via SQL, they are executed synchronously: + +| Dialect | Sync? | Syntax | +|---------|-------|--------| +| [Transact-SQL](https://docs.microsoft.com/en-us/sql/t-sql/statements/restore-statements-transact-sql?view=sql-server-ver15) | Sync | `RESTORE DATABASE db1 FROM URL = 'https://example.com/backup1.bak'` | +| [IBM DB2](https://www.ibm.com/support/knowledgecenter/en/SSEPGG_11.1.0/com.ibm.db2.luw.admin.cmd.doc/doc/r0001976.html) | Sync | `RESTORE DATABASE db1 FROM 'db2remote://example//backup/1'` | +| [SAP HANA](https://help.sap.com/viewer/4fe29514fd584807ac9f2a04f6754767/2.0.04/en-US/93637a07e3b544398aa02de1541b903c.html) | Sync | `RECOVER DATA USING FILE ('/data/backup1') CLEAR LOG` | +| [MemSQL](https://docs.memsql.com/v7.0/reference/sql-reference/pipelines-commands/create-pipeline/) (ETL) | Async | `CREATE PIPELINE restore_pl AS LOAD DATA S3 'bucket/backup1' CREDENTIALS '{…}' INTO TABLE db1.tbl1; START PIPELINE restore_pl` | +| [CockroachDB](https://www.cockroachlabs.com/docs/stable/restore.html) | Sync | `RESTORE DATABASE db1 FROM 's3://bucket/backup1'` | + +On the other hand, synchronous execution is prone to accidental cancellation due to losing connection. + +Anyway, if we do want to make it asynchronous, the syntax should be changed to emphasize this aspect (similar to MemSQL's syntax). + +### Syntax alternatives + +The current syntax inherits from an early design, but they may be tweaked, e.g. + +* We use `DATABASE *` to indicate no filtering, but perhaps we should omit it? + + ```sql + BACKUP TO 's3://storage/url'; + RESTORE FROM 's3://storage/url'; + IMPORT FROM 'file:///data/dump/'; + EXPORT TO 's3://storage/url'; + ``` + +### Configuring the data source + +Currently we encode all storage-related parameters into the data source URL. +There are two other alternatives: + +1. Use options: + + ```sql + RESTORE DATABASE * FROM 's3://storage/url' + S3_ACCESS_KEY = 'AAAAA' + S3_SECRET_ACCESS_KEY = 'SSSSS' + S3_FORCE_PATH_STYLE = 1 + S3_ENDPOINT = 'http://10.0.1.55:8888' + RATE_LIMIT = 512 MB/SECOND; + ``` + +2. Use a JSON configuration: + + ```sql + RESTORE DATABASE * FROM 's3://storage/url' + CONFIG = '{"s3":{"access-key":"AAAAA", "secret-access-key":"SSSSS", "force-path-style":true, "endpoint":"http://10.0.1.55:8888"}}' + RATE_LIMIT = 512 MB/SECOND; + ``` + +The current BR tool's command line is close to the first option, changed from using query parameters. +This was mainly introduced to avoid accidentally leaking the secret keys. +In this integration we need to strip off the query parameter as soon as possible. + +For comparison, to configure a cloud destination in other RDBMS, + +| Dialect | Configuration | +|---------|---------------| +| Transact-SQL | Automatically configured inside Azure, not supported otherwise | +| IBM DB2 | User needs to configure the `db2remote://` first, before executing the BRIE statements | +| SAP HANA | Cloud storage seems not supported | +| MemSQL | Two JSON strings, one for credentials and one for non-secret settings (e.g. region, endpoint) | +| CockroachDB | URL query parameters | + +## Compatibility and Migration Plan + +The four BRIE statements are pure additions in TiDB 4.0+, and thus compatible with older versions. + +During data migration, the `BACKUP` and `EXPORT` statements from upstream should be ignored. + +`RESTORE` and `IMPORT` statements cannot be properly replicated. +Both of these will execute some DDL statements via SQL (which can be replicated). +However, the data ingestion skips the SQL layer, so binlog replication fails. +Even at TiKV level, because downstream doesn't have the same SST files, CDC may not work properly. + +Currently `RESTORE` and `IMPORT` only support *offline* execution on a *fresh* cluster, i.e. +the cluster is not supposed to be serving external actors, and replication should not be set up, +so this is not expected to be a problem. +However, we should take this in mind when we support true *online* restore. + +## Implementation + +The BRIE statements are implemented by embedding [BR] and [Lightning] as modules into TiDB. + +These modules will expose an entry point (e.g. [the `RunBackup` function] in BR) which + +* accepts a configuration structure defining entirely how a task should run, and +* executes the task in the current goroutine + +The entry point function will be shared by both the CLI tool and TiDB executor. +TiDB should provide every information it knows into the configuration, including: + +* PD addresses +* TLS certificate and key paths + +The tools should upstream logs and metrics (which should happen automatically). + +[the `RunBackup` function]: https://github.com/pingcap/br/blob/f5c37f36457b582710634f31fc2b9a1fb66254c6/pkg/task/backup.go#L94 + +On the Backup/Restore part, PRs are already submitted to implement this: + +* Parser — https://github.com/pingcap/parser/pull/746 +* TiDB — https://github.com/pingcap/tidb/pull/15274 + +The Import (Lightning) part will start after the Backup/Restore design is accepted. + +There is currently no plan to implement the Export part. + +## Testing Plan + +Since BRIE relies on proper TiKV, the built-in testing framework using mocktikv will not work. +Testing is going to be relying entirely on integration tests. diff --git a/domain/domain.go b/domain/domain.go index 239b5b963248a..f0403ea555066 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -408,9 +408,12 @@ func (do *Domain) ShowSlowQuery(showSlow *ast.ShowSlow) []*SlowQueryInfo { func (do *Domain) topNSlowQueryLoop() { defer recoverInDomain("topNSlowQueryLoop", false) - defer do.wg.Done() ticker := time.NewTicker(time.Minute * 10) - defer ticker.Stop() + defer func() { + ticker.Stop() + do.wg.Done() + logutil.BgLogger().Info("topNSlowQueryLoop exited.") + }() for { select { case now := <-ticker.C: @@ -436,8 +439,11 @@ func (do *Domain) topNSlowQueryLoop() { } func (do *Domain) infoSyncerKeeper() { - defer do.wg.Done() - defer recoverInDomain("infoSyncerKeeper", false) + defer func() { + do.wg.Done() + logutil.BgLogger().Info("infoSyncerKeeper exited.") + recoverInDomain("infoSyncerKeeper", false) + }() ticker := time.NewTicker(infosync.ReportInterval) defer ticker.Stop() for { @@ -457,10 +463,13 @@ func (do *Domain) infoSyncerKeeper() { } func (do *Domain) topologySyncerKeeper() { - defer do.wg.Done() defer recoverInDomain("topologySyncerKeeper", false) ticker := time.NewTicker(infosync.TopologyTimeToRefresh) - defer ticker.Stop() + defer func() { + ticker.Stop() + do.wg.Done() + logutil.BgLogger().Info("topologySyncerKeeper exited.") + }() for { select { @@ -482,12 +491,15 @@ func (do *Domain) topologySyncerKeeper() { } func (do *Domain) loadSchemaInLoop(lease time.Duration) { - defer do.wg.Done() + defer recoverInDomain("loadSchemaInLoop", true) // Lease renewal can run at any frequency. // Use lease/2 here as recommend by paper. ticker := time.NewTicker(lease / 2) - defer ticker.Stop() - defer recoverInDomain("loadSchemaInLoop", true) + defer func() { + ticker.Stop() + do.wg.Done() + logutil.BgLogger().Info("loadSchemaInLoop exited.") + }() syncer := do.ddl.SchemaSyncer() for { @@ -552,8 +564,8 @@ func (do *Domain) mustRestartSyncer() error { if do.isClose() { return err } - time.Sleep(time.Second) logutil.BgLogger().Error("restart the schema syncer failed", zap.Error(err)) + time.Sleep(time.Second) } } @@ -824,8 +836,11 @@ func (do *Domain) LoadPrivilegeLoop(ctx sessionctx.Context) error { do.wg.Add(1) go func() { - defer do.wg.Done() - defer recoverInDomain("loadPrivilegeInLoop", false) + defer func() { + do.wg.Done() + logutil.BgLogger().Info("loadPrivilegeInLoop exited.") + recoverInDomain("loadPrivilegeInLoop", false) + }() var count int for { ok := true @@ -885,8 +900,11 @@ func (do *Domain) LoadBindInfoLoop(ctxForHandle sessionctx.Context, ctxForEvolve func (do *Domain) globalBindHandleWorkerLoop() { do.wg.Add(1) go func() { - defer do.wg.Done() - defer recoverInDomain("globalBindHandleWorkerLoop", false) + defer func() { + do.wg.Done() + logutil.BgLogger().Info("globalBindHandleWorkerLoop exited.") + recoverInDomain("globalBindHandleWorkerLoop", false) + }() bindWorkerTicker := time.NewTicker(bindinfo.Lease) defer bindWorkerTicker.Stop() for { @@ -911,8 +929,11 @@ func (do *Domain) globalBindHandleWorkerLoop() { func (do *Domain) handleEvolvePlanTasksLoop(ctx sessionctx.Context) { do.wg.Add(1) go func() { - defer do.wg.Done() - defer recoverInDomain("handleEvolvePlanTasksLoop", false) + defer func() { + do.wg.Done() + logutil.BgLogger().Info("handleEvolvePlanTasksLoop exited.") + recoverInDomain("handleEvolvePlanTasksLoop", false) + }() owner := do.newOwnerManager(bindinfo.Prompt, bindinfo.OwnerKey) for { select { @@ -1004,13 +1025,16 @@ func (do *Domain) newOwnerManager(prompt, ownerKey string) owner.Manager { func (do *Domain) loadStatsWorker() { defer recoverInDomain("loadStatsWorker", false) - defer do.wg.Done() lease := do.statsLease if lease == 0 { lease = 3 * time.Second } loadTicker := time.NewTicker(lease) - defer loadTicker.Stop() + defer func() { + loadTicker.Stop() + do.wg.Done() + logutil.BgLogger().Info("loadStatsWorker exited.") + }() statsHandle := do.StatsHandle() t := time.Now() err := statsHandle.InitStats(do.InfoSchema()) @@ -1040,17 +1064,18 @@ func (do *Domain) updateStatsWorker(ctx sessionctx.Context, owner owner.Manager) defer recoverInDomain("updateStatsWorker", false) lease := do.statsLease deltaUpdateTicker := time.NewTicker(20 * lease) - defer deltaUpdateTicker.Stop() gcStatsTicker := time.NewTicker(100 * lease) - defer gcStatsTicker.Stop() dumpFeedbackTicker := time.NewTicker(200 * lease) - defer dumpFeedbackTicker.Stop() loadFeedbackTicker := time.NewTicker(5 * lease) - defer loadFeedbackTicker.Stop() statsHandle := do.StatsHandle() defer func() { + loadFeedbackTicker.Stop() + dumpFeedbackTicker.Stop() + gcStatsTicker.Stop() + deltaUpdateTicker.Stop() do.SetStatsUpdating(false) do.wg.Done() + logutil.BgLogger().Info("updateStatsWorker exited.") }() for { select { @@ -1103,6 +1128,7 @@ func (do *Domain) autoAnalyzeWorker(owner owner.Manager) { defer func() { analyzeTicker.Stop() do.wg.Done() + logutil.BgLogger().Info("autoAnalyzeWorker exited.") }() for { select { diff --git a/domain/domain_test.go b/domain/domain_test.go index 5669809e15822..8c6be97b16505 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -90,6 +90,7 @@ func TestInfo(t *testing.T) { if !unixSocketAvailable() { return } + testleak.BeforeTest() defer testleak.AfterTestT(t)() ddlLease := 80 * time.Millisecond s, err := mockstore.NewMockTikvStore() @@ -170,7 +171,7 @@ func TestInfo(t *testing.T) { } time.Sleep(15 * time.Millisecond) syncerStarted := false - for i := 0; i < 200; i++ { + for i := 0; i < 1000; i++ { if dom.SchemaValidator.IsStarted() { syncerStarted = true break diff --git a/domain/global_vars_cache.go b/domain/global_vars_cache.go index d0fca6ca30038..fb922317793e1 100644 --- a/domain/global_vars_cache.go +++ b/domain/global_vars_cache.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/stmtsummary" "go.uber.org/zap" + "golang.org/x/sync/singleflight" ) // GlobalVariableCache caches global variables. @@ -34,10 +35,11 @@ type GlobalVariableCache struct { fields []*ast.ResultField // Unit test may like to disable it. - disable bool + disable bool + SingleFight singleflight.Group } -const globalVariableCacheExpiry time.Duration = 2 * time.Second +const globalVariableCacheExpiry = 2 * time.Second // Update updates the global variable cache. func (gvc *GlobalVariableCache) Update(rows []chunk.Row, fields []*ast.ResultField) { @@ -62,6 +64,33 @@ func (gvc *GlobalVariableCache) Get() (succ bool, rows []chunk.Row, fields []*as return } +type loadResult struct { + rows []chunk.Row + fields []*ast.ResultField +} + +// LoadGlobalVariables will load from global cache first, loadFn will be executed if cache is not valid +func (gvc *GlobalVariableCache) LoadGlobalVariables(loadFn func() ([]chunk.Row, []*ast.ResultField, error)) ([]chunk.Row, []*ast.ResultField, error) { + succ, rows, fields := gvc.Get() + if succ { + return rows, fields, nil + } + fn := func() (interface{}, error) { + resRows, resFields, loadErr := loadFn() + if loadErr != nil { + return nil, loadErr + } + gvc.Update(resRows, resFields) + return &loadResult{resRows, resFields}, nil + } + res, err, _ := gvc.SingleFight.Do("loadGlobalVariable", fn) + if err != nil { + return nil, nil, err + } + loadRes := res.(*loadResult) + return loadRes.rows, loadRes.fields, nil +} + // Disable disables the global variable cache, used in test only. func (gvc *GlobalVariableCache) Disable() { gvc.Lock() diff --git a/domain/global_vars_cache_test.go b/domain/global_vars_cache_test.go index f3e3d7d654bae..723d2c1c5a441 100644 --- a/domain/global_vars_cache_test.go +++ b/domain/global_vars_cache_test.go @@ -14,6 +14,8 @@ package domain import ( + "sync" + "sync/atomic" "time" . "github.com/pingcap/check" @@ -29,7 +31,7 @@ import ( "github.com/pingcap/tidb/util/testleak" ) -var _ = Suite(&testGVCSuite{}) +var _ = SerialSuites(&testGVCSuite{}) type testGVCSuite struct{} @@ -99,6 +101,74 @@ func getResultField(colName string, id, offset int) *ast.ResultField { } } +func (gvcSuite *testGVCSuite) TestConcurrentOneFlight(c *C) { + defer testleak.AfterTest(c)() + testleak.BeforeTest() + gvc := &GlobalVariableCache{} + succ, rows, fields := gvc.Get() + c.Assert(succ, IsFalse) + c.Assert(rows, IsNil) + c.Assert(fields, IsNil) + + // Get a variable from global vars cache. + rf := getResultField("c", 1, 0) + rf1 := getResultField("c1", 2, 1) + ft := &types.FieldType{ + Tp: mysql.TypeString, + Charset: charset.CharsetBin, + Collate: charset.CollationBin, + } + ft1 := &types.FieldType{ + Tp: mysql.TypeString, + Charset: charset.CharsetBin, + Collate: charset.CollationBin, + } + ckLow := chunk.NewChunkWithCapacity([]*types.FieldType{ft, ft1}, 1024) + val := "fromStorage" + val1 := "fromStorage1" + ckLow.AppendString(0, val) + ckLow.AppendString(1, val1) + + // Let cache become invalid, and try concurrent load + counter := int32(0) + waitToStart := new(sync.WaitGroup) + waitToStart.Add(1) + gvc.lastModify = time.Now().Add(time.Duration(-10) * time.Second) + loadFunc := func() ([]chunk.Row, []*ast.ResultField, error) { + time.Sleep(100 * time.Millisecond) + atomic.AddInt32(&counter, 1) + return []chunk.Row{ckLow.GetRow(0)}, []*ast.ResultField{rf, rf1}, nil + } + wg := new(sync.WaitGroup) + worker := 100 + resArray := make([]loadResult, worker) + for i := 0; i < worker; i++ { + wg.Add(1) + go func(idx int) { + defer wg.Done() + waitToStart.Wait() + resRow, resField, _ := gvc.LoadGlobalVariables(loadFunc) + resArray[idx].rows = resRow + resArray[idx].fields = resField + }(i) + } + waitToStart.Done() + wg.Wait() + succ, rows, fields = gvc.Get() + c.Assert(counter, Equals, int32(1)) + c.Assert(resArray[0].rows[0].GetString(0), Equals, val) + c.Assert(resArray[0].rows[0].GetString(1), Equals, val1) + for i := 0; i < worker; i++ { + c.Assert(resArray[0].rows[0], Equals, resArray[i].rows[0]) + c.Assert(resArray[i].rows[0].GetString(0), Equals, val) + c.Assert(resArray[i].rows[0].GetString(1), Equals, val1) + } + // Validate cache + c.Assert(succ, IsTrue) + c.Assert(rows[0], Equals, resArray[0].rows[0]) + c.Assert(fields, DeepEquals, []*ast.ResultField{rf, rf1}) +} + func (gvcSuite *testGVCSuite) TestCheckEnableStmtSummary(c *C) { defer testleak.AfterTest(c)() testleak.BeforeTest() diff --git a/domain/infosync/info.go b/domain/infosync/info.go index 47e4c08cb6cb9..9fc07fbf80ffe 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -19,6 +19,7 @@ import ( "fmt" "net/http" "os" + "path" "strconv" "strings" "sync/atomic" @@ -301,7 +302,7 @@ func (is *InfoSyncer) RemoveServerInfo() { type topologyInfo struct { ServerVersionInfo StatusPort uint `json:"status_port"` - BinaryPath string `json:"binary_path"` + DeployPath string `json:"deploy_path"` StartTimestamp int64 `json:"start_timestamp"` } @@ -310,13 +311,14 @@ func (is *InfoSyncer) getTopologyInfo() topologyInfo { if err != nil { s = "" } + dir := path.Dir(s) return topologyInfo{ ServerVersionInfo: ServerVersionInfo{ Version: mysql.TiDBReleaseVersion, GitHash: is.info.ServerVersionInfo.GitHash, }, StatusPort: is.info.StatusPort, - BinaryPath: s, + DeployPath: dir, StartTimestamp: is.info.StartTimestamp, } } diff --git a/domain/infosync/info_test.go b/domain/infosync/info_test.go index 940c77eb39c08..278b8291d1d52 100644 --- a/domain/infosync/info_test.go +++ b/domain/infosync/info_test.go @@ -18,6 +18,8 @@ import ( "encoding/json" "errors" "fmt" + "os" + "path" "reflect" "testing" "time" @@ -127,6 +129,17 @@ func TestTopology(t *testing.T) { t.Fatal(err) } + s, err := os.Executable() + if err != nil { + t.Fatal(err) + } + + dir := path.Dir(s) + + if topo.DeployPath != dir { + t.Fatal("DeployPath not match expected path") + } + if topo.StartTimestamp != 1282967700000 { t.Fatal("start_timestamp of topology info does not match") } diff --git a/errno/errcode.go b/errno/errcode.go index 1651494fd9a2a..dd8c96dd2964c 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1038,6 +1038,10 @@ const ( ErrPrivilegeCheckFail = 8121 ErrInvalidWildCard = 8122 ErrMixOfGroupFuncAndFieldsIncompatible = 8123 + ErrBRIEBackupFailed = 8124 + ErrBRIERestoreFailed = 8125 + ErrBRIEImportFailed = 8126 + ErrBRIEExportFailed = 8127 // Error codes used by TiDB ddl package ErrUnsupportedDDLOperation = 8200 @@ -1081,4 +1085,5 @@ const ( ErrWriteConflict = 9007 ErrTiKVStoreLimit = 9008 ErrPrometheusAddrIsNotSet = 9009 + ErrTiKVStaleCommand = 9010 ) diff --git a/errno/errname.go b/errno/errname.go index e04b43cf57efb..15180f740548d 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1068,6 +1068,11 @@ var MySQLErrName = map[uint16]string{ ErrUnsupportedSecondArgumentType: "JSON_OBJECTAGG: unsupported second argument type %v", ErrLockExpire: "TTL manager has timed out, pessimistic locks may expire, please commit or rollback this transaction", + ErrBRIEBackupFailed: "Backup failed: %s", + ErrBRIERestoreFailed: "Restore failed: %s", + ErrBRIEImportFailed: "Import failed: %s", + ErrBRIEExportFailed: "Export failed: %s", + // TiKV/PD errors. ErrPDServerTimeout: "PD server timeout", ErrTiKVServerTimeout: "TiKV server timeout", @@ -1078,4 +1083,5 @@ var MySQLErrName = map[uint16]string{ ErrWriteConflict: "Write conflict, txnStartTS=%d, conflictStartTS=%d, conflictCommitTS=%d, key=%s", ErrTiKVStoreLimit: "Store token is up to the limit, store id = %d", ErrPrometheusAddrIsNotSet: "Prometheus address is not set in PD and etcd", + ErrTiKVStaleCommand: "TiKV server reports stale command", } diff --git a/executor/adapter.go b/executor/adapter.go index db0dc4f4f6b8f..33422ab0bed9c 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -148,17 +148,7 @@ func (a *recordSet) NewChunk() *chunk.Chunk { func (a *recordSet) Close() error { err := a.executor.Close() - // `LowSlowQuery` and `SummaryStmt` must be called before recording `PrevStmt`. - a.stmt.LogSlowQuery(a.txnStartTS, a.lastErr == nil, false) - a.stmt.SummaryStmt() - sessVars := a.stmt.Ctx.GetSessionVars() - pps := types.CloneRow(sessVars.PreparedParams) - sessVars.PrevStmt = FormatSQL(a.stmt.OriginText(), pps) - a.stmt.logAudit() - // Detach the disk tracker from GlobalDiskUsageTracker after every execution - if stmtCtx := a.stmt.Ctx.GetSessionVars().StmtCtx; stmtCtx != nil && stmtCtx.DiskTracker != nil { - stmtCtx.DiskTracker.Detach() - } + a.stmt.CloseRecordSet(a.txnStartTS, a.lastErr) return err } @@ -418,10 +408,11 @@ func getMaxExecutionTime(sctx sessionctx.Context) uint64 { } type chunkRowRecordSet struct { - rows []chunk.Row - idx int - fields []*ast.ResultField - e Executor + rows []chunk.Row + idx int + fields []*ast.ResultField + e Executor + execStmt *ExecStmt } func (c *chunkRowRecordSet) Fields() []*ast.ResultField { @@ -442,6 +433,7 @@ func (c *chunkRowRecordSet) NewChunk() *chunk.Chunk { } func (c *chunkRowRecordSet) Close() error { + c.execStmt.CloseRecordSet(c.execStmt.Ctx.GetSessionVars().TxnCtx.StartTS, nil) return nil } @@ -473,7 +465,7 @@ func (a *ExecStmt) runPessimisticSelectForUpdate(ctx context.Context, e Executor } if req.NumRows() == 0 { fields := colNames2ResultFields(e.Schema(), a.OutputNames, a.Ctx.GetSessionVars().CurrentDB) - return &chunkRowRecordSet{rows: rows, fields: fields, e: e}, nil + return &chunkRowRecordSet{rows: rows, fields: fields, e: e, execStmt: a}, nil } iter := chunk.NewIterator4Chunk(req) for r := iter.Begin(); r != iter.End(); r = iter.Next() { @@ -772,6 +764,41 @@ func FormatSQL(sql string, pps variable.PreparedParams) stringutil.StringerFunc } } +var ( + sessionExecuteRunDurationInternal = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblInternal) + sessionExecuteRunDurationGeneral = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblGeneral) +) + +// FinishExecuteStmt is used to record some information after `ExecStmt` execution finished: +// 1. record slow log if needed. +// 2. record summary statement. +// 3. record execute duration metric. +// 4. update the `PrevStmt` in session variable. +func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, succ bool, hasMoreResults bool) { + // `LowSlowQuery` and `SummaryStmt` must be called before recording `PrevStmt`. + a.LogSlowQuery(txnTS, succ, hasMoreResults) + a.SummaryStmt(succ) + sessVars := a.Ctx.GetSessionVars() + pps := types.CloneRow(sessVars.PreparedParams) + sessVars.PrevStmt = FormatSQL(a.OriginText(), pps) + executeDuration := time.Since(sessVars.StartTime) - sessVars.DurationCompile + if sessVars.InRestrictedSQL { + sessionExecuteRunDurationInternal.Observe(executeDuration.Seconds()) + } else { + sessionExecuteRunDurationGeneral.Observe(executeDuration.Seconds()) + } +} + +// CloseRecordSet will finish the execution of current statement and do some record work +func (a *ExecStmt) CloseRecordSet(txnStartTS uint64, lastErr error) { + a.FinishExecuteStmt(txnStartTS, lastErr == nil, false) + a.logAudit() + // Detach the disk tracker from GlobalDiskUsageTracker after every execution + if stmtCtx := a.Ctx.GetSessionVars().StmtCtx; stmtCtx != nil && stmtCtx.DiskTracker != nil { + stmtCtx.DiskTracker.DetachFromGlobalTracker() + } +} + // LogSlowQuery is used to print the slow query in the log files. func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { sessVars := a.Ctx.GetSessionVars() @@ -873,8 +900,8 @@ func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (normalized, pla return } -// SummaryStmt collects statements for performance_schema.events_statements_summary_by_digest -func (a *ExecStmt) SummaryStmt() { +// SummaryStmt collects statements for information_schema.statements_summary +func (a *ExecStmt) SummaryStmt(succ bool) { sessVars := a.Ctx.GetSessionVars() // Internal SQLs must also be recorded to keep the consistency of `PrevStmt` and `PrevStmtDigest`. if !stmtsummary.StmtSummaryByDigestMap.Enabled() || (sessVars.InRestrictedSQL && !stmtsummary.StmtSummaryByDigestMap.EnabledInternal()) { @@ -942,5 +969,6 @@ func (a *ExecStmt) SummaryStmt() { MemMax: memMax, StartTime: sessVars.StartTime, IsInternal: sessVars.InRestrictedSQL, + Succeed: succ, }) } diff --git a/executor/admin.go b/executor/admin.go index 056aab7ad2332..ad5f602700e88 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -254,13 +254,13 @@ func (e *RecoverIndexExec) buildDAGPB(txn kv.Transaction, limitCnt uint64) (*tip return dagReq, nil } -func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transaction, t table.Table, startHandle int64, limitCnt uint64) (distsql.SelectResult, error) { +func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transaction, t table.Table, startHandle kv.Handle, limitCnt uint64) (distsql.SelectResult, error) { dagPB, err := e.buildDAGPB(txn, limitCnt) if err != nil { return nil, err } tblInfo := e.table.Meta() - ranges := []*ranger.Range{{LowVal: []types.Datum{types.NewIntDatum(startHandle)}, HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}}} + ranges := []*ranger.Range{{LowVal: []types.Datum{types.NewIntDatum(startHandle.IntValue())}, HighVal: []types.Datum{types.NewIntDatum(math.MaxInt64)}}} var builder distsql.RequestBuilder kvReq, err := builder.SetTableRanges(tblInfo.ID, ranges, nil). SetDAGRequest(dagPB). @@ -284,17 +284,17 @@ func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transactio } type backfillResult struct { - nextHandle int64 + nextHandle kv.Handle addedCount int64 scanRowCount int64 } func (e *RecoverIndexExec) backfillIndex(ctx context.Context) (int64, int64, error) { var ( - nextHandle = int64(math.MinInt64) - totalAddedCnt = int64(0) - totalScanCnt = int64(0) - lastLogCnt = int64(0) + nextHandle kv.Handle = kv.IntHandle(math.MinInt64) + totalAddedCnt = int64(0) + totalScanCnt = int64(0) + lastLogCnt = int64(0) result backfillResult ) for { @@ -312,7 +312,7 @@ func (e *RecoverIndexExec) backfillIndex(ctx context.Context) (int64, int64, err lastLogCnt = totalScanCnt logutil.Logger(ctx).Info("recover index", zap.String("table", e.table.Meta().Name.O), zap.String("index", e.index.Meta().Name.O), zap.Int64("totalAddedCnt", totalAddedCnt), - zap.Int64("totalScanCnt", totalScanCnt), zap.Int64("nextHandle", result.nextHandle)) + zap.Int64("totalScanCnt", totalScanCnt), zap.Stringer("nextHandle", result.nextHandle)) } // no more rows @@ -325,7 +325,7 @@ func (e *RecoverIndexExec) backfillIndex(ctx context.Context) (int64, int64, err } type recoverRows struct { - handle int64 + handle kv.Handle idxVals []types.Datum skip bool } @@ -349,12 +349,12 @@ func (e *RecoverIndexExec) fetchRecoverRows(ctx context.Context, srcResult dists if result.scanRowCount >= int64(e.batchSize) { return e.recoverRows, nil } - handle := row.GetInt64(handleIdx) + handle := kv.IntHandle(row.GetInt64(handleIdx)) idxVals := extractIdxVals(row, e.idxValsBufs[result.scanRowCount], e.colFieldTypes) e.idxValsBufs[result.scanRowCount] = idxVals e.recoverRows = append(e.recoverRows, recoverRows{handle: handle, idxVals: idxVals, skip: false}) result.scanRowCount++ - result.nextHandle = handle + 1 + result.nextHandle = handle.Next() } } @@ -390,15 +390,15 @@ func (e *RecoverIndexExec) batchMarkDup(txn kv.Transaction, rows []recoverRows) for i, key := range e.batchKeys { if val, found := values[string(key)]; found { if distinctFlags[i] { - handle, err1 := tables.DecodeHandle(val) + handle, err1 := tables.DecodeHandleInUniqueIndexValue(val) if err1 != nil { return err1 } - if handle != rows[i].handle { + if handle != rows[i].handle.IntValue() { logutil.BgLogger().Warn("recover index: the constraint of unique index is broken, handle in index is not equal to handle in table", zap.String("index", e.index.Meta().Name.O), zap.ByteString("indexKey", key), - zap.Int64("handleInTable", rows[i].handle), zap.Int64("handleInIndex", handle)) + zap.Stringer("handleInTable", rows[i].handle), zap.Int64("handleInIndex", handle)) } } rows[i].skip = true @@ -407,7 +407,7 @@ func (e *RecoverIndexExec) batchMarkDup(txn kv.Transaction, rows []recoverRows) return nil } -func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transaction, startHandle int64) (result backfillResult, err error) { +func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transaction, startHandle kv.Handle) (result backfillResult, err error) { result.nextHandle = startHandle srcResult, err := e.buildTableScan(ctx, txn, e.table, startHandle, uint64(e.batchSize)) if err != nil { @@ -502,7 +502,7 @@ func (e *CleanupIndexExec) getIdxColTypes() []*types.FieldType { func (e *CleanupIndexExec) batchGetRecord(txn kv.Transaction) (map[string][]byte, error) { for handle := range e.idxValues { - e.batchKeys = append(e.batchKeys, e.table.RecordKey(handle)) + e.batchKeys = append(e.batchKeys, e.table.RecordKey(kv.IntHandle(handle))) } values, err := txn.BatchGet(context.Background(), e.batchKeys) if err != nil { @@ -518,7 +518,7 @@ func (e *CleanupIndexExec) deleteDanglingIdx(txn kv.Transaction, values map[stri if err != nil { return err } - for _, idxVals := range e.idxValues[handle] { + for _, idxVals := range e.idxValues[handle.IntValue()] { if err := e.index.Delete(e.ctx.GetSessionVars().StmtCtx, txn, idxVals, handle); err != nil { return err } @@ -570,7 +570,7 @@ func (e *CleanupIndexExec) fetchIndex(ctx context.Context, txn kv.Transaction) e idxVals := extractIdxVals(row, e.idxValsBufs[e.scanRowCnt], e.idxColFieldTypes) e.idxValsBufs[e.scanRowCnt] = idxVals e.idxValues[handle] = append(e.idxValues[handle], idxVals) - idxKey, _, err := e.index.GenIndexKey(sc, idxVals, handle, nil) + idxKey, _, err := e.index.GenIndexKey(sc, idxVals, kv.IntHandle(handle), nil) if err != nil { return err } @@ -660,7 +660,7 @@ func (e *CleanupIndexExec) Open(ctx context.Context) error { e.batchKeys = make([]kv.Key, 0, e.batchSize) e.idxValsBufs = make([][]types.Datum, e.batchSize) sc := e.ctx.GetSessionVars().StmtCtx - idxKey, _, err := e.index.GenIndexKey(sc, []types.Datum{{}}, math.MinInt64, nil) + idxKey, _, err := e.index.GenIndexKey(sc, []types.Datum{{}}, kv.IntHandle(math.MinInt64), nil) if err != nil { return err } diff --git a/executor/admin_test.go b/executor/admin_test.go index 66db472d18a1d..c7229218e5eaf 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -19,8 +19,10 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" @@ -87,7 +89,7 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { sc := s.ctx.GetSessionVars().StmtCtx txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(1), 1) + err = indexOpr.Delete(sc, txn, types.MakeDatums(1), kv.IntHandle(1)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -110,7 +112,7 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(10), 10) + err = indexOpr.Delete(sc, txn, types.MakeDatums(10), kv.IntHandle(10)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -124,15 +126,15 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(1), 1) + err = indexOpr.Delete(sc, txn, types.MakeDatums(1), kv.IntHandle(1)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(2), 2) + err = indexOpr.Delete(sc, txn, types.MakeDatums(2), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(3), 3) + err = indexOpr.Delete(sc, txn, types.MakeDatums(3), kv.IntHandle(3)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(10), 10) + err = indexOpr.Delete(sc, txn, types.MakeDatums(10), kv.IntHandle(10)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(20), 20) + err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(20)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -184,13 +186,13 @@ func (s *testSuite5) TestAdminRecoverIndex1(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums("1"), 1) + err = indexOpr.Delete(sc, txn, types.MakeDatums("1"), kv.IntHandle(1)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums("2"), 2) + err = indexOpr.Delete(sc, txn, types.MakeDatums("2"), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums("3"), 3) + err = indexOpr.Delete(sc, txn, types.MakeDatums("3"), kv.IntHandle(3)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums("10"), 4) + err = indexOpr.Delete(sc, txn, types.MakeDatums("10"), kv.IntHandle(4)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -242,21 +244,21 @@ func (s *testSuite5) TestAdminCleanupIndex(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(1), -100) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(1), kv.IntHandle(-100)) c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(6), 100) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(6), kv.IntHandle(100)) c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(8), 100) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(8), kv.IntHandle(100)) c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(nil), 101) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(nil), kv.IntHandle(101)) c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(nil), 102) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(nil), kv.IntHandle(102)) c.Assert(err, IsNil) - _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(6), 200) + _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(6), kv.IntHandle(200)) c.Assert(err, IsNil) - _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(6), -200) + _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(6), kv.IntHandle(-200)) c.Assert(err, IsNil) - _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(8), -200) + _, err = indexOpr3.Create(s.ctx, txn, types.MakeDatums(8), kv.IntHandle(-200)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -313,11 +315,11 @@ func (s *testSuite5) TestAdminCleanupIndexPKNotHandle(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(7, 10), -100) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(7, 10), kv.IntHandle(-100)) c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(4, 6), 100) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(4, 6), kv.IntHandle(100)) c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(-7, 4), 101) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(-7, 4), kv.IntHandle(101)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -366,9 +368,9 @@ func (s *testSuite5) TestAdminCleanupIndexMore(c *C) { for i := 0; i < 2000; i++ { c1 := int64(2*i + 7) c2 := int64(2*i + 8) - _, err = indexOpr1.Create(s.ctx, txn, types.MakeDatums(c1, c2), c1) - c.Assert(err, IsNil) - _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(c2), c1) + _, err = indexOpr1.Create(s.ctx, txn, types.MakeDatums(c1, c2), kv.IntHandle(c1)) + c.Assert(err, IsNil, Commentf(errors.ErrorStack(err))) + _, err = indexOpr2.Create(s.ctx, txn, types.MakeDatums(c2), kv.IntHandle(c1)) c.Assert(err, IsNil) } err = txn.Commit(context.Background()) @@ -428,7 +430,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo) txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(i), int64(i)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i), kv.IntHandle(i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -442,7 +444,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { // Manual recover index. txn, err = s.store.Begin() c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(i), int64(i)) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(i), kv.IntHandle(i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -456,7 +458,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo) txn, err := s.store.Begin() c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(i+8), int64(i+8)) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(i+8), kv.IntHandle(i+8)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -466,7 +468,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), int64(i+8)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), kv.IntHandle(i+8)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -479,7 +481,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo) txn, err := s.store.Begin() c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(i+8), int64(i)) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(i+8), kv.IntHandle(i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -489,7 +491,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), int64(i)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), kv.IntHandle(i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -524,7 +526,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Index c2 is missing 11. txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(-10), -1) + err = indexOpr.Delete(sc, txn, types.MakeDatums(-10), kv.IntHandle(-1)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -541,7 +543,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Index c2 has one more values ​​than table data: 0, and the handle 0 hasn't correlative record. txn, err = s.store.Begin() c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(0), 0) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(0), kv.IntHandle(0)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -553,12 +555,12 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Index c2 has two more values than table data: 10, 13, and these handles have correlative record. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(0), 0) + err = indexOpr.Delete(sc, txn, types.MakeDatums(0), kv.IntHandle(0)) c.Assert(err, IsNil) // Make sure the index value "19" is smaller "21". Then we scan to "19" before "21". - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(19), 10) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(19), kv.IntHandle(10)) c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(13), 2) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(13), kv.IntHandle(2)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -569,9 +571,9 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Two indices have the same handle. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(13), 2) + err = indexOpr.Delete(sc, txn, types.MakeDatums(13), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(12), 2) + err = indexOpr.Delete(sc, txn, types.MakeDatums(12), kv.IntHandle(2)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -582,9 +584,9 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Index c2 has one line of data is 19, the corresponding table data is 20. txn, err = s.store.Begin() c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(12), 2) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(12), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(20), 10) + err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(10)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -594,9 +596,9 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Recover records. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn, types.MakeDatums(19), 10) + err = indexOpr.Delete(sc, txn, types.MakeDatums(19), kv.IntHandle(10)) c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(20), 10) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(20), kv.IntHandle(10)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -753,7 +755,7 @@ func (s *testSuite5) TestAdminCheckWithSnapshot(c *C) { idxOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) txn, err := s.store.Begin() c.Assert(err, IsNil) - _, err = idxOpr.Create(s.ctx, txn, types.MakeDatums(2), 100) + _, err = idxOpr.Create(s.ctx, txn, types.MakeDatums(2), kv.IntHandle(100)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/executor/aggfuncs/builder.go b/executor/aggfuncs/builder.go index f23706bec65d9..d255c6b48faaa 100644 --- a/executor/aggfuncs/builder.go +++ b/executor/aggfuncs/builder.go @@ -485,7 +485,7 @@ func buildLeadLag(aggFuncDesc *aggregation.AggFuncDesc, ordinal int) baseLeadLag offset, _, _ = expression.GetUint64FromConstant(aggFuncDesc.Args[1]) } var defaultExpr expression.Expression - defaultExpr = expression.Null + defaultExpr = expression.NewNull() if len(aggFuncDesc.Args) == 3 { defaultExpr = aggFuncDesc.Args[2] } diff --git a/executor/aggfuncs/func_bitfuncs.go b/executor/aggfuncs/func_bitfuncs.go index a0f62e3008d38..f576f3f5221b8 100644 --- a/executor/aggfuncs/func_bitfuncs.go +++ b/executor/aggfuncs/func_bitfuncs.go @@ -62,7 +62,7 @@ func (e *bitOrUint64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ func (*bitOrUint64) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { p1, p2 := (*partialResult4BitFunc)(src), (*partialResult4BitFunc)(dst) - *p2 |= uint64(*p1) + *p2 |= *p1 return nil } @@ -85,9 +85,34 @@ func (e *bitXorUint64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup return nil } +func (e *bitXorUint64) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { + p := (*partialResult4BitFunc)(pr) + for i := uint64(0); i < shiftStart; i++ { + inputValue, isNull, err := e.args[0].EvalInt(sctx, rows[lastStart+i]) + if err != nil { + return err + } + if isNull { + continue + } + *p ^= uint64(inputValue) + } + for i := uint64(0); i < shiftEnd; i++ { + inputValue, isNull, err := e.args[0].EvalInt(sctx, rows[lastEnd+i]) + if err != nil { + return err + } + if isNull { + continue + } + *p ^= uint64(inputValue) + } + return nil +} + func (*bitXorUint64) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { p1, p2 := (*partialResult4BitFunc)(src), (*partialResult4BitFunc)(dst) - *p2 ^= uint64(*p1) + *p2 ^= *p1 return nil } @@ -123,6 +148,6 @@ func (e *bitAndUint64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup func (*bitAndUint64) MergePartialResult(sctx sessionctx.Context, src, dst PartialResult) error { p1, p2 := (*partialResult4BitFunc)(src), (*partialResult4BitFunc)(dst) - *p2 &= uint64(*p1) + *p2 &= *p1 return nil } diff --git a/executor/aggfuncs/func_first_row.go b/executor/aggfuncs/func_first_row.go index d13950e5765fa..e963c5e12ccf2 100644 --- a/executor/aggfuncs/func_first_row.go +++ b/executor/aggfuncs/func_first_row.go @@ -479,7 +479,7 @@ func (e *firstRow4Enum) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup if err != nil { return err } - p.gotFirstRow, p.isNull, p.val = true, d.IsNull(), d.GetMysqlEnum() + p.gotFirstRow, p.isNull, p.val = true, d.IsNull(), d.GetMysqlEnum().Copy() break } return nil @@ -492,6 +492,7 @@ func (*firstRow4Enum) MergePartialResult(sctx sessionctx.Context, src PartialRes } return nil } + func (e *firstRow4Enum) AppendFinalResult2Chunk(sctx sessionctx.Context, pr PartialResult, chk *chunk.Chunk) error { p := (*partialResult4FirstRowEnum)(pr) if p.isNull || !p.gotFirstRow { @@ -525,7 +526,7 @@ func (e *firstRow4Set) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup if err != nil { return err } - p.gotFirstRow, p.isNull, p.val = true, d.IsNull(), d.GetMysqlSet() + p.gotFirstRow, p.isNull, p.val = true, d.IsNull(), d.GetMysqlSet().Copy() break } return nil diff --git a/executor/aggfuncs/func_lead_lag_test.go b/executor/aggfuncs/func_lead_lag_test.go index fd4e5aa23dfcb..c66b50c311185 100644 --- a/executor/aggfuncs/func_lead_lag_test.go +++ b/executor/aggfuncs/func_lead_lag_test.go @@ -22,8 +22,8 @@ import ( ) func (s *testSuite) TestLeadLag(c *C) { - zero := expression.Zero - one := expression.One + zero := expression.NewZero() + one := expression.NewOne() two := &expression.Constant{ Value: types.NewDatum(2), RetType: types.NewFieldType(mysql.TypeTiny), diff --git a/executor/aggregate.go b/executor/aggregate.go index 35df9e0a52aa9..a5b67eb770a4d 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -448,7 +448,7 @@ func getGroupKey(ctx sessionctx.Context, input *chunk.Chunk, groupKey [][]byte, return nil, err } - if err := expression.VecEval(ctx, item, input, buf); err != nil { + if err := expression.EvalExpr(ctx, item, input, buf); err != nil { expression.PutColumn(buf) return nil, err } @@ -1106,7 +1106,7 @@ func (e *vecGroupChecker) evalGroupItemsAndResolveGroups(item expression.Express return err } defer e.releaseBuffer(col) - err = expression.VecEval(e.ctx, item, chk, col) + err = expression.EvalExpr(e.ctx, item, chk, col) if err != nil { return err } diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 1dcb22a417030..56eb6627e6c45 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -621,6 +621,17 @@ func (s *testSuiteAgg) TestOnlyFullGroupBy(c *C) { c.Assert(terror.ErrorEqual(err, plannercore.ErrAmbiguous), IsTrue, Commentf("err %v", err)) } +func (s *testSuiteAgg) TestIssue16279(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'") + tk.MustExec("drop table if exists s") + tk.MustExec("create table s(a int);") + tk.MustQuery("select count(a) , date_format(a, '%Y-%m-%d') from s group by date_format(a, '%Y-%m-%d');") + tk.MustQuery("select count(a) , date_format(a, '%Y-%m-%d') as xx from s group by date_format(a, '%Y-%m-%d');") + tk.MustQuery("select count(a) , date_format(a, '%Y-%m-%d') as xx from s group by xx") +} + func (s *testSuiteAgg) TestIssue13652(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/analyze.go b/executor/analyze.go index 95a4f310eed1b..747bf0ac56a03 100755 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -509,7 +509,10 @@ func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range) (hists []*statis cms = append(cms, nil) } for i, col := range e.colsInfo { - collectors[i].ExtractTopN(uint32(e.opts[ast.AnalyzeOptNumTopN])) + err := collectors[i].ExtractTopN(uint32(e.opts[ast.AnalyzeOptNumTopN]), e.ctx.GetSessionVars().StmtCtx, &col.FieldType, timeZone) + if err != nil { + return nil, nil, err + } for j, s := range collectors[i].Samples { collectors[i].Samples[j].Ordinal = j collectors[i].Samples[j].Value, err = tablecodec.DecodeColumnValue(s.Value.GetBytes(), &col.FieldType, timeZone) @@ -809,7 +812,7 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if err != nil { return err } - var rowID int64 + var rowID kv.Handle rowID, err = tablecodec.DecodeRowKey(sKey) if err != nil { return err @@ -818,12 +821,12 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if hasPKInfo > 0 { v, ok := values[e.pkInfo.ID] if !ok { - var key int64 + var key kv.Handle _, key, err = tablecodec.DecodeRecordKey(sKey) if err != nil { return err } - v = types.NewIntDatum(key) + v = types.NewIntDatum(key.IntValue()) } if mysql.HasUnsignedFlag(e.pkInfo.Flag) { v.SetUint64(uint64(v.GetInt64())) @@ -831,7 +834,7 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if e.collectors[0].Samples[samplePos] == nil { e.collectors[0].Samples[samplePos] = &statistics.SampleItem{} } - e.collectors[0].Samples[samplePos].RowID = rowID + e.collectors[0].Samples[samplePos].RowID = rowID.IntValue() e.collectors[0].Samples[samplePos].Value = v } // Update the columns' collectors. @@ -843,7 +846,7 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if e.collectors[hasPKInfo+j].Samples[samplePos] == nil { e.collectors[hasPKInfo+j].Samples[samplePos] = &statistics.SampleItem{} } - e.collectors[hasPKInfo+j].Samples[samplePos].RowID = rowID + e.collectors[hasPKInfo+j].Samples[samplePos].RowID = rowID.IntValue() e.collectors[hasPKInfo+j].Samples[samplePos].Value = v } // Update the indexes' collectors. @@ -869,7 +872,7 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos] == nil { e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos] = &statistics.SampleItem{} } - e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos].RowID = rowID + e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos].RowID = rowID.IntValue() e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos].Value = types.NewBytesDatum(bytes) } return nil @@ -963,7 +966,8 @@ func (e *AnalyzeFastExec) handleSampTasks(bo *tikv.Backoffer, workID int, err *e continue } - var tableID, minRowID, maxRowID int64 + var tableID int64 + var minRowID, maxRowID kv.Handle startKey, endKey := task.Location.StartKey, task.Location.EndKey tableID, minRowID, *err = tablecodec.DecodeRecordKey(startKey) if *err != nil { @@ -976,8 +980,8 @@ func (e *AnalyzeFastExec) handleSampTasks(bo *tikv.Backoffer, workID int, err *e keys := make([]kv.Key, 0, task.SampSize) for i := 0; i < int(task.SampSize); i++ { - randKey := rander.Int63n(maxRowID-minRowID) + minRowID - keys = append(keys, tablecodec.EncodeRowKeyWithHandle(tableID, randKey)) + randKey := rander.Int63n(maxRowID.IntValue()-minRowID.IntValue()) + minRowID.IntValue() + keys = append(keys, tablecodec.EncodeRowKeyWithHandle(tableID, kv.IntHandle(randKey))) } kvMap := make(map[string][]byte, len(keys)) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 2b49e4cf382b1..72141b4832f67 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -480,6 +480,7 @@ func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) { cluster := mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(cluster) mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) store, err := mockstore.NewMockTikvStore( mockstore.WithHijackClient(hijackClient), mockstore.WithCluster(cluster), @@ -504,7 +505,7 @@ func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) { for i := 0; i < 30; i++ { tk.MustExec(fmt.Sprintf("insert into retry_row_count values (%d)", i)) } - cluster.SplitTable(mvccStore, tid, 6) + cluster.SplitTable(tid, 6) // Flush the region cache first. tk.MustQuery("select * from retry_row_count") tk.MustExec("analyze table retry_row_count") @@ -551,3 +552,41 @@ func (s *testSuite1) TestExtractTopN(c *C) { item = idxStats.CMSketch.TopN()[0] c.Assert(item.Count, Equals, uint64(11)) } + +func (s *testSuite1) TestHashInTopN(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 float, c decimal(30, 10), d varchar(20))") + tk.MustExec(`insert into t values + (1, 1.1, 11.1, "0110"), + (2, 2.2, 22.2, "0110"), + (3, 3.3, 33.3, "0110"), + (4, 4.4, 44.4, "0440")`) + for i := 0; i < 3; i++ { + tk.MustExec("insert into t select * from t") + } + // get stats of normal analyze + tk.MustExec("analyze table t") + is := s.dom.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tblInfo := tbl.Meta() + tblStats1 := s.dom.StatsHandle().GetTableStats(tblInfo).Copy() + // get stats of fast analyze + tk.MustExec("set @@tidb_enable_fast_analyze = 1") + tk.MustExec("analyze table t") + tblStats2 := s.dom.StatsHandle().GetTableStats(tblInfo).Copy() + // check the hash for topn + for _, col := range tblInfo.Columns { + topn1 := tblStats1.Columns[col.ID].CMSketch.TopNMap() + cm2 := tblStats2.Columns[col.ID].CMSketch + for h1, topnMetas := range topn1 { + for _, topnMeta1 := range topnMetas { + count2, exists := cm2.QueryTopN(h1, topnMeta1.GetH2(), topnMeta1.Data) + c.Assert(exists, Equals, true) + c.Assert(count2, Equals, topnMeta1.Count) + } + } + } +} diff --git a/executor/batch_checker.go b/executor/batch_checker.go index 7b46b716c87f1..7efe92545c661 100644 --- a/executor/batch_checker.go +++ b/executor/batch_checker.go @@ -117,7 +117,7 @@ func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.D handle := row[handleCol.Offset].GetInt64() handleKey = &keyValueWithDupInfo{ newKV: keyValue{ - key: t.RecordKey(handle), + key: t.RecordKey(kv.IntHandle(handle)), value: newRowValue, }, dupErr: kv.ErrKeyExists.FastGenByArgs(strconv.FormatInt(handle, 10), "PRIMARY"), @@ -136,7 +136,7 @@ func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.D // Pass handle = 0 to GenIndexKey, // due to we only care about distinct key. key, distinct, err1 := v.GenIndexKey(ctx.GetSessionVars().StmtCtx, - colVals, 0, nil) + colVals, kv.IntHandle(0), nil) if err1 != nil { return nil, err1 } @@ -167,7 +167,7 @@ func getKeysNeedCheckOneRow(ctx sessionctx.Context, t table.Table, row []types.D // getOldRow gets the table record row from storage for batch check. // t could be a normal table or a partition, but it must not be a PartitionedTable. -func getOldRow(ctx context.Context, sctx sessionctx.Context, txn kv.Transaction, t table.Table, handle int64, +func getOldRow(ctx context.Context, sctx sessionctx.Context, txn kv.Transaction, t table.Table, handle kv.Handle, genExprs []expression.Expression) ([]types.Datum, error) { oldValue, err := txn.Get(ctx, t.RecordKey(handle)) if err != nil { diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index f09d0e083d4ad..6ebb8162d8cdc 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -36,7 +36,7 @@ type BatchPointGetExec struct { tblInfo *model.TableInfo idxInfo *model.IndexInfo - handles []int64 + handles []kv.Handle physIDs []int64 partPos int idxVals [][]types.Datum @@ -175,7 +175,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { return err } - e.handles = make([]int64, 0, len(keys)) + e.handles = make([]kv.Handle, 0, len(keys)) if e.tblInfo.Partition != nil { e.physIDs = make([]int64, 0, len(keys)) } @@ -184,11 +184,11 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { if len(handleVal) == 0 { continue } - handle, err1 := tables.DecodeHandle(handleVal) + handle, err1 := tables.DecodeHandleInUniqueIndexValue(handleVal) if err1 != nil { return err1 } - e.handles = append(e.handles, handle) + e.handles = append(e.handles, kv.IntHandle(handle)) if e.tblInfo.Partition != nil { e.physIDs = append(e.physIDs, tablecodec.DecodeTableID(key)) } @@ -210,9 +210,9 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { } else if e.keepOrder { sort.Slice(e.handles, func(i int, j int) bool { if e.desc { - return e.handles[i] > e.handles[j] + return e.handles[i].Compare(e.handles[j]) > 0 } - return e.handles[i] < e.handles[j] + return e.handles[i].Compare(e.handles[j]) < 0 }) } @@ -222,7 +222,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { if len(e.physIDs) > 0 { tID = e.physIDs[i] } else { - tID = getPhysID(e.tblInfo, handle) + tID = getPhysID(e.tblInfo, handle.IntValue()) } key := tablecodec.EncodeRowKeyWithHandle(tID, handle) keys[i] = key @@ -250,7 +250,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { if err != nil { return err } - handles := make([]int64, 0, len(values)) + handles := make([]kv.Handle, 0, len(values)) var existKeys []kv.Key if e.lock && rc { existKeys = make([]kv.Key, 0, len(values)) diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index b041ceead54b3..43320312503a4 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -230,8 +230,8 @@ func buildMockDataSourceWithIndex(opt mockDataSourceParameters, index []int) *mo return buildMockDataSource(opt) } +// aggTestCase has a fixed schema (aggCol Double, groupBy LongLong). type aggTestCase struct { - // The test table's schema is fixed (aggCol Double, groupBy LongLong). execType string // "hash" or "stream" aggFunc string // sum, avg, count .... groupByNDV int // the number of distinct group-by keys @@ -505,8 +505,8 @@ func buildWindowExecutor(ctx sessionctx.Context, windowFunc string, funcs int, f return exec } +// windowTestCase has a fixed schema (col Double, partitionBy LongLong, rawData VarString(16), col LongLong). type windowTestCase struct { - // The test table's schema is fixed (col Double, partitionBy LongLong, rawData VarString(16), col LongLong). windowFunc string numFunc int // The number of windowFuncs. Default: 1. frame *core.WindowFrame @@ -681,6 +681,7 @@ func baseBenchmarkWindowFunctionsWithSlidingWindow(b *testing.B, frameType ast.F {ast.AggFuncCount, mysql.TypeLong}, {ast.AggFuncAvg, mysql.TypeFloat}, {ast.AggFuncAvg, mysql.TypeNewDecimal}, + {ast.AggFuncBitXor, mysql.TypeLong}, } row := 100000 ndv := 100 @@ -1341,12 +1342,28 @@ func BenchmarkIndexJoinExec(b *testing.B) { type mergeJoinTestCase struct { indexJoinTestCase + childrenUsedSchema [][]bool } func prepare4MergeJoin(tc *mergeJoinTestCase, leftExec, rightExec *mockDataSource) *MergeJoinExec { outerCols, innerCols := tc.columns(), tc.columns() - joinSchema := expression.NewSchema(outerCols...) - joinSchema.Append(innerCols...) + + joinSchema := expression.NewSchema() + if tc.childrenUsedSchema != nil { + for i, used := range tc.childrenUsedSchema[0] { + if used { + joinSchema.Append(outerCols[i]) + } + } + for i, used := range tc.childrenUsedSchema[1] { + if used { + joinSchema.Append(innerCols[i]) + } + } + } else { + joinSchema.Append(outerCols...) + joinSchema.Append(innerCols...) + } outerJoinKeys := make([]*expression.Column, 0, len(tc.outerJoinKeyIdx)) innerJoinKeys := make([]*expression.Column, 0, len(tc.innerJoinKeyIdx)) @@ -1370,19 +1387,20 @@ func prepare4MergeJoin(tc *mergeJoinTestCase, leftExec, rightExec *mockDataSourc stmtCtx: tc.ctx.GetSessionVars().StmtCtx, baseExecutor: newBaseExecutor(tc.ctx, joinSchema, stringutil.StringerStr("MergeJoin"), leftExec, rightExec), compareFuncs: compareFuncs, - joiner: newJoiner( - tc.ctx, - 0, - false, - defaultValues, - nil, - retTypes(leftExec), - retTypes(rightExec), - nil, - ), - isOuterJoin: false, + isOuterJoin: false, } + e.joiner = newJoiner( + tc.ctx, + 0, + false, + defaultValues, + nil, + retTypes(leftExec), + retTypes(rightExec), + tc.childrenUsedSchema, + ) + e.innerTable = &mergeJoinTable{ isInner: true, childIndex: 1, @@ -1399,7 +1417,7 @@ func prepare4MergeJoin(tc *mergeJoinTestCase, leftExec, rightExec *mockDataSourc } func defaultMergeJoinTestCase() *mergeJoinTestCase { - return &mergeJoinTestCase{*defaultIndexJoinTestCase()} + return &mergeJoinTestCase{*defaultIndexJoinTestCase(), nil} } func newMergeJoinBenchmark(numOuterRows, numInnerDup, numInnerRedundant int) (tc *mergeJoinTestCase, innerDS, outerDS *mockDataSource) { @@ -1421,7 +1439,7 @@ func newMergeJoinBenchmark(numOuterRows, numInnerDup, numInnerRedundant int) (tc innerIdx: []int{0, 1}, rawData: wideString, } - tc = &mergeJoinTestCase{*itc} + tc = &mergeJoinTestCase{*itc, nil} outerOpt := mockDataSourceParameters{ schema: expression.NewSchema(tc.columns()...), rows: numOuterRows, @@ -1514,37 +1532,35 @@ func BenchmarkMergeJoinExec(b *testing.B) { totalRows := 300000 - { - numInnerDup := 1 - tc, innerDS, outerDS := newMergeJoinBenchmark(totalRows/numInnerDup, numInnerDup, 0) - b.Run(fmt.Sprintf("merge join %v", tc), func(b *testing.B) { - benchmarkMergeJoinExecWithCase(b, tc, outerDS, innerDS, innerMergeJoin) - }) + innerDupAndRedundant := [][]int{ + {1, 0}, + {100, 0}, + {10000, 0}, + {1, 30000}, } - { - numInnerDup := 100 - tc, innerDS, outerDS := newMergeJoinBenchmark(totalRows/numInnerDup, numInnerDup, 0) - b.Run(fmt.Sprintf("merge join %v", tc), func(b *testing.B) { - benchmarkMergeJoinExecWithCase(b, tc, outerDS, innerDS, innerMergeJoin) - }) + childrenUsedSchemas := [][][]bool{ + nil, + { + {true, false, false}, + {false, true, false}, + }, } - { - numInnerDup := 10000 - tc, innerDS, outerDS := newMergeJoinBenchmark(totalRows/numInnerDup, numInnerDup, 0) - b.Run(fmt.Sprintf("merge join %v", tc), func(b *testing.B) { - benchmarkMergeJoinExecWithCase(b, tc, outerDS, innerDS, innerMergeJoin) - }) - } + for _, params := range innerDupAndRedundant { + numInnerDup, numInnerRedundant := params[0], params[1] + for _, childrenUsedSchema := range childrenUsedSchemas { + tc, innerDS, outerDS := newMergeJoinBenchmark(totalRows/numInnerDup, numInnerDup, numInnerRedundant) + inlineProj := false + if childrenUsedSchema != nil { + inlineProj = true + tc.childrenUsedSchema = childrenUsedSchema + } - { - numInnerDup := 1 - numInnerRedundant := 30000 - tc, innerDS, outerDS := newMergeJoinBenchmark(totalRows/numInnerDup, numInnerDup, numInnerRedundant) - b.Run(fmt.Sprintf("merge join %v", tc), func(b *testing.B) { - benchmarkMergeJoinExecWithCase(b, tc, outerDS, innerDS, innerMergeJoin) - }) + b.Run(fmt.Sprintf("merge join %v InlineProj:%v", tc, inlineProj), func(b *testing.B) { + benchmarkMergeJoinExecWithCase(b, tc, outerDS, innerDS, innerMergeJoin) + }) + } } } diff --git a/executor/brie.go b/executor/brie.go new file mode 100644 index 0000000000000..c23b018e1f0e7 --- /dev/null +++ b/executor/brie.go @@ -0,0 +1,470 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor + +import ( + "context" + "fmt" + "net/url" + "strings" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/br/pkg/glue" + "github.com/pingcap/br/pkg/storage" + "github.com/pingcap/br/pkg/task" + "github.com/pingcap/errors" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" + pd "github.com/pingcap/pd/v4/client" + "github.com/pingcap/tidb-tools/pkg/filter" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/stringutil" +) + +// brieTaskProgress tracks a task's current progress. +type brieTaskProgress struct { + // current progress of the task. + // this field is atomically updated outside of the lock below. + current int64 + + // lock is the mutex protected the two fields below. + lock sync.Mutex + // cmd is the name of the step the BRIE task is currently performing. + cmd string + // total is the total progress of the task. + // the percentage of completeness is `(100%) * current / total`. + total int64 +} + +// Inc implements glue.Progress +func (p *brieTaskProgress) Inc() { + atomic.AddInt64(&p.current, 1) +} + +// Close implements glue.Progress +func (p *brieTaskProgress) Close() { + p.lock.Lock() + atomic.StoreInt64(&p.current, p.total) + p.lock.Unlock() +} + +type brieTaskInfo struct { + queueTime types.Time + execTime types.Time + kind ast.BRIEKind + storage string + connID uint64 + backupTS uint64 + archiveSize uint64 +} + +type brieQueueItem struct { + info *brieTaskInfo + progress *brieTaskProgress + cancel func() +} + +type brieQueue struct { + nextID uint64 + tasks sync.Map + + workerCh chan struct{} +} + +// globalBRIEQueue is the BRIE execution queue. Only one BRIE task can be executed each time. +// TODO: perhaps copy the DDL Job queue so only one task can be executed in the whole cluster. +var globalBRIEQueue = &brieQueue{ + workerCh: make(chan struct{}, 1), +} + +// registerTask registers a BRIE task in the queue. +func (bq *brieQueue) registerTask( + ctx context.Context, + info *brieTaskInfo, +) (context.Context, uint64) { + taskCtx, taskCancel := context.WithCancel(ctx) + item := &brieQueueItem{ + info: info, + cancel: taskCancel, + progress: &brieTaskProgress{ + cmd: "Wait", + total: 1, + }, + } + + taskID := atomic.AddUint64(&bq.nextID, 1) + bq.tasks.Store(taskID, item) + + return taskCtx, taskID +} + +// acquireTask prepares to execute a BRIE task. Only one BRIE task can be +// executed at a time, and this function blocks until the task is ready. +// +// Returns an object to track the task's progress. +func (bq *brieQueue) acquireTask(taskCtx context.Context, taskID uint64) (*brieTaskProgress, error) { + // wait until we are at the front of the queue. + select { + case bq.workerCh <- struct{}{}: + if item, ok := bq.tasks.Load(taskID); ok { + return item.(*brieQueueItem).progress, nil + } + // cannot find task, perhaps it has been canceled. allow the next task to run. + bq.releaseTask() + return nil, errors.Errorf("backup/restore task %d is canceled", taskID) + case <-taskCtx.Done(): + return nil, taskCtx.Err() + } +} + +func (bq *brieQueue) releaseTask() { + <-bq.workerCh +} + +func (bq *brieQueue) cancelTask(taskID uint64) { + item, ok := bq.tasks.Load(taskID) + if !ok { + return + } + bq.tasks.Delete(taskID) + item.(*brieQueueItem).cancel() +} + +func (b *executorBuilder) parseTSString(ts string) (uint64, error) { + sc := &stmtctx.StatementContext{TimeZone: b.ctx.GetSessionVars().Location()} + t, err := types.ParseTime(sc, ts, mysql.TypeTimestamp, types.MaxFsp) + if err != nil { + return 0, err + } + t1, err := t.GoTime(sc.TimeZone) + if err != nil { + return 0, err + } + return variable.GoTimeToTS(t1), nil +} + +var brieStmtLabel fmt.Stringer = stringutil.StringerStr("BRIEStmt") + +func (b *executorBuilder) buildBRIE(s *ast.BRIEStmt, schema *expression.Schema) Executor { + e := &BRIEExec{ + baseExecutor: newBaseExecutor(b.ctx, schema, brieStmtLabel), + info: &brieTaskInfo{ + kind: s.Kind, + }, + } + + tidbCfg := config.GetGlobalConfig() + if tidbCfg.Store != "tikv" { + b.err = errors.Errorf("%s requires tikv store, not %s", s.Kind, tidbCfg.Store) + return nil + } + + cfg := task.Config{ + TLS: task.TLSConfig{ + CA: tidbCfg.Security.ClusterSSLCA, + Cert: tidbCfg.Security.ClusterSSLCert, + Key: tidbCfg.Security.ClusterSSLKey, + }, + PD: strings.Split(tidbCfg.Path, ","), + Concurrency: 4, + Checksum: true, + SendCreds: true, + LogProgress: true, + CaseSensitive: tidbCfg.LowerCaseTableNames == 0, + Filter: filter.Rules{ + DoDBs: s.Schemas, + }, + } + + storageURL, err := url.Parse(s.Storage) + if err != nil { + b.err = errors.Annotate(err, "invalid destination URL") + return nil + } + + switch storageURL.Scheme { + case "s3": + storage.ExtractQueryParameters(storageURL, &cfg.S3) + case "gs", "gcs": + storage.ExtractQueryParameters(storageURL, &cfg.GCS) + default: + break + } + + cfg.Storage = storageURL.String() + e.info.storage = cfg.Storage + + for _, opt := range s.Options { + switch opt.Tp { + case ast.BRIEOptionRateLimit: + cfg.RateLimit = opt.UintValue + case ast.BRIEOptionConcurrency: + cfg.Concurrency = uint32(opt.UintValue) + case ast.BRIEOptionChecksum: + cfg.Checksum = opt.UintValue != 0 + case ast.BRIEOptionSendCreds: + cfg.SendCreds = opt.UintValue != 0 + } + } + + if len(s.Tables) != 0 { + cfg.Filter.DoTables = make([]*filter.Table, 0, len(s.Tables)) + for _, tbl := range s.Tables { + // the `tbl.Schema` is always not empty if a database is used. + // this is handled by (*preprocessor).handleTableName(). + cfg.Filter.DoTables = append(cfg.Filter.DoTables, &filter.Table{ + Name: tbl.Name.O, + Schema: tbl.Schema.O, + }) + } + } + + switch s.Kind { + case ast.BRIEKindBackup: + e.backupCfg = &task.BackupConfig{Config: cfg} + + for _, opt := range s.Options { + switch opt.Tp { + case ast.BRIEOptionLastBackupTS: + tso, err := b.parseTSString(opt.StrValue) + if err != nil { + b.err = err + return nil + } + e.backupCfg.LastBackupTS = tso + case ast.BRIEOptionLastBackupTSO: + e.backupCfg.LastBackupTS = opt.UintValue + case ast.BRIEOptionBackupTimeAgo: + e.backupCfg.TimeAgo = time.Duration(opt.UintValue) + case ast.BRIEOptionBackupTSO: + e.backupCfg.BackupTS = opt.UintValue + case ast.BRIEOptionBackupTS: + tso, err := b.parseTSString(opt.StrValue) + if err != nil { + b.err = err + return nil + } + e.backupCfg.BackupTS = tso + } + } + + case ast.BRIEKindRestore: + e.restoreCfg = &task.RestoreConfig{Config: cfg} + for _, opt := range s.Options { + switch opt.Tp { + case ast.BRIEOptionOnline: + e.restoreCfg.Online = opt.UintValue != 0 + } + } + + default: + b.err = errors.Errorf("unsupported BRIE statement kind: %s", s.Kind) + return nil + } + + return e +} + +// BRIEExec represents an executor for BRIE statements (BACKUP, RESTORE, etc) +type BRIEExec struct { + baseExecutor + + backupCfg *task.BackupConfig + restoreCfg *task.RestoreConfig + info *brieTaskInfo +} + +// Next implements the Executor Next interface. +func (e *BRIEExec) Next(ctx context.Context, req *chunk.Chunk) error { + req.Reset() + if e.info == nil { + return nil + } + + bq := globalBRIEQueue + + e.info.connID = e.ctx.GetSessionVars().ConnectionID + e.info.queueTime = types.CurrentTime(mysql.TypeDatetime) + taskCtx, taskID := bq.registerTask(ctx, e.info) + defer bq.cancelTask(taskID) + + // manually monitor the Killed status... + go func() { + ticker := time.NewTicker(3 * time.Second) + defer ticker.Stop() + for { + select { + case <-ticker.C: + if atomic.LoadUint32(&e.ctx.GetSessionVars().Killed) == 1 { + bq.cancelTask(taskID) + return + } + case <-taskCtx.Done(): + return + } + } + }() + + progress, err := bq.acquireTask(taskCtx, taskID) + if err != nil { + return err + } + defer bq.releaseTask() + + e.info.execTime = types.CurrentTime(mysql.TypeDatetime) + glue := &tidbGlueSession{se: e.ctx, progress: progress, info: e.info} + + switch e.info.kind { + case ast.BRIEKindBackup: + err = handleBRIEError(task.RunBackup(taskCtx, glue, "Backup", e.backupCfg), ErrBRIEBackupFailed) + case ast.BRIEKindRestore: + err = handleBRIEError(task.RunRestore(taskCtx, glue, "Restore", e.restoreCfg), ErrBRIERestoreFailed) + default: + err = errors.Errorf("unsupported BRIE statement kind: %s", e.info.kind) + } + if err != nil { + return err + } + + req.AppendString(0, e.info.storage) + req.AppendUint64(1, e.info.archiveSize) + req.AppendUint64(2, e.info.backupTS) + req.AppendTime(3, e.info.queueTime) + req.AppendTime(4, e.info.execTime) + e.info = nil + return nil +} + +func handleBRIEError(err error, terror *terror.Error) error { + if err == nil { + return nil + } + return terror.GenWithStackByArgs(err) +} + +func (e *ShowExec) fetchShowBRIE(kind ast.BRIEKind) error { + globalBRIEQueue.tasks.Range(func(key, value interface{}) bool { + item := value.(*brieQueueItem) + if item.info.kind == kind { + item.progress.lock.Lock() + defer item.progress.lock.Unlock() + current := atomic.LoadInt64(&item.progress.current) + e.result.AppendString(0, item.info.storage) + e.result.AppendString(1, item.progress.cmd) + e.result.AppendFloat64(2, 100.0*float64(current)/float64(item.progress.total)) + e.result.AppendTime(3, item.info.queueTime) + e.result.AppendTime(4, item.info.execTime) + e.result.AppendNull(5) // FIXME: fill in finish time after keeping history. + e.result.AppendUint64(6, item.info.connID) + } + return true + }) + return nil +} + +type tidbGlueSession struct { + se sessionctx.Context + progress *brieTaskProgress + info *brieTaskInfo +} + +// BootstrapSession implements glue.Glue +func (gs *tidbGlueSession) GetDomain(store kv.Storage) (*domain.Domain, error) { + return domain.GetDomain(gs.se), nil +} + +// CreateSession implements glue.Glue +func (gs *tidbGlueSession) CreateSession(store kv.Storage) (glue.Session, error) { + return gs, nil +} + +// Execute implements glue.Session +func (gs *tidbGlueSession) Execute(ctx context.Context, sql string) error { + _, err := gs.se.(sqlexec.SQLExecutor).Execute(ctx, sql) + return err +} + +// CreateDatabase implements glue.Session +func (gs *tidbGlueSession) CreateDatabase(ctx context.Context, schema *model.DBInfo) error { + d := domain.GetDomain(gs.se).DDL() + schema = schema.Clone() + if len(schema.Charset) == 0 { + schema.Charset = mysql.DefaultCharset + } + return d.CreateSchemaWithInfo(gs.se, schema, ddl.OnExistIgnore, true) +} + +// CreateTable implements glue.Session +func (gs *tidbGlueSession) CreateTable(ctx context.Context, dbName model.CIStr, table *model.TableInfo) error { + d := domain.GetDomain(gs.se).DDL() + + // Clone() does not clone partitions yet :( + table = table.Clone() + if table.Partition != nil { + newPartition := *table.Partition + newPartition.Definitions = append([]model.PartitionDefinition{}, table.Partition.Definitions...) + table.Partition = &newPartition + } + + return d.CreateTableWithInfo(gs.se, dbName, table, ddl.OnExistIgnore, true) +} + +// Close implements glue.Session +func (gs *tidbGlueSession) Close() { +} + +// Open implements glue.Glue +func (gs *tidbGlueSession) Open(string, pd.SecurityOption) (kv.Storage, error) { + return gs.se.GetStore(), nil +} + +// OwnsStorage implements glue.Glue +func (gs *tidbGlueSession) OwnsStorage() bool { + return false +} + +// StartProgress implements glue.Glue +func (gs *tidbGlueSession) StartProgress(ctx context.Context, cmdName string, total int64, redirectLog bool) glue.Progress { + gs.progress.lock.Lock() + gs.progress.cmd = cmdName + gs.progress.total = total + atomic.StoreInt64(&gs.progress.current, 0) + gs.progress.lock.Unlock() + return gs.progress +} + +// Record implements glue.Glue +func (gs *tidbGlueSession) Record(name string, value uint64) { + switch name { + case "BackupTS": + gs.info.backupTS = value + case "Size": + gs.info.archiveSize = value + } +} diff --git a/executor/builder.go b/executor/builder.go index 108385da0875f..7ebc117d3bfa6 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -25,12 +25,12 @@ import ( "unsafe" "github.com/cznic/mathutil" - "github.com/cznic/sortutil" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/diagnosticspb" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor/aggfuncs" @@ -95,6 +95,9 @@ type MockPhysicalPlan interface { } func (b *executorBuilder) build(p plannercore.Plan) Executor { + if config.GetGlobalConfig().EnableCollectExecutionInfo && b.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl == nil { + b.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl = execdetails.NewRuntimeStatsColl() + } switch v := p.(type) { case nil: return nil @@ -166,6 +169,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor { return b.buildSimple(v) case *plannercore.Set: return b.buildSet(v) + case *plannercore.SetConfig: + return b.buildSetConfig(v) case *plannercore.PhysicalSort: return b.buildSort(v) case *plannercore.PhysicalTopN: @@ -669,6 +674,8 @@ func (b *executorBuilder) buildSimple(v *plannercore.Simple) Executor { return b.buildGrant(s) case *ast.RevokeStmt: return b.buildRevoke(s) + case *ast.BRIEStmt: + return b.buildBRIE(s, v.Schema()) } base := newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()) base.initCap = chunk.ZeroCapacity @@ -690,6 +697,13 @@ func (b *executorBuilder) buildSet(v *plannercore.Set) Executor { return e } +func (b *executorBuilder) buildSetConfig(v *plannercore.SetConfig) Executor { + return &SetConfigExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), + p: v, + } +} + func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { if v.SelectPlan != nil { // Try to update the forUpdateTS for insert/replace into select statements. @@ -879,7 +893,9 @@ func (b *executorBuilder) buildExplain(v *plannercore.Explain) Executor { explain: v, } if v.Analyze { - b.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl = execdetails.NewRuntimeStatsColl() + if b.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl == nil { + b.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl = execdetails.NewRuntimeStatsColl() + } explainExec.analyzeExec = b.build(v.TargetPlan) } return explainExec @@ -1005,7 +1021,7 @@ func (b *executorBuilder) buildMergeJoin(v *plannercore.PhysicalMergeJoin) Execu v.OtherConditions, retTypes(leftExec), retTypes(rightExec), - nil, + markChildrenUsedCols(v.Schema(), v.Children()[0].Schema(), v.Children()[1].Schema()), ), isOuterJoin: v.JoinType.IsOuterJoin(), desc: v.Desc, @@ -1456,7 +1472,11 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableConstraints), strings.ToLower(infoschema.TableTiFlashReplica), strings.ToLower(infoschema.TableTiDBServersInfo), - strings.ToLower(infoschema.TableTiKVStoreStatus): + strings.ToLower(infoschema.TableTiKVStoreStatus), + strings.ToLower(infoschema.TableStatementsSummary), + strings.ToLower(infoschema.TableStatementsSummaryHistory), + strings.ToLower(infoschema.ClusterTableStatementsSummary), + strings.ToLower(infoschema.ClusterTableStatementsSummaryHistory): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), table: v.Table, @@ -1487,7 +1507,7 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ExplainID()), t: tb, columns: v.Columns, - seekHandle: math.MinInt64, + seekHandle: kv.IntHandle(math.MinInt64), isVirtualTable: !tb.Type().IsNormalTable(), } } @@ -1992,6 +2012,10 @@ func (b *executorBuilder) constructDAGReq(plans []plannercore.PhysicalPlan, stor dagReq = &tipb.DAGRequest{} dagReq.TimeZoneName, dagReq.TimeZoneOffset = timeutil.Zone(b.ctx.GetSessionVars().Location()) sc := b.ctx.GetSessionVars().StmtCtx + if sc.RuntimeStatsColl != nil { + collExec := true + dagReq.CollectExecutionSummaries = &collExec + } dagReq.Flags = sc.PushDownFlags() if storeType == kv.TiFlash { var executors []*tipb.Executor @@ -2653,13 +2677,13 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte if err != nil { return nil, err } - handles := make([]int64, 0, len(lookUpContents)) + handles := make([]kv.Handle, 0, len(lookUpContents)) var isValidHandle bool for _, content := range lookUpContents { - handle := content.keys[0].GetInt64() + handle := kv.IntHandle(content.keys[0].GetInt64()) isValidHandle = true for _, key := range content.keys { - if handle != key.GetInt64() { + if handle.IntValue() != key.GetInt64() { isValidHandle = false break } @@ -2671,17 +2695,14 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte return builder.buildTableReaderFromHandles(ctx, e, handles) } -func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Context, e *TableReaderExecutor, handles []int64) (Executor, error) { - if e.runtimeStats != nil && e.dagPB.CollectExecutionSummaries == nil { - colExec := true - e.dagPB.CollectExecutionSummaries = &colExec - } +func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Context, e *TableReaderExecutor, handles []kv.Handle) (Executor, error) { startTS, err := builder.getSnapshotTS() if err != nil { return nil, err } - - sort.Sort(sortutil.Int64Slice(handles)) + sort.Slice(handles, func(i, j int) bool { + return handles[i].Compare(handles[j]) < 0 + }) var b distsql.RequestBuilder kvReq, err := b.SetTableHandles(getPhysicalTableID(e.table), handles). SetDAGRequest(e.dagPB). @@ -2897,7 +2918,7 @@ func (b *executorBuilder) buildShuffle(v *plannercore.PhysicalShuffle) *ShuffleE } // head & tail of physical plans' chain within "partition". - var head, tail plannercore.PhysicalPlan = v.Children()[0], v.Tail + var head, tail = v.Children()[0], v.Tail shuffle.workers = make([]*shuffleWorker, shuffle.concurrency) for i := range shuffle.workers { @@ -2985,7 +3006,7 @@ func newRowDecoder(ctx sessionctx.Context, schema *expression.Schema, tbl *model chk.AppendDatum(i, &d) return nil } - return rowcodec.NewChunkDecoder(reqCols, handleColID, defVal, ctx.GetSessionVars().TimeZone) + return rowcodec.NewChunkDecoder(reqCols, []int64{handleColID}, defVal, ctx.GetSessionVars().TimeZone) } func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan) Executor { @@ -3023,13 +3044,13 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan capacity = len(e.idxVals) } else { // `SELECT a FROM t WHERE a IN (1, 1, 2, 1, 2)` should not return duplicated rows - handles := make([]int64, 0, len(plan.Handles)) - dedup := make(map[int64]struct{}, len(plan.Handles)) + handles := make([]kv.Handle, 0, len(plan.Handles)) + dedup := kv.NewHandleMap() for _, handle := range plan.Handles { - if _, found := dedup[handle]; found { + if _, found := dedup.Get(handle); found { continue } - dedup[handle] = struct{}{} + dedup.Set(handle, true) handles = append(handles, handle) } e.handles = handles diff --git a/executor/chunk_size_control_test.go b/executor/chunk_size_control_test.go index 5c4f2990901af..2fbfeea0788b5 100644 --- a/executor/chunk_size_control_test.go +++ b/executor/chunk_size_control_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -67,7 +68,7 @@ func (c *testSlowClient) GetDelay(regionID uint64) time.Duration { } // manipulateCluster splits this cluster's region by splitKeys and returns regionIDs after split -func manipulateCluster(cluster *mocktikv.Cluster, splitKeys [][]byte) []uint64 { +func manipulateCluster(cluster cluster.Cluster, splitKeys [][]byte) []uint64 { if len(splitKeys) == 0 { return nil } @@ -113,7 +114,7 @@ type testChunkSizeControlKit struct { dom *domain.Domain tk *testkit.TestKit client *testSlowClient - cluster *mocktikv.Cluster + cluster cluster.Cluster } type testChunkSizeControlSuite struct { @@ -132,12 +133,13 @@ func (s *testChunkSizeControlSuite) SetUpSuite(c *C) { kit := new(testChunkSizeControlKit) s.m[name] = kit kit.client = &testSlowClient{regionDelay: make(map[uint64]time.Duration)} - kit.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(kit.cluster) + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + kit.cluster = cluster var err error kit.store, err = mockstore.NewMockTikvStore( - mockstore.WithCluster(kit.cluster), + mockstore.WithCluster(cluster), mockstore.WithHijackClient(func(c tikv.Client) tikv.Client { kit.client.Client = c return kit.client @@ -156,7 +158,7 @@ func (s *testChunkSizeControlSuite) SetUpSuite(c *C) { } func (s *testChunkSizeControlSuite) getKit(name string) ( - kv.Storage, *domain.Domain, *testkit.TestKit, *testSlowClient, *mocktikv.Cluster) { + kv.Storage, *domain.Domain, *testkit.TestKit, *testSlowClient, cluster.Cluster) { x := s.m[name] return x.store, x.dom, x.tk, x.client, x.cluster } diff --git a/executor/ddl.go b/executor/ddl.go index 9ebcf4584e9a0..ae4c79501ad23 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" @@ -458,22 +459,45 @@ func (e *DDLExec) getRecoverTableByJobID(s *ast.RecoverTableStmt, t *meta.Meta, return job, table.Meta(), nil } +// GetDropOrTruncateTableInfoFromJobs gets the dropped/truncated table information from DDL jobs, +// it will use the `start_ts` of DDL job as snapshot to get the dropped/truncated table information. +func GetDropOrTruncateTableInfoFromJobs(jobs []*model.Job, gcSafePoint uint64, dom *domain.Domain, fn func(*model.Job, *model.TableInfo) (bool, error)) (bool, error) { + for _, job := range jobs { + if job.Type != model.ActionDropTable && job.Type != model.ActionTruncateTable { + continue + } + // Check GC safe point for getting snapshot infoSchema. + err := gcutil.ValidateSnapshotWithGCSafePoint(job.StartTS, gcSafePoint) + if err != nil { + return false, err + } + // Get the snapshot infoSchema before drop table. + // TODO: only get the related database info and table info. + snapInfo, err := dom.GetSnapshotInfoSchema(job.StartTS) + if err != nil { + return false, err + } + // Get table meta from snapshot infoSchema. + table, ok := snapInfo.TableByID(job.TableID) + if !ok { + return false, infoschema.ErrTableNotExists.GenWithStackByArgs( + fmt.Sprintf("(Schema ID %d)", job.SchemaID), + fmt.Sprintf("(Table ID %d)", job.TableID), + ) + } + finish, err := fn(job, table.Meta()) + if err != nil || finish { + return finish, err + } + } + return false, nil +} + func (e *DDLExec) getRecoverTableByTableName(tableName *ast.TableName) (*model.Job, *model.TableInfo, error) { txn, err := e.ctx.Txn(true) if err != nil { return nil, nil, err } - t := meta.NewMeta(txn) - jobs, err := t.GetAllHistoryDDLJobs() - if err != nil { - return nil, nil, err - } - var job *model.Job - var tblInfo *model.TableInfo - gcSafePoint, err := gcutil.GetGCSafePoint(e.ctx) - if err != nil { - return nil, nil, err - } schemaName := tableName.Schema.L if schemaName == "" { schemaName = e.ctx.GetSessionVars().CurrentDB @@ -481,48 +505,44 @@ func (e *DDLExec) getRecoverTableByTableName(tableName *ast.TableName) (*model.J if schemaName == "" { return nil, nil, errors.Trace(core.ErrNoDB) } + gcSafePoint, err := gcutil.GetGCSafePoint(e.ctx) + if err != nil { + return nil, nil, err + } + var jobInfo *model.Job + var tableInfo *model.TableInfo dom := domain.GetDomain(e.ctx) - // TODO: only search recent `e.JobNum` DDL jobs. - for i := len(jobs) - 1; i > 0; i-- { - job = jobs[i] - if job.Type != model.ActionDropTable && job.Type != model.ActionTruncateTable { - continue - } - // Check GC safe point for getting snapshot infoSchema. - err = gcutil.ValidateSnapshotWithGCSafePoint(job.StartTS, gcSafePoint) - if err != nil { - return nil, nil, errors.Errorf("Can't find dropped/truncated table '%s' in GC safe point %s", tableName.Name.O, model.TSConvert2Time(gcSafePoint).String()) - } - // Get the snapshot infoSchema before drop table. - snapInfo, err := dom.GetSnapshotInfoSchema(job.StartTS) - if err != nil { - return nil, nil, err + handleJobAndTableInfo := func(job *model.Job, tblInfo *model.TableInfo) (bool, error) { + if tblInfo.Name.L != tableName.Name.L { + return false, nil } - // Get table meta from snapshot infoSchema. - table, ok := snapInfo.TableByID(job.TableID) + schema, ok := dom.InfoSchema().SchemaByID(job.SchemaID) if !ok { - return nil, nil, infoschema.ErrTableNotExists.GenWithStackByArgs( + return true, infoschema.ErrDatabaseNotExists.GenWithStackByArgs( fmt.Sprintf("(Schema ID %d)", job.SchemaID), - fmt.Sprintf("(Table ID %d)", job.TableID), ) } - if table.Meta().Name.L == tableName.Name.L { - schema, ok := dom.InfoSchema().SchemaByID(job.SchemaID) - if !ok { - return nil, nil, infoschema.ErrDatabaseNotExists.GenWithStackByArgs( - fmt.Sprintf("(Schema ID %d)", job.SchemaID), - ) - } - if schema.Name.L == schemaName { - tblInfo = table.Meta() - break - } + if schema.Name.L == schemaName { + tableInfo = tblInfo + jobInfo = job + return true, nil } + return false, nil + } + fn := func(jobs []*model.Job) (bool, error) { + return GetDropOrTruncateTableInfoFromJobs(jobs, gcSafePoint, dom, handleJobAndTableInfo) + } + err = admin.IterHistoryDDLJobs(txn, fn) + if err != nil { + if terror.ErrorEqual(variable.ErrSnapshotTooOld, err) { + return nil, nil, errors.Errorf("Can't find dropped/truncated table '%s' in GC safe point %s", tableName.Name.O, model.TSConvert2Time(gcSafePoint).String()) + } + return nil, nil, err } - if tblInfo == nil { + if tableInfo == nil || jobInfo == nil { return nil, nil, errors.Errorf("Can't find dropped/truncated table: %v in DDL history jobs", tableName.Name) } - return job, tblInfo, nil + return jobInfo, tableInfo, nil } func (e *DDLExec) executeFlashbackTable(s *ast.FlashBackTableStmt) error { diff --git a/executor/ddl_test.go b/executor/ddl_test.go index a44e7fb0501ca..3f9447778bc1e 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -278,6 +278,15 @@ func (s *testSuite6) TestCreateView(c *C) { tk.MustExec("drop view v_nested, v_nested2") } +func (s *testSuite6) TestIssue16250(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table if not exists t(a int)") + tk.MustExec("create view view_issue16250 as select * from t") + _, err := tk.Exec("truncate table view_issue16250") + c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.view_issue16250' doesn't exist") +} + func (s *testSuite6) TestCreateViewWithOverlongColName(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -333,6 +342,27 @@ func (s *testSuite6) TestCreateDropDatabase(c *C) { _, err = tk.Exec("drop database mysql") c.Assert(err, NotNil) + + tk.MustExec("create database charset_test charset ascii;") + tk.MustQuery("show create database charset_test;").Check(testutil.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET ascii */", + )) + tk.MustExec("drop database charset_test;") + tk.MustExec("create database charset_test charset binary;") + tk.MustQuery("show create database charset_test;").Check(testutil.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET binary */", + )) + tk.MustExec("drop database charset_test;") + tk.MustExec("create database charset_test collate utf8_general_ci;") + tk.MustQuery("show create database charset_test;").Check(testutil.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET utf8 COLLATE utf8_general_ci */", + )) + tk.MustExec("drop database charset_test;") + tk.MustExec("create database charset_test charset utf8 collate utf8_general_ci;") + tk.MustQuery("show create database charset_test;").Check(testutil.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET utf8 COLLATE utf8_general_ci */", + )) + tk.MustGetErrMsg("create database charset_test charset utf8 collate utf8mb4_unicode_ci;", "[ddl:1253]COLLATION 'utf8mb4_unicode_ci' is not valid for CHARACTER SET 'utf8'") } func (s *testSuite6) TestCreateDropTable(c *C) { @@ -392,7 +422,7 @@ func (s *testSuite6) TestAlterTableAddColumn(c *C) { tk.MustExec("insert into alter_test values(1)") tk.MustExec("alter table alter_test add column c2 timestamp default current_timestamp") time.Sleep(1 * time.Millisecond) - now := time.Now().Add(-time.Duration(1 * time.Millisecond)).Format(types.TimeFormat) + now := time.Now().Add(-1 * time.Millisecond).Format(types.TimeFormat) r, err := tk.Exec("select c2 from alter_test") c.Assert(err, IsNil) req := r.NewChunk() @@ -408,6 +438,10 @@ func (s *testSuite6) TestAlterTableAddColumn(c *C) { _, err = tk.Exec("alter table alter_view add column c4 varchar(50)") c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error()) tk.MustExec("drop view alter_view") + tk.MustExec("create sequence alter_seq") + _, err = tk.Exec("alter table alter_seq add column c int") + c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error()) + tk.MustExec("drop sequence alter_seq") } func (s *testSuite6) TestAlterTableAddColumns(c *C) { @@ -430,6 +464,10 @@ func (s *testSuite6) TestAlterTableAddColumns(c *C) { _, err = tk.Exec("alter table alter_view add column (c4 varchar(50), c5 varchar(50))") c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error()) tk.MustExec("drop view alter_view") + tk.MustExec("create sequence alter_seq") + _, err = tk.Exec("alter table alter_seq add column (c1 int, c2 varchar(10))") + c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error()) + tk.MustExec("drop sequence alter_seq") } func (s *testSuite6) TestAddNotNullColumnNoDefault(c *C) { @@ -479,6 +517,10 @@ func (s *testSuite6) TestAlterTableModifyColumn(c *C) { _, err = tk.Exec("alter table alter_view modify column c2 text") c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error()) tk.MustExec("drop view alter_view") + tk.MustExec("create sequence alter_seq") + _, err = tk.Exec("alter table alter_seq modify column c int") + c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error()) + tk.MustExec("drop sequence alter_seq") // test multiple collate modification in column. tk.MustExec("drop table if exists modify_column_multiple_collate") @@ -764,7 +806,7 @@ func (s *testSuite8) TestShardRowIDBits(c *C) { tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) c.Assert(err, IsNil) maxID := 1<<(64-15-1) - 1 - err = tbl.RebaseAutoID(tk.Se, int64(maxID)-1, false) + err = tbl.RebaseAutoID(tk.Se, int64(maxID)-1, false, autoid.RowIDAllocType) c.Assert(err, IsNil) tk.MustExec("insert into t1 values(1)") @@ -850,6 +892,144 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { c.Assert(err, NotNil) c.Assert(err.Error(), Equals, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error()) tk.MustExec("drop table t") + + // Test insert negative integers explicitly won't trigger rebase. + tk.MustExec("create table t (a bigint primary key auto_random(15), b int)") + for i := 1; i <= 100; i++ { + tk.MustExec("insert into t(b) values (?)", i) + tk.MustExec("insert into t(a, b) values (?, ?)", -i, i) + } + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test_auto_random_bits", "t") + c.Assert(err, IsNil) + // orderedHandles should be [-100, -99, ..., -2, -1, 1, 2, ..., 99, 100] + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 15, mysql.TypeLonglong) + size = int64(len(allHandles)) + for i := int64(0); i < 100; i++ { + c.Assert(orderedHandles[i], Equals, i-100) + } + for i := int64(100); i < size; i++ { + c.Assert(orderedHandles[i], Equals, i-99) + } + tk.MustExec("drop table t") +} + +func (s *testAutoRandomSuite) TestAutoRandomTableOption(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + // test table option is auto-random + testutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig() + + tk.MustExec("drop table if exists auto_random_table_option") + tk.MustExec("create table auto_random_table_option (a bigint auto_random(5) key) auto_random_base = 1000") + t, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto_random_table_option")) + c.Assert(err, IsNil) + c.Assert(t.Meta().AutoRandID, Equals, int64(1000)) + tk.MustExec("insert into auto_random_table_option values (),(),(),(),()") + allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Se, "test", "auto_random_table_option") + c.Assert(err, IsNil) + c.Assert(len(allHandles), Equals, 5) + // Test non-shard-bits part of auto random id is monotonic increasing and continuous. + orderedHandles := testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + size := int64(len(allHandles)) + for i := int64(0); i < size; i++ { + c.Assert(i+1000, Equals, orderedHandles[i]) + } + + tk.MustExec("drop table if exists alter_table_auto_random_option") + tk.MustExec("create table alter_table_auto_random_option (a bigint primary key auto_random(4), b int)") + t, err = domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) + c.Assert(err, IsNil) + c.Assert(t.Meta().AutoRandID, Equals, int64(0)) + tk.MustExec("insert into alter_table_auto_random_option values(),(),(),(),()") + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "alter_table_auto_random_option") + c.Assert(err, IsNil) + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + size = int64(len(allHandles)) + for i := int64(0); i < size; i++ { + c.Assert(orderedHandles[i], Equals, i+1) + } + tk.MustExec("delete from alter_table_auto_random_option") + + // alter table to change the auto_random option (it will dismiss the local allocator cache) + // To avoid the new base is in the range of local cache, which will leading the next + // value is not what we rebased, because the local cache is dropped, here we choose + // a quite big value to do this. + tk.MustExec("alter table alter_table_auto_random_option auto_random_base = 3000000") + t, err = domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) + c.Assert(err, IsNil) + c.Assert(t.Meta().AutoRandID, Equals, int64(3000000)) + tk.MustExec("insert into alter_table_auto_random_option values(),(),(),(),()") + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "alter_table_auto_random_option") + c.Assert(err, IsNil) + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + size = int64(len(allHandles)) + for i := int64(0); i < size; i++ { + c.Assert(orderedHandles[i], Equals, i+3000000) + } + tk.MustExec("drop table alter_table_auto_random_option") +} + +// Test filter different kind of allocators. +// In special ddl type, for example: +// 1: ActionRenameTable : it will abandon all the old allocators. +// 2: ActionRebaseAutoID : it will drop row-id-type allocator. +// 3: ActionModifyTableAutoIdCache : it will drop row-id-type allocator. +// 3: ActionRebaseAutoRandomBase : it will drop auto-rand-type allocator. +func (s *testAutoRandomSuite) TestFilterDifferentAllocators(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t1") + + testutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig() + + tk.MustExec("create table t(a bigint auto_random(5) key, b int auto_increment unique)") + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("1")) + allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t") + c.Assert(err, IsNil) + c.Assert(len(allHandles), Equals, 1) + orderedHandles := testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + c.Assert(orderedHandles[0], Equals, int64(1)) + tk.MustExec("delete from t") + + // Test rebase auto_increment. + tk.MustExec("alter table t auto_increment 3000000") + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("3000000")) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t") + c.Assert(err, IsNil) + c.Assert(len(allHandles), Equals, 1) + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + c.Assert(orderedHandles[0], Equals, int64(2)) + tk.MustExec("delete from t") + + // Test rebase auto_random. + tk.MustExec("alter table t auto_random_base 3000000") + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("3000001")) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t") + c.Assert(err, IsNil) + c.Assert(len(allHandles), Equals, 1) + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + c.Assert(orderedHandles[0], Equals, int64(3000000)) + tk.MustExec("delete from t") + + // Test rename table. + tk.MustExec("rename table t to t1") + tk.MustExec("insert into t1 values()") + res := tk.MustQuery("select b from t1") + strInt64, err := strconv.ParseInt(res.Rows()[0][0].(string), 10, 64) + c.Assert(err, IsNil) + c.Assert(strInt64, Greater, int64(3000002)) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t1") + c.Assert(err, IsNil) + c.Assert(len(allHandles), Equals, 1) + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + c.Assert(orderedHandles[0], Greater, int64(3000001)) } func (s *testSuite6) TestMaxHandleAddIndex(c *C) { @@ -914,12 +1094,12 @@ func (s *testSuite6) TestSetDDLReorgBatchSize(c *C) { tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '1'")) err = ddlutil.LoadDDLReorgVars(tk.Se) c.Assert(err, IsNil) - c.Assert(variable.GetDDLReorgBatchSize(), Equals, int32(variable.MinDDLReorgBatchSize)) + c.Assert(variable.GetDDLReorgBatchSize(), Equals, variable.MinDDLReorgBatchSize) tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_batch_size = %v", variable.MaxDDLReorgBatchSize+1)) tk.MustQuery("show warnings;").Check(testkit.Rows(fmt.Sprintf("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '%d'", variable.MaxDDLReorgBatchSize+1))) err = ddlutil.LoadDDLReorgVars(tk.Se) c.Assert(err, IsNil) - c.Assert(variable.GetDDLReorgBatchSize(), Equals, int32(variable.MaxDDLReorgBatchSize)) + c.Assert(variable.GetDDLReorgBatchSize(), Equals, variable.MaxDDLReorgBatchSize) _, err = tk.Exec("set @@global.tidb_ddl_reorg_batch_size = invalid_val") c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 100") diff --git a/executor/delete.go b/executor/delete.go index 69d0c5a9b4503..e1b736a65270f 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -17,6 +17,7 @@ import ( "context" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" @@ -53,7 +54,7 @@ func (e *DeleteExec) deleteOneRow(tbl table.Table, handleIndex int, isExtraHandl if isExtraHandle { end-- } - handle := row[handleIndex].GetInt64() + handle := kv.IntHandle(row[handleIndex].GetInt64()) err := e.removeRow(e.ctx, tbl, handle, row[:end]) if err != nil { return err @@ -122,11 +123,11 @@ func (e *DeleteExec) composeTblRowMap(tblRowMap tableRowMapType, colPosInfos []p // iterate all the joined tables, and got the copresonding rows in joinedRow. for _, info := range colPosInfos { if tblRowMap[info.TblID] == nil { - tblRowMap[info.TblID] = make(map[int64][]types.Datum) + tblRowMap[info.TblID] = kv.NewHandleMap() } - handle := joinedRow[info.HandleOrdinal].GetInt64() + handle := kv.IntHandle(joinedRow[info.HandleOrdinal].GetInt64()) // tblRowMap[info.TblID][handle] hold the row datas binding to this table and this handle. - tblRowMap[info.TblID][handle] = joinedRow[info.Start:info.End] + tblRowMap[info.TblID].Set(handle, joinedRow[info.Start:info.End]) } } @@ -161,18 +162,23 @@ func (e *DeleteExec) deleteMultiTablesByChunk(ctx context.Context) error { func (e *DeleteExec) removeRowsInTblRowMap(tblRowMap tableRowMapType) error { for id, rowMap := range tblRowMap { - for handle, data := range rowMap { - err := e.removeRow(e.ctx, e.tblID2Table[id], handle, data) + var err error + rowMap.Range(func(h kv.Handle, val interface{}) bool { + err = e.removeRow(e.ctx, e.tblID2Table[id], h, val.([]types.Datum)) if err != nil { - return err + return false } + return true + }) + if err != nil { + return err } } return nil } -func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h int64, data []types.Datum) error { +func (e *DeleteExec) removeRow(ctx sessionctx.Context, t table.Table, h kv.Handle, data []types.Datum) error { txnState, err := e.ctx.Txn(false) if err != nil { return err @@ -203,4 +209,4 @@ func (e *DeleteExec) Open(ctx context.Context) error { // tableRowMapType is a map for unique (Table, Row) pair. key is the tableID. // the key in map[int64]Row is the joined table handle, which represent a unique reference row. // the value in map[int64]Row is the deleting row. -type tableRowMapType map[int64]map[int64][]types.Datum +type tableRowMapType map[int64]*kv.HandleMap diff --git a/executor/distsql.go b/executor/distsql.go index a246890cac2c8..faff88d0ea90c 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -58,7 +58,7 @@ var LookupTableTaskChannelSize int32 = 50 // lookupTableTask is created from a partial result of an index request which // contains the handles in those index keys. type lookupTableTask struct { - handles []int64 + handles []kv.Handle rowIdx []int // rowIdx represents the handle index for every row. Only used when keep order. rows []chunk.Row idxRows *chunk.Chunk @@ -70,10 +70,10 @@ type lookupTableTask struct { // Without this map, the original index order might be lost. // The handles fetched from index is originally ordered by index, but we need handles to be ordered by itself // to do table request. - indexOrder map[int64]int + indexOrder *kv.HandleMap // duplicatedIndexOrder map likes indexOrder. But it's used when checkIndexValue isn't nil and // the same handle of index has multiple values. - duplicatedIndexOrder map[int64]int + duplicatedIndexOrder *kv.HandleMap // memUsage records the memory usage of this task calculated by table worker. // memTracker is used to release memUsage after task is done and unused. @@ -530,7 +530,7 @@ func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-cha } } -func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []int64) (Executor, error) { +func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []kv.Handle) (Executor, error) { tableReaderExec := &TableReaderExecutor{ baseExecutor: newBaseExecutor(e.ctx, e.schema, stringutil.MemoizeStr(func() string { return e.id.String() + "_tableReader" })), table: e.table, @@ -693,9 +693,9 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes } func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult, count uint64) ( - handles []int64, retChk *chunk.Chunk, scannedKeys uint64, err error) { + handles []kv.Handle, retChk *chunk.Chunk, scannedKeys uint64, err error) { handleOffset := chk.NumCols() - 1 - handles = make([]int64, 0, w.batchSize) + handles = make([]kv.Handle, 0, w.batchSize) // PushedLimit would always be nil for CheckIndex or CheckTable, we add this check just for insurance. checkLimit := (w.PushedLimit != nil) && (w.checkIndexValue == nil) for len(handles) < w.batchSize { @@ -729,7 +729,7 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, return handles, nil, scannedKeys, nil } } - h := chk.GetRow(i).GetInt64(handleOffset) + h := kv.IntHandle(chk.GetRow(i).GetInt64(handleOffset)) handles = append(handles, h) } if w.checkIndexValue != nil { @@ -746,26 +746,26 @@ func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, return handles, retChk, scannedKeys, nil } -func (w *indexWorker) buildTableTask(handles []int64, retChk *chunk.Chunk) *lookupTableTask { - var indexOrder map[int64]int - var duplicatedIndexOrder map[int64]int +func (w *indexWorker) buildTableTask(handles []kv.Handle, retChk *chunk.Chunk) *lookupTableTask { + var indexOrder *kv.HandleMap + var duplicatedIndexOrder *kv.HandleMap if w.keepOrder { // Save the index order. - indexOrder = make(map[int64]int, len(handles)) + indexOrder = kv.NewHandleMap() for i, h := range handles { - indexOrder[h] = i + indexOrder.Set(h, i) } } if w.checkIndexValue != nil { // Save the index order. - indexOrder = make(map[int64]int, len(handles)) - duplicatedIndexOrder = make(map[int64]int) + indexOrder = kv.NewHandleMap() + duplicatedIndexOrder = kv.NewHandleMap() for i, h := range handles { - if _, ok := indexOrder[h]; ok { - duplicatedIndexOrder[h] = i + if _, ok := indexOrder.Get(h); ok { + duplicatedIndexOrder.Set(h, i) } else { - indexOrder[h] = i + indexOrder.Set(h, i) } } } @@ -786,7 +786,7 @@ type tableWorker struct { idxLookup *IndexLookUpExecutor workCh <-chan *lookupTableTask finished <-chan struct{} - buildTblReader func(ctx context.Context, handles []int64) (Executor, error) + buildTblReader func(ctx context.Context, handles []kv.Handle) (Executor, error) keepOrder bool handleIdx int @@ -837,9 +837,13 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta return errors.Trace(err) } if chk.NumRows() == 0 { - for h := range task.indexOrder { - idxRow := task.idxRows.GetRow(task.indexOrder[h]) - return errors.Errorf("handle %#v, index:%#v != record:%#v", h, idxRow.GetDatum(0, w.idxColTps[0]), nil) + task.indexOrder.Range(func(h kv.Handle, val interface{}) bool { + idxRow := task.idxRows.GetRow(val.(int)) + err = errors.Errorf("handle %#v, index:%#v != record:%#v", h, idxRow.GetDatum(0, w.idxColTps[0]), nil) + return false + }) + if err != nil { + return err } break } @@ -847,12 +851,13 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta tblReaderExec := tableReader.(*TableReaderExecutor) iter := chunk.NewIterator4Chunk(chk) for row := iter.Begin(); row != iter.End(); row = iter.Next() { - handle := row.GetInt64(w.handleIdx) - offset, ok := task.indexOrder[handle] + handle := kv.IntHandle(row.GetInt64(w.handleIdx)) + v, ok := task.indexOrder.Get(handle) if !ok { - offset = task.duplicatedIndexOrder[handle] + v, _ = task.duplicatedIndexOrder.Get(handle) } - delete(task.indexOrder, handle) + offset, _ := v.(int) + task.indexOrder.Delete(handle) idxRow := task.idxRows.GetRow(offset) vals = vals[:0] for i, col := range w.idxTblCols { @@ -932,8 +937,9 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er if w.keepOrder { task.rowIdx = make([]int, 0, len(task.rows)) for i := range task.rows { - handle := task.rows[i].GetInt64(w.handleIdx) - task.rowIdx = append(task.rowIdx, task.indexOrder[handle]) + handle := kv.IntHandle(task.rows[i].GetInt64(w.handleIdx)) + rowIdx, _ := task.indexOrder.Get(handle) + task.rowIdx = append(task.rowIdx, rowIdx.(int)) } memUsage = int64(cap(task.rowIdx) * 4) task.memUsage += memUsage @@ -943,16 +949,16 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er if handleCnt != len(task.rows) { if len(w.idxLookup.tblPlans) == 1 { - obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) + obtainedHandlesMap := kv.NewHandleMap() for _, row := range task.rows { - handle := row.GetInt64(w.handleIdx) - obtainedHandlesMap[handle] = struct{}{} + handle := kv.IntHandle(row.GetInt64(w.handleIdx)) + obtainedHandlesMap.Set(handle, true) } logutil.Logger(ctx).Error("inconsistent index handles", zap.String("index", w.idxLookup.index.Name.O), zap.Int("index_cnt", handleCnt), zap.Int("table_cnt", len(task.rows)), - zap.Int64s("missing_handles", GetLackHandles(task.handles, obtainedHandlesMap)), - zap.Int64s("total_handles", task.handles)) + zap.String("missing_handles", fmt.Sprint(GetLackHandles(task.handles, obtainedHandlesMap))), + zap.String("total_handles", fmt.Sprint(task.handles))) // table scan in double read can never has conditions according to convertToIndexScan. // if this table scan has no condition, the number of rows it returns must equal to the length of handles. @@ -965,14 +971,14 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er } // GetLackHandles gets the handles in expectedHandles but not in obtainedHandlesMap. -func GetLackHandles(expectedHandles []int64, obtainedHandlesMap map[int64]struct{}) []int64 { - diffCnt := len(expectedHandles) - len(obtainedHandlesMap) - diffHandles := make([]int64, 0, diffCnt) +func GetLackHandles(expectedHandles []kv.Handle, obtainedHandlesMap *kv.HandleMap) []kv.Handle { + diffCnt := len(expectedHandles) - obtainedHandlesMap.Len() + diffHandles := make([]kv.Handle, 0, diffCnt) var cnt int for _, handle := range expectedHandles { isExist := false - if _, ok := obtainedHandlesMap[handle]; ok { - delete(obtainedHandlesMap, handle) + if _, ok := obtainedHandlesMap.Get(handle); ok { + obtainedHandlesMap.Delete(handle) isExist = true } if !isExist { diff --git a/executor/distsql_test.go b/executor/distsql_test.go index 156465ea7292d..250b45e3e1053 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -65,7 +66,7 @@ func (s *testSuite3) TestCopClientSend(c *C) { tblID := tbl.Meta().ID // Split the table. - s.cluster.SplitTable(s.mvccStore, tblID, 100) + s.cluster.SplitTable(tblID, 100) ctx := context.Background() // Send coprocessor request when the table split. @@ -78,8 +79,8 @@ func (s *testSuite3) TestCopClientSend(c *C) { rs.Close() // Split one region. - key := tablecodec.EncodeRowKeyWithHandle(tblID, 500) - region, _ := s.cluster.GetRegionByKey([]byte(key)) + key := tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(500)) + region, _ := s.cluster.GetRegionByKey(key) peerID := s.cluster.AllocID() s.cluster.Split(region.GetId(), s.cluster.AllocID(), key, []uint64{peerID}, peerID) @@ -104,25 +105,27 @@ func (s *testSuite3) TestCopClientSend(c *C) { } func (s *testSuite3) TestGetLackHandles(c *C) { - expectedHandles := []int64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10} - handlesMap := make(map[int64]struct{}) + expectedHandles := []kv.Handle{kv.IntHandle(1), kv.IntHandle(2), kv.IntHandle(3), kv.IntHandle(4), + kv.IntHandle(5), kv.IntHandle(6), kv.IntHandle(7), kv.IntHandle(8), kv.IntHandle(9), kv.IntHandle(10)} + handlesMap := kv.NewHandleMap() for _, h := range expectedHandles { - handlesMap[h] = struct{}{} + handlesMap.Set(h, true) } // expected handles 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 // obtained handles 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 diffHandles := executor.GetLackHandles(expectedHandles, handlesMap) c.Assert(diffHandles, HasLen, 0) - c.Assert(handlesMap, HasLen, 0) + c.Assert(handlesMap.Len(), Equals, 0) // expected handles 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 // obtained handles 2, 3, 4, 6, 7, 8, 9 - retHandles := []int64{2, 3, 4, 6, 7, 8, 9} - handlesMap = make(map[int64]struct{}) - handlesMap[1] = struct{}{} - handlesMap[5] = struct{}{} - handlesMap[10] = struct{}{} + retHandles := []kv.Handle{kv.IntHandle(2), kv.IntHandle(3), kv.IntHandle(4), kv.IntHandle(6), + kv.IntHandle(7), kv.IntHandle(8), kv.IntHandle(9)} + handlesMap = kv.NewHandleMap() + handlesMap.Set(kv.IntHandle(1), true) + handlesMap.Set(kv.IntHandle(5), true) + handlesMap.Set(kv.IntHandle(10), true) diffHandles = executor.GetLackHandles(expectedHandles, handlesMap) c.Assert(retHandles, DeepEquals, diffHandles) } @@ -214,7 +217,7 @@ func (s *testSuite3) TestInconsistentIndex(c *C) { for i := 0; i < 10; i++ { txn, err := s.store.Begin() c.Assert(err, IsNil) - _, err = idxOp.Create(ctx, txn, types.MakeDatums(i+10), int64(100+i)) + _, err = idxOp.Create(ctx, txn, types.MakeDatums(i+10), kv.IntHandle(100+i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -231,7 +234,7 @@ func (s *testSuite3) TestInconsistentIndex(c *C) { for i := 0; i < 10; i++ { txn, err := s.store.Begin() c.Assert(err, IsNil) - err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(i+10), int64(100+i)) + err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(i+10), kv.IntHandle(100+i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/executor/errors.go b/executor/errors.go index 56d4957d3b826..df9dbed68e2e4 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -42,4 +42,9 @@ var ( ErrRoleNotGranted = terror.ClassPrivilege.New(mysql.ErrRoleNotGranted, mysql.MySQLErrName[mysql.ErrRoleNotGranted]) ErrDeadlock = terror.ClassExecutor.New(mysql.ErrLockDeadlock, mysql.MySQLErrName[mysql.ErrLockDeadlock]) ErrQueryInterrupted = terror.ClassExecutor.New(mysql.ErrQueryInterrupted, mysql.MySQLErrName[mysql.ErrQueryInterrupted]) + + ErrBRIEBackupFailed = terror.ClassExecutor.New(mysql.ErrBRIEBackupFailed, mysql.MySQLErrName[mysql.ErrBRIEBackupFailed]) + ErrBRIERestoreFailed = terror.ClassExecutor.New(mysql.ErrBRIERestoreFailed, mysql.MySQLErrName[mysql.ErrBRIERestoreFailed]) + ErrBRIEImportFailed = terror.ClassExecutor.New(mysql.ErrBRIEImportFailed, mysql.MySQLErrName[mysql.ErrBRIEImportFailed]) + ErrBRIEExportFailed = terror.ClassExecutor.New(mysql.ErrBRIEExportFailed, mysql.MySQLErrName[mysql.ErrBRIEExportFailed]) ) diff --git a/executor/executor.go b/executor/executor.go index 3fd9de734eac6..b196f55a60e72 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -125,7 +125,7 @@ const ( ) func init() { - GlobalDiskUsageTracker = disk.NewTracker(stringutil.StringerStr("GlobalStorageLabel"), -1) + GlobalDiskUsageTracker = disk.NewGlobalTrcaker(stringutil.StringerStr("GlobalStorageLabel"), -1) action := &globalPanicOnExceed{} GlobalDiskUsageTracker.SetActionOnExceed(action) } @@ -295,21 +295,41 @@ func (e *ShowNextRowIDExec) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil { return err } - colName := model.ExtraHandleName - for _, col := range tbl.Meta().Columns { - if mysql.HasAutoIncrementFlag(col.Flag) { - colName = col.Name - break + tblMeta := tbl.Meta() + + allocators := tbl.Allocators(e.ctx) + for _, alloc := range allocators { + nextGlobalID, err := alloc.NextGlobalAutoID(tblMeta.ID) + if err != nil { + return err } + + var colName, idType string + switch alloc.GetType() { + case autoid.RowIDAllocType, autoid.AutoIncrementType: + idType = "AUTO_INCREMENT" + if col := tblMeta.GetAutoIncrementColInfo(); col != nil { + colName = col.Name.O + } else { + colName = model.ExtraHandleName.O + } + case autoid.AutoRandomType: + idType = "AUTO_RANDOM" + colName = tblMeta.GetPkName().O + case autoid.SequenceType: + idType = "SEQUENCE" + colName = "" + default: + return autoid.ErrInvalidAllocatorType.GenWithStackByArgs() + } + + req.AppendString(0, e.tblName.Schema.O) + req.AppendString(1, e.tblName.Name.O) + req.AppendString(2, colName) + req.AppendInt64(3, nextGlobalID) + req.AppendString(4, idType) } - nextGlobalID, err := tbl.Allocators(e.ctx).Get(autoid.RowIDAllocType).NextGlobalAutoID(tbl.Meta().ID) - if err != nil { - return err - } - req.AppendString(0, e.tblName.Schema.O) - req.AppendString(1, e.tblName.Name.O) - req.AppendString(2, colName.O) - req.AppendInt64(3, nextGlobalID) + e.done = true return nil } @@ -930,7 +950,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { } for _, col := range cols { - e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physicalID, row.GetInt64(col.Index))) + e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physicalID, kv.IntHandle(row.GetInt64(col.Index)))) } } } @@ -1086,7 +1106,7 @@ func init() { // While doing optimization in the plan package, we need to execute uncorrelated subquery, // but the plan package cannot import the executor package because of the dependency cycle. // So we assign a function implemented in the executor package to the plan package to avoid the dependency cycle. - plannercore.EvalSubquery = func(ctx context.Context, p plannercore.PhysicalPlan, is infoschema.InfoSchema, sctx sessionctx.Context) (rows [][]types.Datum, err error) { + plannercore.EvalSubqueryFirstRow = func(ctx context.Context, p plannercore.PhysicalPlan, is infoschema.InfoSchema, sctx sessionctx.Context) ([]types.Datum, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("executor.EvalSubQuery", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -1096,28 +1116,24 @@ func init() { e := &executorBuilder{is: is, ctx: sctx} exec := e.build(p) if e.err != nil { - return rows, e.err + return nil, e.err } - err = exec.Open(ctx) + err := exec.Open(ctx) defer terror.Call(exec.Close) if err != nil { - return rows, err + return nil, err } chk := newFirstChunk(exec) for { err = Next(ctx, exec, chk) if err != nil { - return rows, err + return nil, err } if chk.NumRows() == 0 { - return rows, nil - } - iter := chunk.NewIterator4Chunk(chk) - for r := iter.Begin(); r != iter.End(); r = iter.Next() { - row := r.GetDatumRow(retTypes(exec)) - rows = append(rows, row) + return nil, nil } - chk = chunk.Renew(chk, sctx.GetSessionVars().MaxChunkSize) + row := chk.GetRow(0).GetDatumRow(retTypes(exec)) + return row, err } } } @@ -1265,7 +1281,7 @@ type TableScanExec struct { baseExecutor t table.Table - seekHandle int64 + seekHandle kv.Handle iter kv.Iterator columns []*model.ColumnInfo isVirtualTable bool @@ -1290,7 +1306,7 @@ func (e *TableScanExec) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil { return err } - e.seekHandle = handle + 1 + e.seekHandle = handle.Next() mutableRow.SetDatums(row...) req.AppendRow(mutableRow.ToRow()) } @@ -1326,15 +1342,15 @@ func (e *TableScanExec) nextChunk4InfoSchema(ctx context.Context, chk *chunk.Chu } // nextHandle gets the unique handle for next row. -func (e *TableScanExec) nextHandle() (handle int64, found bool, err error) { +func (e *TableScanExec) nextHandle() (handle kv.Handle, found bool, err error) { handle, found, err = e.t.Seek(e.ctx, e.seekHandle) if err != nil || !found { - return 0, false, err + return nil, false, err } return handle, true, nil } -func (e *TableScanExec) getRow(handle int64) ([]types.Datum, error) { +func (e *TableScanExec) getRow(handle kv.Handle) ([]types.Datum, error) { columns := make([]*table.Column, e.schema.Len()) for i, v := range e.columns { columns[i] = table.ToColumn(v) @@ -1556,7 +1572,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars := ctx.GetSessionVars() // Detach the disk tracker for the previous stmtctx from GlobalDiskUsageTracker if vars.StmtCtx != nil && vars.StmtCtx.DiskTracker != nil { - vars.StmtCtx.DiskTracker.Detach() + vars.StmtCtx.DiskTracker.DetachFromGlobalTracker() } sc := &stmtctx.StatementContext{ TimeZone: vars.Location(), @@ -1564,7 +1580,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { DiskTracker: disk.NewTracker(stringutil.MemoizeStr(s.Text), -1), } if config.GetGlobalConfig().OOMUseTmpStorage && GlobalDiskUsageTracker != nil { - sc.DiskTracker.AttachTo(GlobalDiskUsageTracker) + sc.DiskTracker.AttachToGlobalTracker(GlobalDiskUsageTracker) } switch config.GetGlobalConfig().OOMAction { case config.OOMActionCancel: @@ -1590,6 +1606,9 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.InExplainStmt = true s = explainStmt.Stmt } + if _, ok := s.(*ast.ExplainForStmt); ok { + sc.InExplainStmt = true + } // TODO: Many same bool variables here. // We should set only two variables ( // IgnoreErr and StrictSQLMode) to avoid setting the same bool variables and @@ -1692,6 +1711,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars.SysErrorCount = errCount vars.SysWarningCount = warnCount vars.StmtCtx = sc + vars.PrevFoundInPlanCache = vars.FoundInPlanCache + vars.FoundInPlanCache = false return } diff --git a/executor/executor_required_rows_test.go b/executor/executor_required_rows_test.go index 56527396346a9..4709192299c7a 100644 --- a/executor/executor_required_rows_test.go +++ b/executor/executor_required_rows_test.go @@ -855,6 +855,10 @@ func (mp *mockPlan) GetExecutor() Executor { return mp.exec } +func (mp *mockPlan) Schema() *expression.Schema { + return mp.exec.Schema() +} + func (s *testExecSuite) TestVecGroupCheckerDATARACE(c *C) { ctx := mock.NewContext() diff --git a/executor/executor_test.go b/executor/executor_test.go index 67168cf725b66..bfe00245c2c53 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" @@ -65,7 +66,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" - "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mock" @@ -124,17 +124,16 @@ var _ = Suite(&testMemTableReaderSuite{&testClusterTableBase{}}) var _ = SerialSuites(&testFlushSuite{}) var _ = SerialSuites(&testAutoRandomSuite{&baseTestSuite{}}) var _ = SerialSuites(&testClusterTableSuite{}) -var _ = SerialSuites(&testSuite10{&baseTestSuite{}}) +var _ = SerialSuites(&testPrepareSerialSuite{&baseTestSuite{}}) type testSuite struct{ *baseTestSuite } type testSuiteP1 struct{ *baseTestSuite } type testSuiteP2 struct{ *baseTestSuite } type baseTestSuite struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - domain *domain.Domain + cluster cluster.Cluster + store kv.Storage + domain *domain.Domain *parser.Parser ctx *mock.Context } @@ -146,12 +145,15 @@ func (s *baseTestSuite) SetUpSuite(c *C) { flag.Lookup("mockTikv") useMockTikv := *mockTikv if useMockTikv { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) s.store = store @@ -397,7 +399,7 @@ func (s *testSuite3) TestAdmin(c *C) { tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("admin_test")) c.Assert(err, IsNil) c.Assert(tb.Indices(), HasLen, 1) - _, err = tb.Indices()[0].Create(mock.NewContext(), txn, types.MakeDatums(int64(10)), 1) + _, err = tb.Indices()[0].Create(mock.NewContext(), txn, types.MakeDatums(int64(10)), kv.IntHandle(1)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -449,7 +451,7 @@ func (s *testSuite3) TestAdmin(c *C) { m := meta.NewMeta(txn) startKey := meta.DDLJobHistoryKey(m, 0) endKey := meta.DDLJobHistoryKey(m, historyJobs[0].ID) - s.cluster.SplitKeys(s.mvccStore, startKey, endKey, int(historyJobs[0].ID/5)) + s.cluster.SplitKeys(startKey, endKey, int(historyJobs[0].ID/5)) historyJobs2, err := admin.GetHistoryDDLJobs(txn, 20) c.Assert(err, IsNil) @@ -3326,9 +3328,9 @@ func (s *testSuite) TestCheckIndex(c *C) { // table data (handle, data): (1, 10), (2, 20), (4, 40) txn, err = s.store.Begin() c.Assert(err, IsNil) - _, err = idx.Create(mockCtx, txn, types.MakeDatums(int64(30)), 3) + _, err = idx.Create(mockCtx, txn, types.MakeDatums(int64(30)), kv.IntHandle(3)) c.Assert(err, IsNil) - key := tablecodec.EncodeRowKey(tb.Meta().ID, codec.EncodeInt(nil, 4)) + key := tablecodec.EncodeRowKey(tb.Meta().ID, kv.IntHandle(4).Encoded()) setColValue(c, txn, key, types.NewDatum(int64(40))) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -3341,7 +3343,7 @@ func (s *testSuite) TestCheckIndex(c *C) { // table data (handle, data): (1, 10), (2, 20), (4, 40) txn, err = s.store.Begin() c.Assert(err, IsNil) - _, err = idx.Create(mockCtx, txn, types.MakeDatums(int64(40)), 4) + _, err = idx.Create(mockCtx, txn, types.MakeDatums(int64(40)), kv.IntHandle(4)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -3353,9 +3355,9 @@ func (s *testSuite) TestCheckIndex(c *C) { // table data (handle, data): (1, 10), (2, 20), (4, 40) txn, err = s.store.Begin() c.Assert(err, IsNil) - err = idx.Delete(sc, txn, types.MakeDatums(int64(30)), 3) + err = idx.Delete(sc, txn, types.MakeDatums(int64(30)), kv.IntHandle(3)) c.Assert(err, IsNil) - err = idx.Delete(sc, txn, types.MakeDatums(int64(20)), 2) + err = idx.Delete(sc, txn, types.MakeDatums(int64(20)), kv.IntHandle(2)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -4338,6 +4340,24 @@ func (s *testSuiteP2) TestShowTableRegion(c *C) { c.Assert(rows[1][1], Matches, fmt.Sprintf("t_%d_.*", partitionDef[1].ID)) c.Assert(rows[2][1], Matches, fmt.Sprintf("t_%d_.*", partitionDef[2].ID)) + // Test split partition region when add new partition. + tk.MustExec("drop table if exists partition_t;") + tk.MustExec(`create table partition_t (a int, b int,index(a)) PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (10), + PARTITION p1 VALUES LESS THAN (20), + PARTITION p2 VALUES LESS THAN (30));`) + tk.MustExec(`alter table partition_t add partition ( partition p3 values less than (40), partition p4 values less than (50) );`) + re = tk.MustQuery("show table partition_t regions") + rows = re.Rows() + c.Assert(len(rows), Equals, 5) + tbl = testGetTableByName(c, tk.Se, "test", "partition_t") + partitionDef = tbl.Meta().GetPartitionInfo().Definitions + c.Assert(rows[0][1], Matches, fmt.Sprintf("t_%d_.*", partitionDef[0].ID)) + c.Assert(rows[1][1], Matches, fmt.Sprintf("t_%d_.*", partitionDef[1].ID)) + c.Assert(rows[2][1], Matches, fmt.Sprintf("t_%d_.*", partitionDef[2].ID)) + c.Assert(rows[3][1], Matches, fmt.Sprintf("t_%d_.*", partitionDef[3].ID)) + c.Assert(rows[4][1], Matches, fmt.Sprintf("t_%d_.*", partitionDef[4].ID)) + // Test pre-split table region when create table. tk.MustExec("drop table if exists t_pre") tk.MustExec("create table t_pre (a int, b int) shard_row_id_bits = 2 pre_split_regions=2;") @@ -4565,7 +4585,7 @@ func setOOMAction(action string) { type testRecoverTable struct { store kv.Storage dom *domain.Domain - cluster *mocktikv.Cluster + cluster cluster.Cluster cli *regionProperityClient } @@ -4578,11 +4598,12 @@ func (s *testRecoverTable) SetUpSuite(c *C) { s.cli = cli var err error - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster s.store, err = mockstore.NewMockTikvStore( mockstore.WithHijackClient(hijackClient), - mockstore.WithCluster(s.cluster), + mockstore.WithCluster(cluster), ) c.Assert(err, IsNil) s.dom, err = session.BootstrapSession(s.store) @@ -4616,8 +4637,8 @@ func (s *testRecoverTable) TestRecoverTable(c *C) { // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. ddl.EmulatorGCDisable() gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - time.Duration(48*60*60*time.Second)).Format(gcTimeFormat) - timeAfterDrop := time.Now().Add(time.Duration(48 * 60 * 60 * time.Second)).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) + timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` @@ -4637,7 +4658,7 @@ func (s *testRecoverTable) TestRecoverTable(c *C) { // if GC enable is not exists in mysql.tidb _, err = tk.Exec("recover table t_recover") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:-1]current error msg: Cancelled DDL job, original error msg: can not get 'tikv_gc_enable'") + c.Assert(err.Error(), Equals, "[ddl:-1]can not get 'tikv_gc_enable'") err = gcutil.EnableGC(tk.Se) c.Assert(err, IsNil) @@ -4728,7 +4749,7 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. ddl.EmulatorGCDisable() gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - time.Duration(48*60*60*time.Second)).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` @@ -4831,8 +4852,10 @@ func (s *testSuiteP2) TestPointGetPreparedPlan(c *C) { pspk1Id, _, _, err := tk1.Se.PrepareStmt("select * from t where a = ?") c.Assert(err, IsNil) + tk1.Se.GetSessionVars().PreparedStmts[pspk1Id].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false pspk2Id, _, _, err := tk1.Se.PrepareStmt("select * from t where ? = a ") c.Assert(err, IsNil) + tk1.Se.GetSessionVars().PreparedStmts[pspk2Id].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false ctx := context.Background() // first time plan generated @@ -4872,6 +4895,7 @@ func (s *testSuiteP2) TestPointGetPreparedPlan(c *C) { // unique index psuk1Id, _, _, err := tk1.Se.PrepareStmt("select * from t where b = ? ") c.Assert(err, IsNil) + tk1.Se.GetSessionVars().PreparedStmts[psuk1Id].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false rs, err = tk1.Se.ExecutePreparedStmt(ctx, psuk1Id, []types.Datum{types.NewDatum(1)}) c.Assert(err, IsNil) @@ -4985,6 +5009,7 @@ func (s *testSuiteP2) TestPointGetPreparedPlanWithCommitMode(c *C) { pspk1Id, _, _, err := tk1.Se.PrepareStmt("select * from t where a = ?") c.Assert(err, IsNil) + tk1.Se.GetSessionVars().PreparedStmts[pspk1Id].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false ctx := context.Background() // first time plan generated @@ -5048,9 +5073,11 @@ func (s *testSuiteP2) TestPointUpdatePreparedPlan(c *C) { updateID1, pc, _, err := tk1.Se.PrepareStmt(`update t set c = c + 1 where a = ?`) c.Assert(err, IsNil) + tk1.Se.GetSessionVars().PreparedStmts[updateID1].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false c.Assert(pc, Equals, 1) updateID2, pc, _, err := tk1.Se.PrepareStmt(`update t set c = c + 2 where ? = a`) c.Assert(err, IsNil) + tk1.Se.GetSessionVars().PreparedStmts[updateID2].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false c.Assert(pc, Equals, 1) ctx := context.Background() @@ -5085,6 +5112,7 @@ func (s *testSuiteP2) TestPointUpdatePreparedPlan(c *C) { // unique index updUkID1, _, _, err := tk1.Se.PrepareStmt(`update t set c = c + 10 where b = ?`) c.Assert(err, IsNil) + tk1.Se.GetSessionVars().PreparedStmts[updUkID1].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false rs, err = tk1.Se.ExecutePreparedStmt(ctx, updUkID1, []types.Datum{types.NewDatum(3)}) c.Assert(rs, IsNil) c.Assert(err, IsNil) @@ -5149,6 +5177,7 @@ func (s *testSuiteP2) TestPointUpdatePreparedPlanWithCommitMode(c *C) { ctx := context.Background() updateID1, _, _, err := tk1.Se.PrepareStmt(`update t set c = c + 1 where a = ?`) + tk1.Se.GetSessionVars().PreparedStmts[updateID1].(*plannercore.CachedPrepareStmt).PreparedAst.UseCache = false c.Assert(err, IsNil) // first time plan generated @@ -5377,6 +5406,17 @@ func (s *testSuite1) TestIssue15718(c *C) { tk.MustExec("create table tt(a decimal(10, 0), b varchar(1), c time);") tk.MustExec("insert into tt values(0, '2', null), (7, null, '1122'), (NULL, 'w', null), (NULL, '2', '3344'), (NULL, NULL, '0'), (7, 'f', '33');") tk.MustQuery("select a and b as d, a or c as e from tt;").Check(testkit.Rows("0 ", " 1", "0 ", " 1", " ", "0 1")) + + tk.MustExec("drop table if exists tt;") + tk.MustExec("create table tt(a decimal(10, 0), b varchar(1), c time);") + tk.MustExec("insert into tt values(0, '2', '123'), (7, null, '1122'), (null, 'w', null);") + tk.MustQuery("select a and b as d, a, b from tt order by d limit 1;").Check(testkit.Rows(" 7 ")) + tk.MustQuery("select b or c as d, b, c from tt order by d limit 1;").Check(testkit.Rows(" w ")) + + tk.MustExec("drop table if exists t0;") + tk.MustExec("CREATE TABLE t0(c0 FLOAT);") + tk.MustExec("INSERT INTO t0(c0) VALUES (NULL);") + tk.MustQuery("SELECT * FROM t0 WHERE NOT(0 OR t0.c0);").Check(testkit.Rows()) } func (s *testSuite1) TestIssue15767(c *C) { @@ -5390,3 +5430,61 @@ func (s *testSuite1) TestIssue15767(c *C) { tk.MustExec("insert into t select * from t;") tk.MustQuery("select b, count(*) from ( select b from t order by a limit 20 offset 2) as s group by b order by b;").Check(testkit.Rows("a 6", "c 7", "s 7")) } + +func (s *testSuite1) TestIssue16025(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0;") + tk.MustExec("CREATE TABLE t0(c0 NUMERIC PRIMARY KEY);") + tk.MustExec("INSERT IGNORE INTO t0(c0) VALUES (NULL);") + tk.MustQuery("SELECT * FROM t0 WHERE c0;").Check(testkit.Rows()) +} + +func (s *testSuite1) TestIssue16854(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` enum('WAITING','PRINTED','STOCKUP','CHECKED','OUTSTOCK','PICKEDUP','WILLBACK','BACKED') DEFAULT NULL)") + tk.MustExec("insert into t values(1),(2),(3),(4),(5),(6),(7);") + for i := 0; i < 7; i++ { + tk.MustExec("insert into t select * from t;") + } + tk.MustExec("set @@tidb_max_chunk_size=100;") + tk.MustQuery("select distinct a from t order by a").Check(testkit.Rows("WAITING", "PRINTED", "STOCKUP", "CHECKED", "OUTSTOCK", "PICKEDUP", "WILLBACK")) + tk.MustExec("drop table t") + + tk.MustExec("CREATE TABLE `t` ( `a` set('WAITING','PRINTED','STOCKUP','CHECKED','OUTSTOCK','PICKEDUP','WILLBACK','BACKED') DEFAULT NULL)") + tk.MustExec("insert into t values(1),(2),(3),(4),(5),(6),(7);") + for i := 0; i < 7; i++ { + tk.MustExec("insert into t select * from t;") + } + tk.MustExec("set @@tidb_max_chunk_size=100;") + tk.MustQuery("select distinct a from t order by a").Check(testkit.Rows("WAITING", "PRINTED", "WAITING,PRINTED", "STOCKUP", "WAITING,STOCKUP", "PRINTED,STOCKUP", "WAITING,PRINTED,STOCKUP")) + tk.MustExec("drop table t") +} + +// this is from jira issue #5856 +func (s *testSuite1) TestInsertValuesWithSubQuery(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2(a int, b int, c int)") + defer tk.MustExec("drop table if exists t2") + + // should not reference upper scope + c.Assert(tk.ExecToErr("insert into t2 values (11, 8, (select not b))"), NotNil) + c.Assert(tk.ExecToErr("insert into t2 set a = 11, b = 8, c = (select b))"), NotNil) + + // subquery reference target table is allowed + tk.MustExec("insert into t2 values(1, 1, (select b from t2))") + tk.MustQuery("select * from t2").Check(testkit.Rows("1 1 ")) + tk.MustExec("insert into t2 set a = 1, b = 1, c = (select b+1 from t2)") + tk.MustQuery("select * from t2").Check(testkit.Rows("1 1 ", "1 1 2")) + + // insert using column should work normally + tk.MustExec("delete from t2") + tk.MustExec("insert into t2 values(2, 4, a)") + tk.MustQuery("select * from t2").Check(testkit.Rows("2 4 2")) + tk.MustExec("insert into t2 set a = 3, b = 5, c = b") + tk.MustQuery("select * from t2").Check(testkit.Rows("2 4 2", "3 5 5")) +} diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index dd58008e44461..2dbe6f1f17e3b 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -16,11 +16,14 @@ package executor_test import ( "crypto/tls" "fmt" + "math" . "github.com/pingcap/check" "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/testkit" ) @@ -129,3 +132,39 @@ func (s *testSuite) TestInspectionRuleTable(c *C) { tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection' and type='summary'")).Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES skip_request: true`)) } + +type testPrepareSerialSuite struct { + *baseTestSuite +} + +func (s *testPrepareSerialSuite) TestExplainForConnPlanCache(c *C) { + tk := testkit.NewTestKit(c, s.store) + orgEnable := core.PreparedPlanCacheEnabled() + defer func() { + core.SetPreparedPlanCache(orgEnable) + }() + core.SetPreparedPlanCache(true) + var err error + tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + c.Assert(err, IsNil) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + rows := tk.MustQuery("select connection_id()").Rows() + c.Assert(len(rows), Equals, 1) + connID := rows[0][0].(string) + tk.MustExec("prepare stmt from 'select * from t where a = ?'") + tk.MustExec("set @p0='1'") + tk.MustExec("execute stmt using @p0") + tkProcess := tk.Se.ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %s", connID)).Check(testkit.Rows( + "TableReader_7 8000.00 root data:Selection_6", + "└─Selection_6 8000.00 cop[tikv] eq(cast(test.t.a), 1)", + " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + )) +} diff --git a/executor/index_lookup_hash_join.go b/executor/index_lookup_hash_join.go index bccd28e0f4511..f3dbf8560c203 100644 --- a/executor/index_lookup_hash_join.go +++ b/executor/index_lookup_hash_join.go @@ -166,7 +166,7 @@ func (e *IndexNestedLoopHashJoin) startWorkers(ctx context.Context) { e.resultCh = nil } e.joinChkResourceCh = make([]chan *chunk.Chunk, concurrency) - for i := int(0); i < concurrency; i++ { + for i := 0; i < concurrency; i++ { if !e.keepOuterOrder { e.joinChkResourceCh[i] = make(chan *chunk.Chunk, 1) e.joinChkResourceCh[i] <- newFirstChunk(e) @@ -179,7 +179,7 @@ func (e *IndexNestedLoopHashJoin) startWorkers(ctx context.Context) { } e.workerWg.Add(concurrency) - for i := int(0); i < concurrency; i++ { + for i := 0; i < concurrency; i++ { workerID := i go util.WithRecovery(func() { e.newInnerWorker(innerCh, workerID).run(workerCtx, cancelFunc) }, e.finishJoinWorkers) } diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 2dbc5986c75b7..9a019b6e52813 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" - "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -189,6 +188,7 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, SetKeepOrder(false). SetStreaming(e.partialStreamings[workID]). SetFromSessionVars(e.ctx.GetSessionVars()). + SetMemTracker(e.memTracker). Build() if err != nil { return err @@ -347,8 +347,8 @@ func (w *partialTableWorker) fetchHandles(ctx context.Context, exitCh <-chan str } func (w *partialTableWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, handleOffset int) ( - handles []int64, retChk *chunk.Chunk, err error) { - handles = make([]int64, 0, w.batchSize) + handles []kv.Handle, retChk *chunk.Chunk, err error) { + handles = make([]kv.Handle, 0, w.batchSize) for len(handles) < w.batchSize { chk.SetRequiredRows(w.batchSize-len(handles), w.maxChunkSize) err = errors.Trace(w.tableReader.Next(ctx, chk)) @@ -359,7 +359,7 @@ func (w *partialTableWorker) extractTaskHandles(ctx context.Context, chk *chunk. return handles, retChk, nil } for i := 0; i < chk.NumRows(); i++ { - h := chk.GetRow(i).GetInt64(handleOffset) + h := kv.IntHandle(chk.GetRow(i).GetInt64(handleOffset)) handles = append(handles, h) } } @@ -370,14 +370,10 @@ func (w *partialTableWorker) extractTaskHandles(ctx context.Context, chk *chunk. return handles, retChk, nil } -func (w *partialTableWorker) buildTableTask(handles []int64, retChk *chunk.Chunk) *lookupTableTask { - var indexOrder map[int64]int - var duplicatedIndexOrder map[int64]int +func (w *partialTableWorker) buildTableTask(handles []kv.Handle, retChk *chunk.Chunk) *lookupTableTask { task := &lookupTableTask{ - handles: handles, - indexOrder: indexOrder, - duplicatedIndexOrder: duplicatedIndexOrder, - idxRows: retChk, + handles: handles, + idxRows: retChk, } task.doneCh = make(chan error, 1) @@ -408,16 +404,18 @@ func (e *IndexMergeReaderExecutor) startIndexMergeTableScanWorker(ctx context.Co } } -func (e *IndexMergeReaderExecutor) buildFinalTableReader(ctx context.Context, handles []int64) (Executor, error) { +func (e *IndexMergeReaderExecutor) buildFinalTableReader(ctx context.Context, handles []kv.Handle) (Executor, error) { tableReaderExec := &TableReaderExecutor{ baseExecutor: newBaseExecutor(e.ctx, e.schema, stringutil.MemoizeStr(func() string { return e.id.String() + "_tableReader" })), table: e.table, dagPB: e.tableRequest, startTS: e.startTS, streaming: e.tableStreaming, + columns: e.columns, feedback: statistics.NewQueryFeedback(0, nil, 0, false), plans: e.tblPlans, } + tableReaderExec.buildVirtualColumnInfo() tableReader, err := e.dataReaderBuilder.buildTableReaderFromHandles(ctx, tableReaderExec, handles) if err != nil { logutil.Logger(ctx).Error("build table reader from handles failed", zap.Error(err)) @@ -513,15 +511,15 @@ func (w *indexMergeProcessWorker) fetchLoop(ctx context.Context, fetchCh <-chan close(resultCh) }() - distinctHandles := set.NewInt64Set() + distinctHandles := kv.NewHandleMap() for task := range fetchCh { handles := task.handles - fhs := make([]int64, 0, 8) + fhs := make([]kv.Handle, 0, 8) for _, h := range handles { - if !distinctHandles.Exist(h) { + if _, ok := distinctHandles.Get(h); !ok { fhs = append(fhs, h) - distinctHandles.Insert(h) + distinctHandles.Set(h, true) } } if len(fhs) == 0 { @@ -594,9 +592,9 @@ func (w *partialIndexWorker) fetchHandles(ctx context.Context, result distsql.Se } func (w *partialIndexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult) ( - handles []int64, retChk *chunk.Chunk, err error) { + handles []kv.Handle, retChk *chunk.Chunk, err error) { handleOffset := chk.NumCols() - 1 - handles = make([]int64, 0, w.batchSize) + handles = make([]kv.Handle, 0, w.batchSize) for len(handles) < w.batchSize { chk.SetRequiredRows(w.batchSize-len(handles), w.maxChunkSize) err = errors.Trace(idxResult.Next(ctx, chk)) @@ -607,7 +605,7 @@ func (w *partialIndexWorker) extractTaskHandles(ctx context.Context, chk *chunk. return handles, retChk, nil } for i := 0; i < chk.NumRows(); i++ { - h := chk.GetRow(i).GetInt64(handleOffset) + h := kv.IntHandle(chk.GetRow(i).GetInt64(handleOffset)) handles = append(handles, h) } } @@ -618,14 +616,10 @@ func (w *partialIndexWorker) extractTaskHandles(ctx context.Context, chk *chunk. return handles, retChk, nil } -func (w *partialIndexWorker) buildTableTask(handles []int64, retChk *chunk.Chunk) *lookupTableTask { - var indexOrder map[int64]int - var duplicatedIndexOrder map[int64]int +func (w *partialIndexWorker) buildTableTask(handles []kv.Handle, retChk *chunk.Chunk) *lookupTableTask { task := &lookupTableTask{ - handles: handles, - indexOrder: indexOrder, - duplicatedIndexOrder: duplicatedIndexOrder, - idxRows: retChk, + handles: handles, + idxRows: retChk, } task.doneCh = make(chan error, 1) @@ -635,7 +629,7 @@ func (w *partialIndexWorker) buildTableTask(handles []int64, retChk *chunk.Chunk type indexMergeTableScanWorker struct { workCh <-chan *lookupTableTask finished <-chan struct{} - buildTblReader func(ctx context.Context, handles []int64) (Executor, error) + buildTblReader func(ctx context.Context, handles []kv.Handle) (Executor, error) tblPlans []plannercore.PhysicalPlan // memTracker is used to track the memory usage of this executor. diff --git a/executor/index_merge_reader_test.go b/executor/index_merge_reader_test.go index 4eb03d33fbff6..5a8e5ed3872d5 100644 --- a/executor/index_merge_reader_test.go +++ b/executor/index_merge_reader_test.go @@ -50,3 +50,14 @@ func (s *testSuite1) TestJoin(c *C) { tk.MustQuery("select /*+ use_index_merge(t1, t1a, t1b) */ sum(t1.a) from t1 join t2 on t1.id = t2.id where t1.a < 2 or t1.b > 4").Check(testkit.Rows("6")) tk.MustQuery("select /*+ use_index_merge(t1, t1a, t1b) */ sum(t1.a) from t1 join t2 on t1.id = t2.id where t1.a < 2 or t1.b > 5").Check(testkit.Rows("1")) } + +func (s *testSuite1) TestIndexMergeReaderAndGeneratedColumn(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t0") + tk.MustExec("CREATE TABLE t0(c0 INT AS (1), c1 INT PRIMARY KEY)") + tk.MustExec("INSERT INTO t0(c1) VALUES (0)") + tk.MustExec("CREATE INDEX i0 ON t0(c0)") + tk.MustQuery("SELECT /*+ USE_INDEX_MERGE(t0, i0, PRIMARY)*/ t0.c0 FROM t0 WHERE t0.c1 OR t0.c0").Check(testkit.Rows("1")) + tk.MustQuery("SELECT t0.c0 FROM t0 WHERE t0.c1 OR t0.c0").Check(testkit.Rows("1")) +} diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 6522a3bc2016a..04fe642f01a5b 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -45,6 +45,7 @@ import ( "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/stmtsummary" ) type memtableRetriever struct { @@ -126,6 +127,11 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex e.dataForTableTiFlashReplica(sctx, dbs) case infoschema.TableTiKVStoreStatus: err = e.dataForTiKVStoreStatus(sctx) + case infoschema.TableStatementsSummary, + infoschema.TableStatementsSummaryHistory, + infoschema.ClusterTableStatementsSummary, + infoschema.ClusterTableStatementsSummaryHistory: + err = e.setDataForStatementsSummary(sctx, e.table.Name.O) } if err != nil { return nil, err @@ -1253,6 +1259,10 @@ func (e *memtableRetriever) setNewTiKVRegionStatusCol(region *helper.RegionInfo, row[12].SetInt64(region.ReadBytes) row[13].SetInt64(region.ApproximateSize) row[14].SetInt64(region.ApproximateKeys) + if region.ReplicationStatus != nil { + row[15].SetString(region.ReplicationStatus.State, mysql.DefaultCollationName) + row[16].SetInt64(region.ReplicationStatus.StateID) + } e.rows = append(e.rows, row) } @@ -1592,3 +1602,29 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, s e.rows = rows return } + +func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, tableName string) error { + user := ctx.GetSessionVars().User + isSuper := false + if pm := privilege.GetPrivilegeManager(ctx); pm != nil { + isSuper = pm.RequestVerificationWithUser("", "", "", mysql.SuperPriv, user) + } + switch tableName { + case infoschema.TableStatementsSummary, + infoschema.ClusterTableStatementsSummary: + e.rows = stmtsummary.StmtSummaryByDigestMap.ToCurrentDatum(user, isSuper) + case infoschema.TableStatementsSummaryHistory, + infoschema.ClusterTableStatementsSummaryHistory: + e.rows = stmtsummary.StmtSummaryByDigestMap.ToHistoryDatum(user, isSuper) + } + switch tableName { + case infoschema.ClusterTableStatementsSummary, + infoschema.ClusterTableStatementsSummaryHistory: + rows, err := infoschema.AppendHostInfoToRows(e.rows) + if err != nil { + return err + } + e.rows = rows + } + return nil +} diff --git a/executor/insert.go b/executor/insert.go index ea6ed6e77150c..1e0b75fc26264 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -133,11 +133,11 @@ func prefetchConflictedOldRows(ctx context.Context, txn kv.Transaction, rows []t for _, r := range rows { for _, uk := range r.uniqueKeys { if val, found := values[string(uk.newKV.key)]; found { - handle, err := tables.DecodeHandle(val) + handle, err := tables.DecodeHandleInUniqueIndexValue(val) if err != nil { return err } - batchKeys = append(batchKeys, r.t.RecordKey(handle)) + batchKeys = append(batchKeys, r.t.RecordKey(kv.IntHandle(handle))) } } } @@ -159,13 +159,13 @@ func prefetchDataCache(ctx context.Context, txn kv.Transaction, rows []toBeCheck } // updateDupRow updates a duplicate row to a new row. -func (e *InsertExec) updateDupRow(ctx context.Context, txn kv.Transaction, row toBeCheckedRow, handle int64, onDuplicate []*expression.Assignment) error { +func (e *InsertExec) updateDupRow(ctx context.Context, txn kv.Transaction, row toBeCheckedRow, handle kv.Handle, onDuplicate []*expression.Assignment) error { oldRow, err := getOldRow(ctx, e.ctx, txn, row.t, handle, e.GenExprs) if err != nil { return err } - _, _, _, err = e.doDupRowUpdate(ctx, handle, oldRow, row.row, e.OnDuplicate) + err = e.doDupRowUpdate(ctx, handle, oldRow, row.row, e.OnDuplicate) if e.ctx.GetSessionVars().StmtCtx.DupKeyAsWarning && kv.ErrKeyExists.Equal(err) { e.ctx.GetSessionVars().StmtCtx.AppendWarning(err) return nil @@ -216,12 +216,12 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D } return err } - handle, err := tables.DecodeHandle(val) + handle, err := tables.DecodeHandleInUniqueIndexValue(val) if err != nil { return err } - err = e.updateDupRow(ctx, txn, r, handle, e.OnDuplicate) + err = e.updateDupRow(ctx, txn, r, kv.IntHandle(handle), e.OnDuplicate) if err != nil { if kv.IsErrNotFound(err) { // Data index inconsistent? A unique key provide the handle information, but the @@ -312,8 +312,8 @@ func (e *InsertExec) initEvalBuffer4Dup() { } // doDupRowUpdate updates the duplicate row. -func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle int64, oldRow []types.Datum, newRow []types.Datum, - cols []*expression.Assignment) ([]types.Datum, bool, int64, error) { +func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRow []types.Datum, newRow []types.Datum, + cols []*expression.Assignment) error { assignFlag := make([]bool, len(e.Table.WritableCols())) // See http://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values e.curInsertVals.SetDatums(newRow...) @@ -331,22 +331,22 @@ func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle int64, oldRow [] for _, col := range cols { val, err1 := col.Expr.Eval(e.evalBuffer4Dup.ToRow()) if err1 != nil { - return nil, false, 0, err1 + return err1 } e.row4Update[col.Col.Index], err1 = table.CastValue(e.ctx, val, col.Col.ToInfo()) if err1 != nil { - return nil, false, 0, err1 + return err1 } e.evalBuffer4Dup.SetDatum(col.Col.Index, e.row4Update[col.Col.Index]) assignFlag[col.Col.Index] = true } newData := e.row4Update[:len(oldRow)] - _, handleChanged, newHandle, err := updateRecord(ctx, e.ctx, handle, oldRow, newData, assignFlag, e.Table, true, e.memTracker) + _, err := updateRecord(ctx, e.ctx, handle, oldRow, newData, assignFlag, e.Table, true, e.memTracker) if err != nil { - return nil, false, 0, err + return err } - return newData, handleChanged, newHandle, nil + return nil } // setMessage sets info message(ERR_INSERT_INFO) generated by INSERT statement diff --git a/executor/insert_common.go b/executor/insert_common.go index aa0116e2f6c4d..99208a73cd221 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -682,7 +682,7 @@ func (e *InsertValues) lazyAdjustAutoIncrementDatum(ctx context.Context, rows [] } // Use the value if it's not null and not 0. if recordID != 0 { - err = e.Table.RebaseAutoID(e.ctx, recordID, true) + err = e.Table.RebaseAutoID(e.ctx, recordID, true, autoid.RowIDAllocType) if err != nil { return nil, err } @@ -769,7 +769,7 @@ func (e *InsertValues) adjustAutoIncrementDatum(ctx context.Context, d types.Dat } // Use the value if it's not null and not 0. if recordID != 0 { - err = e.Table.RebaseAutoID(e.ctx, recordID, true) + err = e.Table.RebaseAutoID(e.ctx, recordID, true, autoid.RowIDAllocType) if err != nil { return types.Datum{}, err } @@ -904,6 +904,9 @@ func (e *InsertValues) allocAutoRandomID(fieldType *types.FieldType) (int64, err } func (e *InsertValues) rebaseAutoRandomID(recordID int64, fieldType *types.FieldType) error { + if recordID < 0 { + return nil + } alloc := e.Table.Allocators(e.ctx).Get(autoid.AutoRandomType) tableInfo := e.Table.Meta() @@ -993,7 +996,7 @@ func (e *InsertValues) addRecordWithAutoIDHint(ctx context.Context, row []types. if !e.ctx.GetSessionVars().ConstraintCheckInPlace { txn.SetOption(kv.PresumeKeyNotExists, nil) } - var h int64 + var h kv.Handle if reserveAutoIDCount > 0 { h, err = e.Table.AddRecord(e.ctx, row, table.WithCtx(ctx), table.WithReserveAutoIDHint(reserveAutoIDCount)) } else { @@ -1006,5 +1009,5 @@ func (e *InsertValues) addRecordWithAutoIDHint(ctx context.Context, row []types. if e.lastInsertID != 0 { e.ctx.GetSessionVars().SetLastInsertID(e.lastInsertID) } - return h, nil + return h.IntValue(), nil } diff --git a/executor/insert_test.go b/executor/insert_test.go index ee0ca94c28f81..d6e912f9591b4 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -1132,3 +1132,14 @@ func (s *testSuite9) TestInsertErrorMsg(c *C) { c.Assert(err, NotNil) c.Assert(strings.Contains(err.Error(), "Incorrect datetime value: '2019-02-11 30:00:00' for column 'b' at row 1"), IsTrue, Commentf("%v", err)) } + +func (s *testSuite9) TestIssue16366(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test;`) + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t(c numeric primary key);`) + tk.MustExec("insert ignore into t values(null);") + _, err := tk.Exec(`insert into t values(0);`) + c.Assert(err, NotNil) + c.Assert(strings.Contains(err.Error(), "Duplicate entry '0' for key 'PRIMARY'"), IsTrue, Commentf("%v", err)) +} diff --git a/executor/inspection_result.go b/executor/inspection_result.go index 3ae1c7479245b..fc286a537899e 100644 --- a/executor/inspection_result.go +++ b/executor/inspection_result.go @@ -480,7 +480,6 @@ type inspectSwapMemoryUsed struct{} func (inspectSwapMemoryUsed) genSQL(timeRange plannercore.QueryTimeRange) string { sql := fmt.Sprintf("select instance, max(value) as max_used from metrics_schema.node_memory_swap_used %s group by instance having max_used > 0", timeRange.Condition()) - fmt.Println(sql) return sql } diff --git a/executor/inspection_summary.go b/executor/inspection_summary.go index 290ffa89b35b1..78742b73f12e3 100644 --- a/executor/inspection_summary.go +++ b/executor/inspection_summary.go @@ -64,7 +64,7 @@ var inspectionSummaryRules = map[string][]string{ "pd_start_tso_wait_duration", "tidb_transaction_local_latch_wait_duration", "tidb_transaction_duration", - "pd_handle_request_duration", + "pd_request_rpc_duration", "tidb_cop_duration", "tidb_batch_client_wait_duration", "tidb_batch_client_unavailable_duration", @@ -355,9 +355,9 @@ var inspectionSummaryRules = map[string][]string{ "pd_cluster_status", "pd_grpc_completed_commands_duration", "pd_grpc_completed_commands_rate", - "pd_handle_request_duration", - "pd_handle_request_ops", - "pd_handle_request_duration_avg", + "pd_request_rpc_duration", + "pd_request_rpc_ops", + "pd_request_rpc_duration_avg", "pd_handle_transactions_duration", "pd_handle_transactions_rate", "pd_hotspot_status", @@ -422,7 +422,7 @@ func (e *inspectionSummaryRetriever) retrieve(ctx context.Context, sctx sessionc condition := e.timeRange.Condition() var finalRows [][]types.Datum for rule, tables := range inspectionSummaryRules { - if !rules.exist(rule) { + if len(rules.set) != 0 && !rules.set.Exist(rule) { continue } for _, name := range tables { @@ -435,6 +435,7 @@ func (e *inspectionSummaryRetriever) retrieve(ctx context.Context, sctx sessionc continue } cols := def.Labels + comment := def.Comment cond := condition if def.Quantile > 0 { cols = append(cols, "quantile") @@ -493,6 +494,7 @@ func (e *inspectionSummaryRetriever) retrieve(ctx context.Context, sctx sessionc row.GetFloat64(0), // avg row.GetFloat64(1), // min row.GetFloat64(2), // max + comment, )) } } diff --git a/executor/inspection_summary_test.go b/executor/inspection_summary_test.go index 3f6275111cd71..f703a3c17bae4 100644 --- a/executor/inspection_summary_test.go +++ b/executor/inspection_summary_test.go @@ -101,11 +101,22 @@ func (s *inspectionSummarySuite) TestInspectionSummary(c *C) { result := tk.ResultSetToResultWithCtx(ctx, rs[0], Commentf("execute inspect SQL failed")) c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0), Commentf("unexpected warnings: %+v", tk.Se.GetSessionVars().StmtCtx.GetWarnings())) result.Check(testkit.Rows( - "query-summary tikv-0 tidb_query_duration Select 0.99 0 0 0", - "query-summary tikv-1 tidb_query_duration Update 0.99 2 1 3", - "query-summary tikv-2 tidb_query_duration Delete 0.99 5 5 5", - "query-summary tidb-0 tidb_qps Query, Error 1 1 1", - "query-summary tidb-0 tidb_qps Query, OK 0 0 0", - "query-summary tidb-1 tidb_qps Quit, Error 7 5 9", + "query-summary tikv-0 tidb_query_duration Select 0.99 0 0 0 The quantile of TiDB query durations(second)", + "query-summary tikv-1 tidb_query_duration Update 0.99 2 1 3 The quantile of TiDB query durations(second)", + "query-summary tikv-2 tidb_query_duration Delete 0.99 5 5 5 The quantile of TiDB query durations(second)", + "query-summary tidb-0 tidb_qps Query, Error 1 1 1 TiDB query processing numbers per second", + "query-summary tidb-0 tidb_qps Query, OK 0 0 0 TiDB query processing numbers per second", + "query-summary tidb-1 tidb_qps Quit, Error 7 5 9 TiDB query processing numbers per second", + )) + + // Test for select * from information_schema.inspection_summary without specify rules. + rs, err = tk.Se.Execute(ctx, "select * from information_schema.inspection_summary where metrics_name = 'tidb_qps'") + c.Assert(err, IsNil) + result = tk.ResultSetToResultWithCtx(ctx, rs[0], Commentf("execute inspect SQL failed")) + c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0), Commentf("unexpected warnings: %+v", tk.Se.GetSessionVars().StmtCtx.GetWarnings())) + result.Check(testkit.Rows( + "query-summary tidb-0 tidb_qps Query, Error 1 1 1 TiDB query processing numbers per second", + "query-summary tidb-0 tidb_qps Query, OK 0 0 0 TiDB query processing numbers per second", + "query-summary tidb-1 tidb_qps Quit, Error 7 5 9 TiDB query processing numbers per second", )) } diff --git a/executor/join.go b/executor/join.go index 4488e0a4dc60a..fb94c34b8bb36 100644 --- a/executor/join.go +++ b/executor/join.go @@ -139,7 +139,9 @@ func (e *HashJoinExec) Close() error { if e.runtimeStats != nil { concurrency := cap(e.joiners) e.runtimeStats.SetConcurrencyInfo("Concurrency", concurrency) - e.runtimeStats.SetAdditionalInfo(e.rowContainer.stat.String()) + if e.rowContainer != nil { + e.runtimeStats.SetAdditionalInfo(e.rowContainer.stat.String()) + } } err := e.baseExecutor.Close() return err diff --git a/executor/join_test.go b/executor/join_test.go index 8ce5592471402..8a7a13ceaaf56 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -1250,6 +1250,9 @@ func (s *testSuiteJoinSerial) TestIndexNestedLoopHashJoin(c *C) { // index hash join with semi join c.Assert(failpoint.Enable("github.com/pingcap/tidb/planner/core/MockOnlyEnableIndexHashJoin", "return(true)"), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/planner/core/MockOnlyEnableIndexHashJoin"), IsNil) + }() tk.MustExec("drop table t") tk.MustExec("CREATE TABLE `t` ( `l_orderkey` int(11) NOT NULL,`l_linenumber` int(11) NOT NULL,`l_partkey` int(11) DEFAULT NULL,`l_suppkey` int(11) DEFAULT NULL,PRIMARY KEY (`l_orderkey`,`l_linenumber`))") tk.MustExec(`insert into t values(0,0,0,0);`) @@ -1290,8 +1293,63 @@ func (s *testSuiteJoinSerial) TestIndexNestedLoopHashJoin(c *C) { " └─Selection_26(Probe) 3.00 cop[tikv] not(isnull(test.t.l_suppkey))", " └─TableRowIDScan_25 3.00 cop[tikv] table:l2 keep order:false")) tk.MustQuery("select count(*) from t l1 where exists ( select * from t l2 where l2.l_orderkey = l1.l_orderkey and l2.l_suppkey <> l1.l_suppkey );").Check(testkit.Rows("9")) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/planner/core/MockOnlyEnableIndexHashJoin"), IsNil) tk.MustExec("DROP TABLE IF EXISTS t, s") + + // issue16586 + tk.MustExec("use test;") + tk.MustExec("drop table if exists lineitem;") + tk.MustExec("drop table if exists orders;") + tk.MustExec("drop table if exists supplier;") + tk.MustExec("drop table if exists nation;") + tk.MustExec("CREATE TABLE `lineitem` (`l_orderkey` int(11) NOT NULL,`l_linenumber` int(11) NOT NULL,`l_partkey` int(11) DEFAULT NULL,`l_suppkey` int(11) DEFAULT NULL,PRIMARY KEY (`l_orderkey`,`l_linenumber`) );") + tk.MustExec("CREATE TABLE `supplier` ( `S_SUPPKEY` bigint(20) NOT NULL,`S_NATIONKEY` bigint(20) NOT NULL,PRIMARY KEY (`S_SUPPKEY`));") + tk.MustExec("CREATE TABLE `orders` (`O_ORDERKEY` bigint(20) NOT NULL,`O_ORDERSTATUS` char(1) NOT NULL,PRIMARY KEY (`O_ORDERKEY`));") + tk.MustExec("CREATE TABLE `nation` (`N_NATIONKEY` bigint(20) NOT NULL,`N_NAME` char(25) NOT NULL,PRIMARY KEY (`N_NATIONKEY`))") + tk.MustExec("insert into lineitem values(0,0,0,1)") + tk.MustExec("insert into lineitem values(0,1,1,1)") + tk.MustExec("insert into lineitem values(0,2,2,0)") + tk.MustExec("insert into lineitem values(0,3,3,3)") + tk.MustExec("insert into lineitem values(0,4,1,4)") + tk.MustExec("insert into supplier values(0, 4)") + tk.MustExec("insert into orders values(0, 'F')") + tk.MustExec("insert into nation values(0, 'EGYPT')") + tk.MustExec("insert into lineitem values(1,0,2,4)") + tk.MustExec("insert into lineitem values(1,1,1,0)") + tk.MustExec("insert into lineitem values(1,2,3,3)") + tk.MustExec("insert into lineitem values(1,3,1,0)") + tk.MustExec("insert into lineitem values(1,4,1,3)") + tk.MustExec("insert into supplier values(1, 1)") + tk.MustExec("insert into orders values(1, 'F')") + tk.MustExec("insert into nation values(1, 'EGYPT')") + tk.MustExec("insert into lineitem values(2,0,1,2)") + tk.MustExec("insert into lineitem values(2,1,3,4)") + tk.MustExec("insert into lineitem values(2,2,2,0)") + tk.MustExec("insert into lineitem values(2,3,3,1)") + tk.MustExec("insert into lineitem values(2,4,4,3)") + tk.MustExec("insert into supplier values(2, 3)") + tk.MustExec("insert into orders values(2, 'F')") + tk.MustExec("insert into nation values(2, 'EGYPT')") + tk.MustExec("insert into lineitem values(3,0,4,3)") + tk.MustExec("insert into lineitem values(3,1,4,3)") + tk.MustExec("insert into lineitem values(3,2,2,2)") + tk.MustExec("insert into lineitem values(3,3,0,0)") + tk.MustExec("insert into lineitem values(3,4,1,0)") + tk.MustExec("insert into supplier values(3, 1)") + tk.MustExec("insert into orders values(3, 'F')") + tk.MustExec("insert into nation values(3, 'EGYPT')") + tk.MustExec("insert into lineitem values(4,0,2,2)") + tk.MustExec("insert into lineitem values(4,1,4,2)") + tk.MustExec("insert into lineitem values(4,2,0,2)") + tk.MustExec("insert into lineitem values(4,3,0,1)") + tk.MustExec("insert into lineitem values(4,4,2,2)") + tk.MustExec("insert into supplier values(4, 4)") + tk.MustExec("insert into orders values(4, 'F')") + tk.MustExec("insert into nation values(4, 'EGYPT')") + tk.MustQuery("select count(*) from supplier, lineitem l1, orders, nation where s_suppkey = l1.l_suppkey and o_orderkey = l1.l_orderkey and o_orderstatus = 'F' and exists ( select * from lineitem l2 where l2.l_orderkey = l1.l_orderkey and l2.l_suppkey < l1.l_suppkey ) and s_nationkey = n_nationkey and n_name = 'EGYPT' order by l1.l_orderkey, l1.l_linenumber;").Check(testkit.Rows("18")) + tk.MustExec("drop table lineitem") + tk.MustExec("drop table nation") + tk.MustExec("drop table supplier") + tk.MustExec("drop table orders") } func (s *testSuiteJoin3) TestIssue15686(c *C) { diff --git a/executor/joiner.go b/executor/joiner.go index 7c8690896999e..8b2faea437154 100644 --- a/executor/joiner.go +++ b/executor/joiner.go @@ -341,7 +341,7 @@ func (j *baseJoiner) Clone() baseJoiner { if j.chk != nil { base.chk = j.chk.CopyConstruct() } else { - base.shallowRow = chunk.MutRow(j.shallowRow.ToRow()) + base.shallowRow = j.shallowRow.Clone() } if !j.defaultInner.IsEmpty() { base.defaultInner = j.defaultInner.CopyConstruct() diff --git a/executor/load_data.go b/executor/load_data.go index 74d286504bef8..90c09aa37f2db 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -632,14 +632,12 @@ func (w *fieldWriter) GetField() (bool, field) { } } else if ch == '\\' { // TODO: escape only support '\' - w.OutputBuf = append(w.OutputBuf, ch) + // When the escaped character is interpreted as if + // it was not escaped, backslash is ignored. flag, ch = w.getChar() if flag { - if ch == w.enclosedChar { - w.OutputBuf = append(w.OutputBuf, ch) - } else { - w.putback() - } + w.OutputBuf = append(w.OutputBuf, '\\') + w.OutputBuf = append(w.OutputBuf, ch) } } else { w.OutputBuf = append(w.OutputBuf, ch) diff --git a/executor/mem_reader.go b/executor/mem_reader.go index 0a696ffa694bd..27791ad88673b 100644 --- a/executor/mem_reader.go +++ b/executor/mem_reader.go @@ -58,7 +58,7 @@ func buildMemIndexReader(us *UnionScanExec, idxReader *IndexReaderExecutor) *mem kvRanges: kvRanges, desc: us.desc, conditions: us.conditions, - addedRows: make([][]types.Datum, 0, len(us.dirty.addedRows)), + addedRows: make([][]types.Datum, 0, us.dirty.addedRows.Len()), retFieldTypes: retTypes(us), outputOffset: outputOffset, belowHandleIndex: us.belowHandleIndex, @@ -178,7 +178,7 @@ func buildMemTableReader(us *UnionScanExec, tblReader *TableReaderExecutor) *mem }) } - rd := rowcodec.NewByteDecoder(colInfo, -1, nil, nil) + rd := rowcodec.NewByteDecoder(colInfo, []int64{-1}, nil, nil) return &memTableReader{ ctx: us.ctx, table: us.table.Meta(), @@ -186,7 +186,7 @@ func buildMemTableReader(us *UnionScanExec, tblReader *TableReaderExecutor) *mem kvRanges: tblReader.kvRanges, desc: us.desc, conditions: us.conditions, - addedRows: make([][]types.Datum, 0, len(us.dirty.addedRows)), + addedRows: make([][]types.Datum, 0, us.dirty.addedRows.Len()), retFieldTypes: retTypes(us), colIDs: colIDs, buffer: allocBuf{ @@ -233,7 +233,7 @@ func (m *memTableReader) decodeRecordKeyValue(key, value []byte) ([]types.Datum, } // decodeRowData uses to decode row data value. -func decodeRowData(ctx sessionctx.Context, tb *model.TableInfo, columns []*model.ColumnInfo, colIDs map[int64]int, handle int64, value []byte, buffer *allocBuf) ([]types.Datum, error) { +func decodeRowData(ctx sessionctx.Context, tb *model.TableInfo, columns []*model.ColumnInfo, colIDs map[int64]int, handle kv.Handle, value []byte, buffer *allocBuf) ([]types.Datum, error) { values, err := getRowData(ctx.GetSessionVars().StmtCtx, tb, columns, colIDs, handle, value, buffer) if err != nil { return nil, err @@ -251,7 +251,7 @@ func decodeRowData(ctx sessionctx.Context, tb *model.TableInfo, columns []*model } // getRowData decodes raw byte slice to row data. -func getRowData(ctx *stmtctx.StatementContext, tb *model.TableInfo, columns []*model.ColumnInfo, colIDs map[int64]int, handle int64, value []byte, buffer *allocBuf) ([][]byte, error) { +func getRowData(ctx *stmtctx.StatementContext, tb *model.TableInfo, columns []*model.ColumnInfo, colIDs map[int64]int, handle kv.Handle, value []byte, buffer *allocBuf) ([][]byte, error) { pkIsHandle := tb.PKIsHandle if rowcodec.IsNewFormat(value) { return buffer.rd.DecodeToBytes(colIDs, handle, value, buffer.handleBytes) @@ -271,9 +271,9 @@ func getRowData(ctx *stmtctx.StatementContext, tb *model.TableInfo, columns []*m var handleDatum types.Datum if mysql.HasUnsignedFlag(col.Flag) { // PK column is Unsigned. - handleDatum = types.NewUintDatum(uint64(handle)) + handleDatum = types.NewUintDatum(uint64(handle.IntValue())) } else { - handleDatum = types.NewIntDatum(handle) + handleDatum = types.NewIntDatum(handle.IntValue()) } handleData, err1 := codec.EncodeValue(ctx, buffer.handleBytes, handleDatum) if err1 != nil { @@ -335,7 +335,7 @@ func reverseDatumSlice(rows [][]types.Datum) { } } -func (m *memIndexReader) getMemRowsHandle() ([]int64, error) { +func (m *memIndexReader) getMemRowsHandle() ([]kv.Handle, error) { pkTp := types.NewFieldType(mysql.TypeLonglong) if m.table.PKIsHandle { for _, col := range m.table.Columns { @@ -345,13 +345,13 @@ func (m *memIndexReader) getMemRowsHandle() ([]int64, error) { } } } - handles := make([]int64, 0, m.addedRowsLen) + handles := make([]kv.Handle, 0, m.addedRowsLen) err := iterTxnMemBuffer(m.ctx, m.kvRanges, func(key, value []byte) error { handle, err := tablecodec.DecodeIndexHandle(key, value, len(m.index.Columns), pkTp) if err != nil { return err } - handles = append(handles, handle) + handles = append(handles, kv.IntHandle(handle)) return nil }) if err != nil { @@ -387,7 +387,7 @@ func buildMemIndexLookUpReader(us *UnionScanExec, idxLookUpReader *IndexLookUpEx table: idxLookUpReader.table.Meta(), kvRanges: kvRanges, desc: idxLookUpReader.desc, - addedRowsLen: len(us.dirty.addedRows), + addedRowsLen: us.dirty.addedRows.Len(), retFieldTypes: retTypes(us), outputOffset: outputOffset, belowHandleIndex: us.belowHandleIndex, @@ -429,7 +429,7 @@ func (m *memIndexLookUpReader) getMemRows() ([][]types.Datum, error) { Flen: col.Flen, }) } - rd := rowcodec.NewByteDecoder(colInfos, -1, nil, nil) + rd := rowcodec.NewByteDecoder(colInfos, []int64{-1}, nil, nil) memTblReader := &memTableReader{ ctx: m.ctx, table: m.table.Meta(), diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index 5941c5fcae0e4..1f5a88e516982 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -25,7 +25,6 @@ import ( "sync" "time" - "github.com/jeremywohl/flatten" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/diagnosticspb" @@ -143,7 +142,10 @@ func (e *clusterConfigRetriever) retrieve(_ context.Context, sctx sessionctx.Con return nil, nil } e.retrieved = true + return fetchClusterConfig(sctx, e.extractor.NodeTypes, e.extractor.Instances) +} +func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.StringSet) ([][]types.Datum, error) { type result struct { idx int rows [][]types.Datum @@ -159,7 +161,7 @@ func (e *clusterConfigRetriever) retrieve(_ context.Context, sctx sessionctx.Con if err != nil { return nil, err } - serversInfo = filterClusterServerInfo(serversInfo, e.extractor.NodeTypes, e.extractor.Instances) + serversInfo = filterClusterServerInfo(serversInfo, nodeTypes, nodeAddrs) var finalRows [][]types.Datum wg := sync.WaitGroup{} @@ -210,19 +212,26 @@ func (e *clusterConfigRetriever) retrieve(_ context.Context, sctx sessionctx.Con ch <- result{err: errors.Trace(err)} return } - data, err := flatten.Flatten(nested, "", flatten.DotStyle) - if err != nil { - ch <- result{err: errors.Trace(err)} - return - } - // Sorts by keys and make the result stable + data := config.FlattenConfigItems(nested) type item struct { key string val string } var items []item for key, val := range data { - items = append(items, item{key: key, val: fmt.Sprintf("%v", val)}) + var str string + switch val.(type) { + case string: // remove quotes + str = val.(string) + default: + tmp, err := json.Marshal(val) + if err != nil { + ch <- result{err: errors.Trace(err)} + return + } + str = string(tmp) + } + items = append(items, item{key: key, val: str}) } sort.Slice(items, func(i, j int) bool { return items[i].key < items[j].key }) var rows [][]types.Datum @@ -458,16 +467,12 @@ func (h *logResponseHeap) Pop() interface{} { } func (e *clusterLogRetriever) initialize(ctx context.Context, sctx sessionctx.Context) ([]chan logStreamResult, error) { - isFailpointTestModeSkipCheck := false serversInfo, err := infoschema.GetClusterServerInfo(sctx) failpoint.Inject("mockClusterLogServerInfo", func(val failpoint.Value) { // erase the error err = nil if s := val.(string); len(s) > 0 { serversInfo = parseFailpointServerInfo(s) - isFailpointTestModeSkipCheck = true - } else { - isFailpointTestModeSkipCheck = false } }) if err != nil { @@ -483,22 +488,22 @@ func (e *clusterLogRetriever) initialize(ctx context.Context, sctx sessionctx.Co levels = append(levels, sysutil.ParseLogLevel(l)) } - startTime, endTime := e.extractor.GetTimeRange(isFailpointTestModeSkipCheck) + // To avoid search log interface overload, the user should specify the time range, and at least one pattern + // in normally SQL. + if e.extractor.StartTime == 0 { + return nil, errors.New("denied to scan logs, please specified the start time, such as `time > '2020-01-01 00:00:00'`") + } + if e.extractor.EndTime == 0 { + return nil, errors.New("denied to scan logs, please specified the end time, such as `time < '2020-01-01 00:00:00'`") + } patterns := e.extractor.Patterns - - // There is no performance issue to check this variable because it will - // be eliminated in non-failpoint mode. - if !isFailpointTestModeSkipCheck { - // To avoid search log interface overload, the user should specify at least one pattern - // in normally SQL. (But in test mode we should relax this limitation) - if len(patterns) == 0 && len(levels) == 0 && len(instances) == 0 && len(nodeTypes) == 0 { - return nil, errors.New("denied to scan full logs (use `SELECT * FROM cluster_log WHERE message LIKE '%'` explicitly if intentionally)") - } + if len(patterns) == 0 && len(levels) == 0 && len(instances) == 0 && len(nodeTypes) == 0 { + return nil, errors.New("denied to scan full logs (use `SELECT * FROM cluster_log WHERE message LIKE '%'` explicitly if intentionally)") } req := &diagnosticspb.SearchLogRequest{ - StartTime: startTime, - EndTime: endTime, + StartTime: e.extractor.StartTime, + EndTime: e.extractor.EndTime, Levels: levels, Patterns: patterns, } diff --git a/executor/memtable_reader_test.go b/executor/memtable_reader_test.go index a5d5192dba3ed..830c8ca474382 100644 --- a/executor/memtable_reader_test.go +++ b/executor/memtable_reader_test.go @@ -570,28 +570,11 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { conditions []string expected [][]string }{ - // all log items - { - conditions: []string{}, - expected: fullLogs, - }, - { - conditions: []string{ - "time>='2019/08/26 06:18:13.011'", - }, - expected: fullLogs, - }, - { - conditions: []string{ - "time>='2019/08/26 06:18:13.011'", - "time<='2019/08/26 06:28:19.011'", - }, - expected: fullLogs, - }, { conditions: []string{ "time>='2019/08/26 06:18:13.011'", "time<='2099/08/26 06:28:19.011'", + "message like '%'", }, expected: fullLogs, }, @@ -599,6 +582,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { conditions: []string{ "time>='2019/08/26 06:19:13.011'", "time<='2019/08/26 06:21:15.011'", + "message like '%'", }, expected: [][]string{ {"2019/08/26 06:19:13.011", "tidb", "INFO", "[test log message tidb 1, foo]"}, @@ -617,6 +601,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { conditions: []string{ "time>='2019/08/26 06:19:13.011'", "time<='2019/08/26 06:21:15.011'", + "message like '%'", "type='pd'", }, expected: [][]string{ @@ -642,6 +627,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { "time>='2019/08/26 06:18:13.011'", "time>='2019/08/26 06:19:13.011'", "time='2019/08/26 06:19:14.011'", + "message like '%'", "type='pd'", }, expected: [][]string{ @@ -652,6 +638,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { conditions: []string{ "time>='2019/08/26 06:19:13.011'", "time<='2019/08/26 06:21:15.011'", + "message like '%'", "type='tidb'", }, expected: [][]string{ @@ -666,6 +653,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { conditions: []string{ "time>='2019/08/26 06:19:13.011'", "time<='2019/08/26 06:21:15.011'", + "message like '%'", "type='tikv'", }, expected: [][]string{ @@ -678,6 +666,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { conditions: []string{ "time>='2019/08/26 06:19:13.011'", "time<='2019/08/26 06:21:15.011'", + "message like '%'", fmt.Sprintf("instance='%s'", testServers["pd"].address), }, expected: [][]string{ @@ -689,6 +678,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { conditions: []string{ "time>='2019/08/26 06:19:13.011'", "time<='2019/08/26 06:21:15.011'", + "message like '%'", fmt.Sprintf("instance='%s'", testServers["tidb"].address), }, expected: [][]string{ @@ -703,6 +693,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { conditions: []string{ "time>='2019/08/26 06:19:13.011'", "time<='2019/08/26 06:21:15.011'", + "message like '%'", fmt.Sprintf("instance='%s'", testServers["tikv"].address), }, expected: [][]string{ @@ -715,6 +706,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { conditions: []string{ "time>='2019/08/26 06:19:13.011'", "time<='2019/08/26 06:21:15.011'", + "message like '%'", fmt.Sprintf("instance in ('%s', '%s')", testServers["pd"].address, testServers["tidb"].address), }, expected: [][]string{ @@ -729,6 +721,9 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, { conditions: []string{ + "time>='2019/08/26 06:18:13.011'", + "time<='2019/08/26 06:28:19.011'", + "message like '%'", "level='critical'", }, expected: [][]string{ @@ -742,6 +737,9 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, { conditions: []string{ + "time>='2019/08/26 06:18:13.011'", + "time<='2019/08/26 06:28:19.011'", + "message like '%'", "level='critical'", "type in ('pd', 'tikv')", }, @@ -754,6 +752,9 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, { conditions: []string{ + "time>='2019/08/26 06:18:13.011'", + "time<='2019/08/26 06:28:19.011'", + "message like '%'", "level='critical'", "(type='pd' or type='tikv')", }, @@ -766,6 +767,8 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, { conditions: []string{ + "time>='2019/08/26 06:18:13.011'", + "time<='2019/08/26 06:28:19.011'", "level='critical'", "message like '%pd%'", }, @@ -776,6 +779,8 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, { conditions: []string{ + "time>='2019/08/26 06:18:13.011'", + "time<='2019/08/26 06:28:19.011'", "level='critical'", "message like '%pd%'", "message like '%5%'", @@ -786,6 +791,8 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, { conditions: []string{ + "time>='2019/08/26 06:18:13.011'", + "time<='2019/08/26 06:28:19.011'", "level='critical'", "message like '%pd%'", "message like '%5%'", @@ -795,6 +802,8 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, { conditions: []string{ + "time>='2019/08/26 06:18:13.011'", + "time<='2019/08/26 06:28:19.011'", "level='critical'", "message regexp '.*pd.*'", }, @@ -805,6 +814,8 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, { conditions: []string{ + "time>='2019/08/26 06:18:13.011'", + "time<='2019/08/26 06:28:19.011'", "level='critical'", "message regexp '.*pd.*'", "message regexp '.*foo]$'", @@ -815,6 +826,8 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, { conditions: []string{ + "time>='2019/08/26 06:18:13.011'", + "time<='2019/08/26 06:28:19.011'", "level='critical'", "message regexp '.*pd.*'", "message regexp '.*5.*'", @@ -824,6 +837,8 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLog(c *C) { }, { conditions: []string{ + "time>='2019/08/26 06:18:13.011'", + "time<='2019/08/26 06:28:19.011'", "level='critical'", "(message regexp '.*pd.*' or message regexp '.*tidb.*')", }, @@ -875,9 +890,24 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLogError(c *C) { c.Assert(failpoint.Enable(fpName, `return("")`), IsNil) defer func() { c.Assert(failpoint.Disable(fpName), IsNil) }() + // Test without start time error. rs, err := tk.Exec("select * from information_schema.cluster_log") c.Assert(err, IsNil) _, err = session.ResultSetToStringSlice(context.Background(), tk.Se, rs) c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "denied to scan logs, please specified the start time, such as `time > '2020-01-01 00:00:00'`") + + // Test without end time error. + rs, err = tk.Exec("select * from information_schema.cluster_log where time>='2019/08/26 06:18:13.011'") + c.Assert(err, IsNil) + _, err = session.ResultSetToStringSlice(context.Background(), tk.Se, rs) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "denied to scan logs, please specified the end time, such as `time < '2020-01-01 00:00:00'`") + + // Test without specified message error. + rs, err = tk.Exec("select * from information_schema.cluster_log where time>='2019/08/26 06:18:13.011' and time<'2019/08/26 16:18:13.011'") + c.Assert(err, IsNil) + _, err = session.ResultSetToStringSlice(context.Background(), tk.Se, rs) + c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "denied to scan full logs (use `SELECT * FROM cluster_log WHERE message LIKE '%'` explicitly if intentionally)") } diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 08448b5ee1b23..47af02e2d4de3 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -16,12 +16,12 @@ package executor_test import ( "bytes" "fmt" - "github.com/pingcap/tidb/sessionctx/variable" "math/rand" "strings" . "github.com/pingcap/check" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/testkit" ) @@ -387,12 +387,11 @@ func (s *testSuite2) TestMergeJoin(c *C) { tk.MustExec("create table s(a int, b int)") tk.MustExec("insert into s values(1,1)") tk.MustQuery("explain select /*+ TIDB_SMJ(t, s) */ a in (select a from s where s.b >= t.b) from t").Check(testkit.Rows( - "Projection_7 10000.00 root Column#7", - "└─MergeJoin_8 10000.00 root left outer semi join, other cond:eq(test.t.a, test.s.a), ge(test.s.b, test.t.b)", - " ├─TableReader_12(Build) 10000.00 root data:TableFullScan_11", - " │ └─TableFullScan_11 10000.00 cop[tikv] table:s keep order:false, stats:pseudo", - " └─TableReader_10(Probe) 10000.00 root data:TableFullScan_9", - " └─TableFullScan_9 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "MergeJoin_8 10000.00 root left outer semi join, other cond:eq(test.t.a, test.s.a), ge(test.s.b, test.t.b)", + "├─TableReader_12(Build) 10000.00 root data:TableFullScan_11", + "│ └─TableFullScan_11 10000.00 cop[tikv] table:s keep order:false, stats:pseudo", + "└─TableReader_10(Probe) 10000.00 root data:TableFullScan_9", + " └─TableFullScan_9 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_SMJ(t, s) */ a in (select a from s where s.b >= t.b) from t").Check(testkit.Rows( "1", @@ -420,16 +419,15 @@ func (s *testSuite2) TestMergeJoin(c *C) { tk.MustExec("create table s (a int)") tk.MustExec("insert into s values (4), (1), (3), (2)") tk.MustQuery("explain select s1.a1 from (select a as a1 from s order by s.a desc) as s1 join (select a as a2 from s order by s.a desc) as s2 on s1.a1 = s2.a2 order by s1.a1 desc").Check(testkit.Rows( - "Projection_27 12487.50 root test.s.a", - "└─MergeJoin_28 12487.50 root inner join, left key:test.s.a, right key:test.s.a", - " ├─Sort_31(Build) 9990.00 root test.s.a:desc", - " │ └─TableReader_26 9990.00 root data:Selection_25", - " │ └─Selection_25 9990.00 cop[tikv] not(isnull(test.s.a))", - " │ └─TableFullScan_24 10000.00 cop[tikv] table:s keep order:false, stats:pseudo", - " └─Sort_29(Probe) 9990.00 root test.s.a:desc", - " └─TableReader_21 9990.00 root data:Selection_20", - " └─Selection_20 9990.00 cop[tikv] not(isnull(test.s.a))", - " └─TableFullScan_19 10000.00 cop[tikv] table:s keep order:false, stats:pseudo", + "MergeJoin_28 12487.50 root inner join, left key:test.s.a, right key:test.s.a", + "├─Sort_31(Build) 9990.00 root test.s.a:desc", + "│ └─TableReader_26 9990.00 root data:Selection_25", + "│ └─Selection_25 9990.00 cop[tikv] not(isnull(test.s.a))", + "│ └─TableFullScan_24 10000.00 cop[tikv] table:s keep order:false, stats:pseudo", + "└─Sort_29(Probe) 9990.00 root test.s.a:desc", + " └─TableReader_21 9990.00 root data:Selection_20", + " └─Selection_20 9990.00 cop[tikv] not(isnull(test.s.a))", + " └─TableFullScan_19 10000.00 cop[tikv] table:s keep order:false, stats:pseudo", )) tk.MustQuery("select s1.a1 from (select a as a1 from s order by s.a desc) as s1 join (select a as a2 from s order by s.a desc) as s2 on s1.a1 = s2.a2 order by s1.a1 desc").Check(testkit.Rows( "4", "3", "2", "1")) diff --git a/executor/oomtest/oom_test.go b/executor/oomtest/oom_test.go new file mode 100644 index 0000000000000..9f2e28e3efa1d --- /dev/null +++ b/executor/oomtest/oom_test.go @@ -0,0 +1,211 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Note: All the tests in this file will be executed sequentially. + +package oomtest + +import ( + "os" + "strings" + "sync" + + . "github.com/pingcap/check" + "github.com/pingcap/log" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testleak" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" +) + +var _ = SerialSuites(&testOOMSuite{}) + +type testOOMSuite struct { + store kv.Storage + do *domain.Domain + oom *oomCapturer +} + +func (s *testOOMSuite) SetUpSuite(c *C) { + testleak.BeforeTest() + s.registerHook() + var err error + s.store, err = mockstore.NewMockTikvStore() + c.Assert(err, IsNil) + domain.RunAutoAnalyze = false + s.do, err = session.BootstrapSession(s.store) + c.Assert(err, IsNil) + originCfg := config.GetGlobalConfig() + newConf := *originCfg + newConf.OOMAction = config.OOMActionLog + config.StoreGlobalConfig(&newConf) +} + +func (s *testOOMSuite) TearDownSuite(c *C) { + s.do.Close() + s.store.Close() +} + +func (s *testOOMSuite) registerHook() { + conf := &log.Config{Level: os.Getenv("log_level"), File: log.FileLogConfig{}} + _, r, _ := log.InitLogger(conf) + s.oom = &oomCapturer{r.Core, "", sync.Mutex{}} + lg := zap.New(s.oom) + log.ReplaceGlobals(lg, r) +} + +func (s *testOOMSuite) TestMemTracker4UpdateExec(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t_MemTracker4UpdateExec (id int, a int, b int, index idx_a(`a`))") + + log.SetLevel(zap.InfoLevel) + s.oom.tracker = "" + tk.MustExec("insert into t_MemTracker4UpdateExec values (1,1,1), (2,2,2), (3,3,3)") + c.Assert(s.oom.tracker, Equals, "") + tk.Se.GetSessionVars().MemQuotaQuery = 244 + tk.MustExec("update t_MemTracker4UpdateExec set a = 4") + c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") +} + +func (s *testOOMSuite) TestMemTracker4InsertAndReplaceExec(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t (id int, a int, b int, index idx_a(`a`))") + tk.MustExec("insert into t values (1,1,1), (2,2,2), (3,3,3)") + + tk.MustExec("create table t_MemTracker4InsertAndReplaceExec (id int, a int, b int, index idx_a(`a`))") + + log.SetLevel(zap.InfoLevel) + s.oom.tracker = "" + tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") + c.Assert(s.oom.tracker, Equals, "") + tk.Se.GetSessionVars().MemQuotaQuery = 1 + tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") + c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") + tk.Se.GetSessionVars().MemQuotaQuery = -1 + + s.oom.tracker = "" + tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") + c.Assert(s.oom.tracker, Equals, "") + tk.Se.GetSessionVars().MemQuotaQuery = 1 + tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") + c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") + tk.Se.GetSessionVars().MemQuotaQuery = -1 + + s.oom.tracker = "" + tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec select * from t") + c.Assert(s.oom.tracker, Equals, "") + tk.Se.GetSessionVars().MemQuotaQuery = 1 + tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec select * from t") + c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") + tk.Se.GetSessionVars().MemQuotaQuery = -1 + + s.oom.tracker = "" + tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec select * from t") + c.Assert(s.oom.tracker, Equals, "") + tk.Se.GetSessionVars().MemQuotaQuery = 1 + tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec select * from t") + c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") + tk.Se.GetSessionVars().MemQuotaQuery = -1 + + tk.Se.GetSessionVars().DMLBatchSize = 1 + tk.Se.GetSessionVars().BatchInsert = true + s.oom.tracker = "" + tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") + c.Assert(s.oom.tracker, Equals, "") + tk.Se.GetSessionVars().MemQuotaQuery = 1 + tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") + c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") + tk.Se.GetSessionVars().MemQuotaQuery = -1 + + s.oom.tracker = "" + tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") + c.Assert(s.oom.tracker, Equals, "") + tk.Se.GetSessionVars().MemQuotaQuery = 1 + tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") + c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") + tk.Se.GetSessionVars().MemQuotaQuery = -1 +} + +func (s *testOOMSuite) TestMemTracker4DeleteExec(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table MemTracker4DeleteExec1 (id int, a int, b int, index idx_a(`a`))") + tk.MustExec("create table MemTracker4DeleteExec2 (id int, a int, b int, index idx_a(`a`))") + + // delete from single table + log.SetLevel(zap.InfoLevel) + tk.MustExec("insert into MemTracker4DeleteExec1 values(1,1,1), (2,2,2), (3,3,3), (4,4,4), (5,5,5)") + s.oom.tracker = "" + tk.MustExec("delete from MemTracker4DeleteExec1") + c.Assert(s.oom.tracker, Equals, "") + tk.MustExec("insert into MemTracker4DeleteExec1 values (1,1,1), (2,2,2), (3,3,3)") + tk.Se.GetSessionVars().MemQuotaQuery = 1 + tk.MustExec("delete from MemTracker4DeleteExec1") + c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") + + // delete from multiple table + tk.Se.GetSessionVars().MemQuotaQuery = 100000 + tk.MustExec("insert into MemTracker4DeleteExec1 values(1,1,1)") + tk.MustExec("insert into MemTracker4DeleteExec2 values(1,1,1)") + s.oom.tracker = "" + tk.MustExec("delete MemTracker4DeleteExec1, MemTracker4DeleteExec2 from MemTracker4DeleteExec1 join MemTracker4DeleteExec2 on MemTracker4DeleteExec1.a=MemTracker4DeleteExec2.a") + c.Assert(s.oom.tracker, Equals, "") + tk.MustExec("insert into MemTracker4DeleteExec1 values(1,1,1)") + tk.MustExec("insert into MemTracker4DeleteExec2 values(1,1,1)") + s.oom.tracker = "" + tk.Se.GetSessionVars().MemQuotaQuery = 10000 + tk.MustExec("delete MemTracker4DeleteExec1, MemTracker4DeleteExec2 from MemTracker4DeleteExec1 join MemTracker4DeleteExec2 on MemTracker4DeleteExec1.a=MemTracker4DeleteExec2.a") + c.Assert(s.oom.tracker, Equals, "expensive_query during bootstrap phase") +} + +type oomCapturer struct { + zapcore.Core + tracker string + mu sync.Mutex +} + +func (h *oomCapturer) Write(entry zapcore.Entry, fields []zapcore.Field) error { + if strings.Contains(entry.Message, "memory exceeds quota") { + err, _ := fields[0].Interface.(error) + str := err.Error() + begin := strings.Index(str, "8001]") + if begin == -1 { + panic("begin not found") + } + end := strings.Index(str, " holds") + if end == -1 { + panic("end not found") + } + h.tracker = str[begin+len("8001]") : end] + return nil + } + + h.mu.Lock() + h.tracker = entry.Message + h.mu.Unlock() + return nil +} + +func (h *oomCapturer) Check(e zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { + if h.Enabled(e.Level) { + return ce.AddCore(e, h) + } + return ce +} diff --git a/executor/point_get.go b/executor/point_get.go index ad2596dfa85eb..1b2f7e06d4001 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/table" @@ -59,7 +60,7 @@ type PointGetExecutor struct { baseExecutor tblInfo *model.TableInfo - handle int64 + handle kv.Handle idxInfo *model.IndexInfo partInfo *model.PartitionDefinition idxKey kv.Key @@ -161,19 +162,24 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { } e.handleVal, err = e.get(ctx, e.idxKey) - if err != nil && !kv.ErrNotExist.Equal(err) { - return err + if err != nil { + if !kv.ErrNotExist.Equal(err) { + return err + } } if len(e.handleVal) == 0 { + // handle is not found, try lock the index key if isolation level is not read consistency if e.ctx.GetSessionVars().IsPessimisticReadConsistency() { return nil } return e.lockKeyIfNeeded(ctx, e.idxKey) } - e.handle, err = tables.DecodeHandle(e.handleVal) + var iv int64 + iv, err = tables.DecodeHandleInUniqueIndexValue(e.handleVal) if err != nil { return err } + e.handle = kv.IntHandle(iv) // The injection is used to simulate following scenario: // 1. Session A create a point get query but pause before second time `GET` kv from backend @@ -217,13 +223,13 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { func (e *PointGetExecutor) getAndLock(ctx context.Context, key kv.Key) (val []byte, err error) { if e.ctx.GetSessionVars().IsPessimisticReadConsistency() { - // Lock the got keys in RC isolation. + // Only Lock the exist keys in RC isolation. val, err = e.get(ctx, key) - if kv.ErrNotExist.Equal(err) { - return nil, nil - } if err != nil { - return nil, err + if !kv.ErrNotExist.Equal(err) { + return nil, err + } + return nil, nil } err = e.lockKeyIfNeeded(ctx, key) if err != nil { @@ -237,8 +243,11 @@ func (e *PointGetExecutor) getAndLock(ctx context.Context, key kv.Key) (val []by return nil, err } val, err = e.get(ctx, key) - if err != nil && !kv.ErrNotExist.Equal(err) { - return nil, err + if err != nil { + if !kv.ErrNotExist.Equal(err) { + return nil, err + } + return nil, nil } return val, nil } @@ -267,21 +276,24 @@ func (e *PointGetExecutor) lockKeyIfNeeded(ctx context.Context, key []byte) erro return nil } -func (e *PointGetExecutor) get(ctx context.Context, key kv.Key) (val []byte, err error) { +// get will first try to get from txn buffer, then check the pessimistic lock cache, +// then the store. Kv.ErrNotExist will be returned if key is not found +func (e *PointGetExecutor) get(ctx context.Context, key kv.Key) ([]byte, error) { if e.txn.Valid() && !e.txn.IsReadOnly() { // We cannot use txn.Get directly here because the snapshot in txn and the snapshot of e.snapshot may be // different for pessimistic transaction. - val, err = e.txn.GetMemBuffer().Get(ctx, key) + val, err := e.txn.GetMemBuffer().Get(ctx, key) if err == nil { return val, err } if !kv.IsErrNotFound(err) { return nil, err } + // key does not exist in mem buffer, check the lock cache var ok bool val, ok = e.ctx.GetSessionVars().TxnCtx.GetKeyInPessimisticLockCache(key) if ok { - return + return val, nil } // fallthrough to snapshot get. } @@ -314,14 +326,14 @@ func encodeIndexKey(e *baseExecutor, tblInfo *model.TableInfo, idxInfo *model.In return tablecodec.EncodeIndexSeekKey(tID, idxInfo.ID, encodedIdxVals), nil } -func decodeRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle int64, rowVal []byte, chk *chunk.Chunk, rd *rowcodec.ChunkDecoder) error { +func decodeRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle kv.Handle, rowVal []byte, chk *chunk.Chunk, rd *rowcodec.ChunkDecoder) error { if rowcodec.IsNewFormat(rowVal) { return rd.DecodeToChunk(rowVal, handle, chk) } return decodeOldRowValToChunk(e, tblInfo, handle, rowVal, chk) } -func decodeOldRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle int64, rowVal []byte, chk *chunk.Chunk) error { +func decodeOldRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle kv.Handle, rowVal []byte, chk *chunk.Chunk) error { colID2CutPos := make(map[int64]int, e.schema.Len()) for _, col := range e.schema.Columns { if _, ok := colID2CutPos[col.ID]; !ok { @@ -342,12 +354,7 @@ func decodeOldRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle in chk.AppendNull(i) continue } - if tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.RetType.Flag) { - chk.AppendInt64(i, handle) - continue - } - if col.ID == model.ExtraHandleID { - chk.AppendInt64(i, handle) + if tryDecodeFromHandle(tblInfo, i, col, handle, chk) { continue } cutPos := colID2CutPos[col.ID] @@ -368,6 +375,18 @@ func decodeOldRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle in return nil } +func tryDecodeFromHandle(tblInfo *model.TableInfo, i int, col *expression.Column, handle kv.Handle, chk *chunk.Chunk) bool { + if tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.RetType.Flag) { + chk.AppendInt64(i, handle.IntValue()) + return true + } + if col.ID == model.ExtraHandleID { + chk.AppendInt64(i, handle.IntValue()) + return true + } + return false +} + func getColInfoByID(tbl *model.TableInfo, colID int64) *model.ColumnInfo { for _, col := range tbl.Columns { if col.ID == colID { diff --git a/executor/point_get_test.go b/executor/point_get_test.go index 73bca25ea6f98..c94da7dde4c2a 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -141,10 +141,10 @@ func (s *testPointGetSuite) TestPointGetCharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows(`a b`)) tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "a ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = "";`).Check(testkit.Rows(` `)) tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = " ";`).Check(testkit.Rows()) } @@ -157,7 +157,7 @@ func (s *testPointGetSuite) TestPointGetAliasTableCharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select * from t tmp where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t tmp where a = "aab";`).Check(testkit.Rows()) tk.MustExec(`truncate table t;`) tk.MustExec(`insert into t values("a ", "b ");`) @@ -165,7 +165,7 @@ func (s *testPointGetSuite) TestPointGetAliasTableCharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t tmp where a = "a";`).Check(testkit.Rows(`a b`)) tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) // Test CHAR BINARY. tk.MustExec(`drop table if exists t;`) @@ -176,10 +176,10 @@ func (s *testPointGetSuite) TestPointGetAliasTableCharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t tmp where a = "a";`).Check(testkit.Rows(`a b`)) tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t tmp where a = "";`).Check(testkit.Rows(` `)) tk.MustPointGet(`select * from t tmp where a = " ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t tmp where a = " ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t tmp where a = " ";`).Check(testkit.Rows()) // Test both wildcard and column name exist in select field list tk.MustExec(`set @@sql_mode="";`) @@ -192,9 +192,9 @@ func (s *testPointGetSuite) TestPointGetAliasTableCharPK(c *C) { tk.MustPointGet(`select tmp.* from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) tk.MustPointGet(`select tmp.a, tmp.b from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) tk.MustPointGet(`select tmp.*, tmp.a, tmp.b from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb aa bb`)) - tk.MustPointGet(`select tmp.* from t tmp where a = "aab";`).Check(testkit.Rows()) - tk.MustPointGet(`select tmp.a, tmp.b from t tmp where a = "aab";`).Check(testkit.Rows()) - tk.MustPointGet(`select tmp.*, tmp.a, tmp.b from t tmp where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select tmp.* from t tmp where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select tmp.a, tmp.b from t tmp where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select tmp.*, tmp.a, tmp.b from t tmp where a = "aab";`).Check(testkit.Rows()) // Test using table alias in where clause tk.MustPointGet(`select * from t tmp where tmp.a = "aa";`).Check(testkit.Rows(`aa bb`)) @@ -265,7 +265,7 @@ func (s *testPointGetSuite) TestPointGetVarcharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select * from t where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "aab";`).Check(testkit.Rows()) tk.MustExec(`truncate table t;`) tk.MustExec(`insert into t values("a ", "b ");`) @@ -273,7 +273,7 @@ func (s *testPointGetSuite) TestPointGetVarcharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows(`a b `)) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "a ";`).Check(testkit.Rows()) // // Test VARCHAR BINARY. tk.MustExec(`drop table if exists t;`) @@ -284,10 +284,10 @@ func (s *testPointGetSuite) TestPointGetVarcharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows(`a b `)) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "a ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows(` `)) - tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = " ";`).Check(testkit.Rows()) } @@ -368,6 +368,20 @@ func (s *testPointGetSuite) TestIndexLookupBinary(c *C) { } +func (s *testPointGetSuite) TestOverflowOrTruncated(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t6 (id bigint, a bigint, primary key(id), unique key(a));") + tk.MustExec("insert into t6 values(9223372036854775807, 9223372036854775807);") + tk.MustExec("insert into t6 values(1, 1);") + var nilVal []string + // for unique key + tk.MustQuery("select * from t6 where a = 9223372036854775808").Check(testkit.Rows(nilVal...)) + tk.MustQuery("select * from t6 where a = '1.123'").Check(testkit.Rows(nilVal...)) + // for primary key + tk.MustQuery("select * from t6 where id = 9223372036854775808").Check(testkit.Rows(nilVal...)) + tk.MustQuery("select * from t6 where id = '1.123'").Check(testkit.Rows(nilVal...)) +} + func (s *testPointGetSuite) TestIssue10448(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -500,9 +514,9 @@ func (s *testPointGetSuite) TestReturnValues(c *C) { txnCtx := tk.Se.GetSessionVars().TxnCtx val, ok := txnCtx.GetKeyInPessimisticLockCache(pk) c.Assert(ok, IsTrue) - handle, err := tables.DecodeHandle(val) + handle, err := tables.DecodeHandleInUniqueIndexValue(val) c.Assert(err, IsNil) - rowKey := tablecodec.EncodeRowKeyWithHandle(tid, handle) + rowKey := tablecodec.EncodeRowKeyWithHandle(tid, kv.IntHandle(handle)) _, ok = txnCtx.GetKeyInPessimisticLockCache(rowKey) c.Assert(ok, IsTrue) tk.MustExec("rollback") diff --git a/executor/prepared.go b/executor/prepared.go index 102f3f4a3f883..de869a875dba8 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -178,7 +178,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { SchemaVersion: e.is.SchemaMetaVersion(), } - prepared.UseCache = plannercore.PreparedPlanCacheEnabled() && plannercore.Cacheable(stmt) + prepared.UseCache = plannercore.PreparedPlanCacheEnabled() && plannercore.Cacheable(stmt, e.is) // We try to build the real statement of preparedStmt. for i := range prepared.Params { diff --git a/executor/reload_expr_pushdown_blacklist.go b/executor/reload_expr_pushdown_blacklist.go index f99210ec350ba..553ee6d9ccab9 100644 --- a/executor/reload_expr_pushdown_blacklist.go +++ b/executor/reload_expr_pushdown_blacklist.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" @@ -36,18 +37,33 @@ func (e *ReloadExprPushdownBlacklistExec) Next(ctx context.Context, _ *chunk.Chu // LoadExprPushdownBlacklist loads the latest data from table mysql.expr_pushdown_blacklist. func LoadExprPushdownBlacklist(ctx sessionctx.Context) (err error) { - sql := "select HIGH_PRIORITY name from mysql.expr_pushdown_blacklist" + sql := "select HIGH_PRIORITY name, store_type from mysql.expr_pushdown_blacklist" rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(sql) if err != nil { return err } - newBlacklist := make(map[string]struct{}, len(rows)) + newBlacklist := make(map[string]uint32, len(rows)) for _, row := range rows { name := strings.ToLower(row.GetString(0)) + storeTypeString := strings.ToLower(row.GetString(1)) if alias, ok := funcName2Alias[name]; ok { name = alias } - newBlacklist[name] = struct{}{} + var value uint32 = 0 + if val, ok := newBlacklist[name]; ok { + value = val + } + storeTypes := strings.Split(storeTypeString, ",") + for _, typeString := range storeTypes { + if typeString == kv.TiDB.Name() { + value |= 1 << kv.TiDB + } else if typeString == kv.TiFlash.Name() { + value |= 1 << kv.TiFlash + } else if typeString == kv.TiKV.Name() { + value |= 1 << kv.TiKV + } + } + newBlacklist[name] = value } expression.DefaultExprPushDownBlacklist.Store(newBlacklist) return nil diff --git a/executor/replace.go b/executor/replace.go index 766839ba4bca8..c6626c92d12ed 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -58,15 +58,15 @@ func (e *ReplaceExec) Open(ctx context.Context) error { // removeRow removes the duplicate row and cleanup its keys in the key-value map, // but if the to-be-removed row equals to the to-be-added row, no remove or add things to do. -func (e *ReplaceExec) removeRow(ctx context.Context, txn kv.Transaction, handle int64, r toBeCheckedRow) (bool, error) { +func (e *ReplaceExec) removeRow(ctx context.Context, txn kv.Transaction, handle kv.Handle, r toBeCheckedRow) (bool, error) { newRow := r.row oldRow, err := getOldRow(ctx, e.ctx, txn, r.t, handle, e.GenExprs) if err != nil { logutil.BgLogger().Error("get old row failed when replace", - zap.Int64("handle", handle), + zap.String("handle", handle.String()), zap.String("toBeInsertedRow", types.DatumsToStrNoErr(r.row))) if kv.IsErrNotFound(err) { - err = errors.NotFoundf("can not be duplicated row, due to old row not found. handle %d", handle) + err = errors.NotFoundf("can not be duplicated row, due to old row not found. handle %s", handle) } return false, err } @@ -155,11 +155,11 @@ func (e *ReplaceExec) removeIndexRow(ctx context.Context, txn kv.Transaction, r return false, false, err } - handle, err := tables.DecodeHandle(val) + handle, err := tables.DecodeHandleInUniqueIndexValue(val) if err != nil { return false, true, err } - rowUnchanged, err := e.removeRow(ctx, txn, handle, r) + rowUnchanged, err := e.removeRow(ctx, txn, kv.IntHandle(handle), r) if err != nil { return false, true, err } diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 86cf58c503bf1..8deb2966b440e 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/log" "github.com/pingcap/parser" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -52,6 +51,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -60,10 +60,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" ) func TestT(t *testing.T) { @@ -75,13 +72,11 @@ func TestT(t *testing.T) { var _ = SerialSuites(&seqTestSuite{}) var _ = SerialSuites(&seqTestSuite1{}) -var _ = SerialSuites(&testOOMSuite{}) type seqTestSuite struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - domain *domain.Domain + cluster cluster.Cluster + store kv.Storage + domain *domain.Domain *parser.Parser ctx *mock.Context } @@ -93,12 +88,15 @@ func (s *seqTestSuite) SetUpSuite(c *C) { flag.Lookup("mockTikv") useMockTikv := *mockTikv if useMockTikv { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) s.store = store @@ -137,7 +135,7 @@ func (s *seqTestSuite) TestEarlyClose(c *C) { tblID := tbl.Meta().ID // Split the table. - s.cluster.SplitTable(s.mvccStore, tblID, N/2) + s.cluster.SplitTable(tblID, N/2) ctx := context.Background() for i := 0; i < N/2; i++ { @@ -199,7 +197,7 @@ func (s *seqTestSuite) TestShow(c *C) { row := result.Rows()[0] // For issue https://github.com/pingcap/tidb/issues/1061 expectedRow := []interface{}{ - "SHOW_test", "CREATE TABLE `SHOW_test` (\n `id` int(11) NOT NULL AUTO_INCREMENT,\n `c1` int(11) DEFAULT NULL COMMENT 'c1_comment',\n `c2` int(11) DEFAULT NULL,\n `c3` int(11) DEFAULT '1',\n `c4` text DEFAULT NULL,\n `c5` tinyint(1) DEFAULT NULL,\n PRIMARY KEY (`id`),\n KEY `idx_wide_c4` (`c3`,`c4`(10))\n) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin AUTO_INCREMENT=28934 COMMENT='table_comment'"} + "SHOW_test", "CREATE TABLE `SHOW_test` (\n `id` int(11) NOT NULL AUTO_INCREMENT,\n `c1` int(11) DEFAULT NULL COMMENT 'c1_comment',\n `c2` int(11) DEFAULT NULL,\n `c3` int(11) DEFAULT 1,\n `c4` text DEFAULT NULL,\n `c5` tinyint(1) DEFAULT NULL,\n PRIMARY KEY (`id`),\n KEY `idx_wide_c4` (`c3`,`c4`(10))\n) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin AUTO_INCREMENT=28934 COMMENT='table_comment'"} for i, r := range row { c.Check(r, Equals, expectedRow[i]) } @@ -219,7 +217,7 @@ func (s *seqTestSuite) TestShow(c *C) { c.Check(result.Rows(), HasLen, 1) row = result.Rows()[0] expectedRow = []interface{}{ - "ptest", "CREATE TABLE `ptest` (\n `a` int(11) NOT NULL,\n `b` double NOT NULL DEFAULT '2.0',\n `c` varchar(10) NOT NULL,\n `d` time DEFAULT NULL,\n `e` timestamp NULL DEFAULT NULL,\n `f` timestamp NULL DEFAULT NULL,\n PRIMARY KEY (`a`),\n UNIQUE KEY `d` (`d`)\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"} + "ptest", "CREATE TABLE `ptest` (\n `a` int(11) NOT NULL,\n `b` double NOT NULL DEFAULT 2.0,\n `c` varchar(10) NOT NULL,\n `d` time DEFAULT NULL,\n `e` timestamp NULL DEFAULT NULL,\n `f` timestamp NULL DEFAULT NULL,\n PRIMARY KEY (`a`),\n UNIQUE KEY `d` (`d`)\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"} for i, r := range row { c.Check(r, Equals, expectedRow[i]) } @@ -566,7 +564,7 @@ func (s *seqTestSuite) TestShow(c *C) { " `b` int(11) DEFAULT NULL,\n"+ " `c` char(1) DEFAULT NULL,\n"+ " `d` int(11) DEFAULT NULL\n"+ - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"+"\nPARTITION BY RANGE COLUMNS(a,d,c) (\n PARTITION `p0` VALUES LESS THAN (5,10,\"ggg\"),\n PARTITION `p1` VALUES LESS THAN (10,20,\"mmm\"),\n PARTITION `p2` VALUES LESS THAN (15,30,\"sss\"),\n PARTITION `p3` VALUES LESS THAN (50,MAXVALUE,MAXVALUE)\n)", + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", )) // Test hash partition @@ -802,39 +800,76 @@ func HelperTestAdminShowNextID(c *C, s *seqTestSuite, str string) { tk.MustExec("create table t(id int, c int)") // Start handle is 1. r := tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 1")) + r.Check(testkit.Rows("test t _tidb_rowid 1 AUTO_INCREMENT")) // Row ID is step + 1. tk.MustExec("insert into t values(1, 1)") r = tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 11")) + r.Check(testkit.Rows("test t _tidb_rowid 11 AUTO_INCREMENT")) // Row ID is original + step. for i := 0; i < int(step); i++ { tk.MustExec("insert into t values(10000, 1)") } r = tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 21")) + r.Check(testkit.Rows("test t _tidb_rowid 21 AUTO_INCREMENT")) tk.MustExec("drop table t") // test for a table with the primary key tk.MustExec("create table tt(id int primary key auto_increment, c int)") // Start handle is 1. r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test tt id 1")) + r.Check(testkit.Rows("test tt id 1 AUTO_INCREMENT")) // After rebasing auto ID, row ID is 20 + step + 1. tk.MustExec("insert into tt values(20, 1)") r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test tt id 31")) + r.Check(testkit.Rows("test tt id 31 AUTO_INCREMENT")) // test for renaming the table tk.MustExec("drop database if exists test1") tk.MustExec("create database test1") tk.MustExec("rename table test.tt to test1.tt") tk.MustExec("use test1") r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test1 tt id 31")) + r.Check(testkit.Rows("test1 tt id 31 AUTO_INCREMENT")) tk.MustExec("insert test1.tt values ()") r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test1 tt id 41")) + r.Check(testkit.Rows("test1 tt id 41 AUTO_INCREMENT")) tk.MustExec("drop table tt") + + oldAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom + config.GetGlobalConfig().Experimental.AllowAutoRandom = true + defer func() { + config.GetGlobalConfig().Experimental.AllowAutoRandom = oldAutoRandom + }() + + // Test for a table with auto_random primary key. + tk.MustExec("create table t3(id int primary key auto_random(5), c int)") + // Start handle is 1. + r = tk.MustQuery(str + " t3 next_row_id") + r.Check(testkit.Rows("test1 t3 _tidb_rowid 1 AUTO_INCREMENT", "test1 t3 id 1 AUTO_RANDOM")) + // Insert some rows. + tk.MustExec("insert into t3 (c) values (1), (2);") + r = tk.MustQuery(str + " t3 next_row_id") + r.Check(testkit.Rows("test1 t3 _tidb_rowid 1 AUTO_INCREMENT", "test1 t3 id 11 AUTO_RANDOM")) + // Rebase. + tk.MustExec("insert into t3 (id, c) values (103, 3);") + r = tk.MustQuery(str + " t3 next_row_id") + r.Check(testkit.Rows("test1 t3 _tidb_rowid 1 AUTO_INCREMENT", "test1 t3 id 114 AUTO_RANDOM")) + + // Test for a sequence. + tk.MustExec("create sequence seq1 start 15 cache 57") + r = tk.MustQuery(str + " seq1 next_row_id") + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 15 SEQUENCE")) + r = tk.MustQuery("select nextval(seq1)") + r.Check(testkit.Rows("15")) + r = tk.MustQuery(str + " seq1 next_row_id") + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 72 SEQUENCE")) + r = tk.MustQuery("select nextval(seq1)") + r.Check(testkit.Rows("16")) + r = tk.MustQuery(str + " seq1 next_row_id") + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 72 SEQUENCE")) + r = tk.MustQuery("select setval(seq1, 96)") + r.Check(testkit.Rows("96")) + r = tk.MustQuery(str + " seq1 next_row_id") + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 97 SEQUENCE")) } func (s *seqTestSuite) TestNoHistoryWhenDisableRetry(c *C) { @@ -1316,7 +1351,7 @@ func (s *seqTestSuite) TestAutoRandRecoverTable(c *C) { // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. ddl.EmulatorGCDisable() gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - time.Duration(48*60*60*time.Second)).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` @@ -1390,178 +1425,3 @@ func (s *seqTestSuite) TestOOMPanicInHashJoinWhenFetchBuildRows(c *C) { err := tk.QueryToErr("select * from t as t2 join t as t1 where t1.c1=t2.c1") c.Assert(err.Error(), Equals, "failpoint panic: ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]") } - -type testOOMSuite struct { - store kv.Storage - do *domain.Domain - oom *oomCapturer -} - -func (s *testOOMSuite) SetUpSuite(c *C) { - testleak.BeforeTest() - s.registerHook() - var err error - s.store, err = mockstore.NewMockTikvStore() - c.Assert(err, IsNil) - domain.RunAutoAnalyze = false - s.do, err = session.BootstrapSession(s.store) - c.Assert(err, IsNil) - originCfg := config.GetGlobalConfig() - newConf := *originCfg - newConf.OOMAction = config.OOMActionLog - config.StoreGlobalConfig(&newConf) -} - -func (s *testOOMSuite) TearDownSuite(c *C) { - s.do.Close() - s.store.Close() -} - -func (s *testOOMSuite) registerHook() { - conf := &log.Config{Level: os.Getenv("log_level"), File: log.FileLogConfig{}} - _, r, _ := log.InitLogger(conf) - s.oom = &oomCapturer{r.Core, "", sync.Mutex{}} - lg := zap.New(s.oom) - log.ReplaceGlobals(lg, r) -} - -func (s *testOOMSuite) TestMemTracker4UpdateExec(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("create table t_MemTracker4UpdateExec (id int, a int, b int, index idx_a(`a`))") - - log.SetLevel(zap.InfoLevel) - s.oom.tracker = "" - tk.MustExec("insert into t_MemTracker4UpdateExec values (1,1,1), (2,2,2), (3,3,3)") - c.Assert(s.oom.tracker, Equals, "") - tk.Se.GetSessionVars().MemQuotaQuery = 244 - tk.MustExec("update t_MemTracker4UpdateExec set a = 4") - c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") -} - -func (s *testOOMSuite) TestMemTracker4InsertAndReplaceExec(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("create table t (id int, a int, b int, index idx_a(`a`))") - tk.MustExec("insert into t values (1,1,1), (2,2,2), (3,3,3)") - - tk.MustExec("create table t_MemTracker4InsertAndReplaceExec (id int, a int, b int, index idx_a(`a`))") - - log.SetLevel(zap.InfoLevel) - s.oom.tracker = "" - tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") - c.Assert(s.oom.tracker, Equals, "") - tk.Se.GetSessionVars().MemQuotaQuery = 1 - tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") - c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") - tk.Se.GetSessionVars().MemQuotaQuery = -1 - - s.oom.tracker = "" - tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") - c.Assert(s.oom.tracker, Equals, "") - tk.Se.GetSessionVars().MemQuotaQuery = 1 - tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") - c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") - tk.Se.GetSessionVars().MemQuotaQuery = -1 - - s.oom.tracker = "" - tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec select * from t") - c.Assert(s.oom.tracker, Equals, "") - tk.Se.GetSessionVars().MemQuotaQuery = 1 - tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec select * from t") - c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") - tk.Se.GetSessionVars().MemQuotaQuery = -1 - - s.oom.tracker = "" - tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec select * from t") - c.Assert(s.oom.tracker, Equals, "") - tk.Se.GetSessionVars().MemQuotaQuery = 1 - tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec select * from t") - c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") - tk.Se.GetSessionVars().MemQuotaQuery = -1 - - tk.Se.GetSessionVars().DMLBatchSize = 1 - tk.Se.GetSessionVars().BatchInsert = true - s.oom.tracker = "" - tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") - c.Assert(s.oom.tracker, Equals, "") - tk.Se.GetSessionVars().MemQuotaQuery = 1 - tk.MustExec("insert into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") - c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") - tk.Se.GetSessionVars().MemQuotaQuery = -1 - - s.oom.tracker = "" - tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") - c.Assert(s.oom.tracker, Equals, "") - tk.Se.GetSessionVars().MemQuotaQuery = 1 - tk.MustExec("replace into t_MemTracker4InsertAndReplaceExec values (1,1,1), (2,2,2), (3,3,3)") - c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") - tk.Se.GetSessionVars().MemQuotaQuery = -1 -} - -func (s *testOOMSuite) TestMemTracker4DeleteExec(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("create table MemTracker4DeleteExec1 (id int, a int, b int, index idx_a(`a`))") - tk.MustExec("create table MemTracker4DeleteExec2 (id int, a int, b int, index idx_a(`a`))") - - // delete from single table - log.SetLevel(zap.InfoLevel) - tk.MustExec("insert into MemTracker4DeleteExec1 values(1,1,1), (2,2,2), (3,3,3), (4,4,4), (5,5,5)") - s.oom.tracker = "" - tk.MustExec("delete from MemTracker4DeleteExec1") - c.Assert(s.oom.tracker, Equals, "") - tk.MustExec("insert into MemTracker4DeleteExec1 values (1,1,1), (2,2,2), (3,3,3)") - tk.Se.GetSessionVars().MemQuotaQuery = 1 - tk.MustExec("delete from MemTracker4DeleteExec1") - c.Assert(s.oom.tracker, Matches, "expensive_query during bootstrap phase") - - // delete from multiple table - tk.Se.GetSessionVars().MemQuotaQuery = 100000 - tk.MustExec("insert into MemTracker4DeleteExec1 values(1,1,1)") - tk.MustExec("insert into MemTracker4DeleteExec2 values(1,1,1)") - s.oom.tracker = "" - tk.MustExec("delete MemTracker4DeleteExec1, MemTracker4DeleteExec2 from MemTracker4DeleteExec1 join MemTracker4DeleteExec2 on MemTracker4DeleteExec1.a=MemTracker4DeleteExec2.a") - c.Assert(s.oom.tracker, Equals, "") - tk.MustExec("insert into MemTracker4DeleteExec1 values(1,1,1)") - tk.MustExec("insert into MemTracker4DeleteExec2 values(1,1,1)") - s.oom.tracker = "" - tk.Se.GetSessionVars().MemQuotaQuery = 10000 - tk.MustExec("delete MemTracker4DeleteExec1, MemTracker4DeleteExec2 from MemTracker4DeleteExec1 join MemTracker4DeleteExec2 on MemTracker4DeleteExec1.a=MemTracker4DeleteExec2.a") - c.Assert(s.oom.tracker, Equals, "expensive_query during bootstrap phase") -} - -type oomCapturer struct { - zapcore.Core - tracker string - mu sync.Mutex -} - -func (h *oomCapturer) Write(entry zapcore.Entry, fields []zapcore.Field) error { - if strings.Contains(entry.Message, "memory exceeds quota") { - err, _ := fields[0].Interface.(error) - str := err.Error() - begin := strings.Index(str, "8001]") - if begin == -1 { - panic("begin not found") - } - end := strings.Index(str, " holds") - if end == -1 { - panic("end not found") - } - h.tracker = str[begin+len("8001]") : end] - return nil - } - - h.mu.Lock() - h.tracker = entry.Message - h.mu.Unlock() - return nil -} - -func (h *oomCapturer) Check(e zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { - if h.Enabled(e.Level) { - return ce.AddCore(e, h) - } - return ce -} diff --git a/executor/set.go b/executor/set.go index 1be6a349cfef4..3ecc8fba9a938 100644 --- a/executor/set.go +++ b/executor/set.go @@ -15,6 +15,7 @@ package executor import ( "context" + "fmt" "strings" "github.com/pingcap/errors" @@ -28,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/stmtsummary" @@ -173,6 +175,10 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e if name == variable.TxnIsolationOneShot && sessionVars.InTxn() { return errors.Trace(ErrCantChangeTxCharacteristics) } + if name == variable.TiDBFoundInPlanCache { + sessionVars.StmtCtx.AppendWarning(fmt.Errorf("Set operation for '%s' will not take effect", variable.TiDBFoundInPlanCache)) + return nil + } err = variable.SetSessionSystemVar(sessionVars, name, value) if err != nil { return err @@ -234,7 +240,7 @@ func (e *SetExecutor) setCharset(cs, co string) error { } } else { var coll *charset.Collation - if coll, err = charset.GetCollationByName(co); err != nil { + if coll, err = collate.GetCollationByName(co); err != nil { return err } if coll.CharsetName != cs { diff --git a/executor/set_config.go b/executor/set_config.go new file mode 100644 index 0000000000000..f7e2b824cc09f --- /dev/null +++ b/executor/set_config.go @@ -0,0 +1,207 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor + +import ( + "bytes" + "context" + "fmt" + "io/ioutil" + "net" + "net/http" + "strings" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/pdapi" + "github.com/pingcap/tidb/util/set" + "github.com/pingcap/tidb/util/stringutil" +) + +// SetConfigExec executes 'SET CONFIG' statement. +type SetConfigExec struct { + baseExecutor + p *core.SetConfig + jsonBody string +} + +// Open implements the Executor Open interface. +func (s *SetConfigExec) Open(ctx context.Context) error { + if s.p.Type != "" { + s.p.Type = strings.ToLower(s.p.Type) + if s.p.Type != "tikv" && s.p.Type != "tidb" && s.p.Type != "pd" { + return errors.Errorf("unknown type %v", s.p.Type) + } + if s.p.Type == "tidb" { + return errors.Errorf("TiDB doesn't support to change configs online, please use SQL variables") + } + } + if s.p.Instance != "" { + s.p.Instance = strings.ToLower(s.p.Instance) + if !isValidInstance(s.p.Instance) { + return errors.Errorf("invalid instance %v", s.p.Instance) + } + } + s.p.Name = strings.ToLower(s.p.Name) + + body, err := ConvertConfigItem2JSON(s.ctx, s.p.Name, s.p.Value) + s.jsonBody = body + return err +} + +// TestSetConfigServerInfoKey is used as the key to store 'TestSetConfigServerInfoFunc' in the context. +var TestSetConfigServerInfoKey stringutil.StringerStr = "TestSetConfigServerInfoKey" + +// TestSetConfigHTTPHandlerKey is used as the key to store 'TestSetConfigDoRequestFunc' in the context. +var TestSetConfigHTTPHandlerKey stringutil.StringerStr = "TestSetConfigHTTPHandlerKey" + +// Next implements the Executor Next interface. +func (s *SetConfigExec) Next(ctx context.Context, req *chunk.Chunk) error { + req.Reset() + getServerFunc := infoschema.GetClusterServerInfo + if v := s.ctx.Value(TestSetConfigServerInfoKey); v != nil { + getServerFunc = v.(func(sessionctx.Context) ([]infoschema.ServerInfo, error)) + } + + serversInfo, err := getServerFunc(s.ctx) + if err != nil { + return err + } + nodeTypes := set.NewStringSet() + nodeAddrs := set.NewStringSet() + if s.p.Type != "" { + nodeTypes.Insert(s.p.Type) + } + if s.p.Instance != "" { + nodeAddrs.Insert(s.p.Instance) + } + serversInfo = filterClusterServerInfo(serversInfo, nodeTypes, nodeAddrs) + + for _, serverInfo := range serversInfo { + var url string + switch serverInfo.ServerType { + case "pd": + url = fmt.Sprintf("%s://%s%s", util.InternalHTTPSchema(), serverInfo.StatusAddr, pdapi.Config) + case "tikv": + url = fmt.Sprintf("%s://%s/config", util.InternalHTTPSchema(), serverInfo.StatusAddr) + default: + continue + } + if err := s.doRequest(url); err != nil { + s.ctx.GetSessionVars().StmtCtx.AppendWarning(err) + } + } + return nil +} + +func (s *SetConfigExec) doRequest(url string) (retErr error) { + body := bytes.NewBufferString(s.jsonBody) + req, err := http.NewRequest(http.MethodPost, url, body) + if err != nil { + return err + } + var httpHandler func(req *http.Request) (*http.Response, error) + if v := s.ctx.Value(TestSetConfigHTTPHandlerKey); v != nil { + httpHandler = v.(func(*http.Request) (*http.Response, error)) + } else { + httpHandler = util.InternalHTTPClient().Do + } + resp, err := httpHandler(req) + if err != nil { + return err + } + defer func() { + if err := resp.Body.Close(); err != nil { + if retErr == nil { + retErr = err + } + } + }() + if resp.StatusCode == http.StatusOK { + return nil + } else if resp.StatusCode >= 400 && resp.StatusCode < 600 { + message, err := ioutil.ReadAll(resp.Body) + if err != nil { + return err + } + return errors.Errorf("bad request to %s: %s", url, message) + } + return errors.Errorf("request %s failed: %s", url, resp.Status) +} + +func isValidInstance(instance string) bool { + ip, port, err := net.SplitHostPort(instance) + if err != nil { + return false + } + if port == "" { + return false + } + v := net.ParseIP(ip) + return v != nil +} + +// ConvertConfigItem2JSON converts the config item specified by key and val to json. +// For example: +// set config x key="val" ==> {"key":"val"} +// set config x key=233 ==> {"key":233} +func ConvertConfigItem2JSON(ctx sessionctx.Context, key string, val expression.Expression) (body string, err error) { + if val == nil { + return "", errors.Errorf("cannot set config to null") + } + isNull := false + str := "" + switch val.GetType().EvalType() { + case types.ETString: + var s string + s, isNull, err = val.EvalString(ctx, chunk.Row{}) + if err == nil && !isNull { + str = fmt.Sprintf(`"%s"`, s) + } + case types.ETInt: + var i int64 + i, isNull, err = val.EvalInt(ctx, chunk.Row{}) + if err == nil && !isNull { + str = fmt.Sprintf("%v", i) + } + case types.ETReal: + var f float64 + f, isNull, err = val.EvalReal(ctx, chunk.Row{}) + if err == nil && !isNull { + str = fmt.Sprintf("%v", f) + } + case types.ETDecimal: + d := new(types.MyDecimal) + d, isNull, err = val.EvalDecimal(ctx, chunk.Row{}) + if err == nil && !isNull { + str = string(d.ToString()) + } + default: + return "", errors.Errorf("unsupported config value type") + } + if err != nil { + return + } + if isNull { + return "", errors.Errorf("can't set config to null") + } + body = fmt.Sprintf(`{"%s":%s}`, key, str) + return body, nil +} diff --git a/executor/set_test.go b/executor/set_test.go index 5ee65788580a6..d6f0bea5a8e73 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -14,15 +14,24 @@ package executor_test import ( + "bytes" "context" + "errors" + "io/ioutil" + "net/http" "strconv" . "github.com/pingcap/check" + "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" ) @@ -927,54 +936,81 @@ func (s *testSuite5) TestEnableNoopFunctionsVar(c *C) { tk.MustQuery(`select @@tidb_enable_noop_functions;`).Check(testkit.Rows("0")) } -type testSuite10 struct { - *baseTestSuite +func (s *testSuite5) TestSetClusterConfig(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + serversInfo := []infoschema.ServerInfo{ + {ServerType: "tidb", Address: "127.0.0.1:1111", StatusAddr: "127.0.0.1:1111"}, + {ServerType: "tidb", Address: "127.0.0.1:2222", StatusAddr: "127.0.0.1:2222"}, + {ServerType: "pd", Address: "127.0.0.1:3333", StatusAddr: "127.0.0.1:3333"}, + {ServerType: "pd", Address: "127.0.0.1:4444", StatusAddr: "127.0.0.1:4444"}, + {ServerType: "tikv", Address: "127.0.0.1:5555", StatusAddr: "127.0.0.1:5555"}, + {ServerType: "tikv", Address: "127.0.0.1:6666", StatusAddr: "127.0.0.1:6666"}, + } + var serverInfoErr error + serverInfoFunc := func(sessionctx.Context) ([]infoschema.ServerInfo, error) { + return serversInfo, serverInfoErr + } + tk.Se.SetValue(executor.TestSetConfigServerInfoKey, serverInfoFunc) + + c.Assert(tk.ExecToErr("set config xxx log.level='info'"), ErrorMatches, "unknown type xxx") + c.Assert(tk.ExecToErr("set config tidb log.level='info'"), ErrorMatches, "TiDB doesn't support to change configs online, please use SQL variables") + c.Assert(tk.ExecToErr("set config '127.a.b.c:1234' log.level='info'"), ErrorMatches, "invalid instance 127.a.b.c:1234") + c.Assert(tk.ExecToErr("set config tikv log.level=null"), ErrorMatches, "can't set config to null") + + httpCnt := 0 + tk.Se.SetValue(executor.TestSetConfigHTTPHandlerKey, func(*http.Request) (*http.Response, error) { + httpCnt++ + return &http.Response{StatusCode: http.StatusOK, Body: ioutil.NopCloser(nil)}, nil + }) + tk.MustExec("set config tikv log.level='info'") + c.Assert(httpCnt, Equals, 2) + + httpCnt = 0 + tk.MustExec("set config '127.0.0.1:5555' log.level='info'") + c.Assert(httpCnt, Equals, 1) + + httpCnt = 0 + tk.Se.SetValue(executor.TestSetConfigHTTPHandlerKey, func(*http.Request) (*http.Response, error) { + return nil, errors.New("something wrong") + }) + tk.MustExec("set config tikv log.level='info'") + tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1105 something wrong", "Warning 1105 something wrong")) + + tk.Se.SetValue(executor.TestSetConfigHTTPHandlerKey, func(*http.Request) (*http.Response, error) { + return &http.Response{StatusCode: http.StatusBadRequest, Body: ioutil.NopCloser(bytes.NewBufferString("WRONG"))}, nil + }) + tk.MustExec("set config tikv log.level='info'") + tk.MustQuery("show warnings").Check(testkit.Rows( + "Warning 1105 bad request to http://127.0.0.1:5555/config: WRONG", "Warning 1105 bad request to http://127.0.0.1:6666/config: WRONG")) } -func (s *testSuite10) TestSetConflictConfigItems(c *C) { - tk := testkit.NewTestKit(c, s.store) - c.Assert(config.GetGlobalConfig().EnableDynamicConfig, IsFalse) - tk.MustExec("set tidb_slow_log_threshold=123") - tk.MustQuery("select @@tidb_slow_log_threshold").Check(testkit.Rows("123")) - tk.MustExec("set tidb_query_log_max_len=123") - tk.MustQuery("select @@tidb_query_log_max_len").Check(testkit.Rows("123")) - tk.MustExec("set tidb_record_plan_in_slow_log=1") - tk.MustQuery("select @@tidb_record_plan_in_slow_log").Check(testkit.Rows("1")) - tk.MustExec("set tidb_check_mb4_value_in_utf8=1") - tk.MustQuery("select @@tidb_check_mb4_value_in_utf8").Check(testkit.Rows("1")) - tk.MustExec("set tidb_enable_slow_log=1") - tk.MustQuery("select @@tidb_enable_slow_log").Check(testkit.Rows("1")) - - config.GetGlobalConfig().EnableDynamicConfig = true - tk.MustExec("set tidb_slow_log_threshold=222") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 cannot update tidb_slow_log_threshold when enabling dynamic configs")) - tk.MustQuery("select @@tidb_slow_log_threshold").Check(testkit.Rows("123")) - - tk.MustExec("set tidb_query_log_max_len=222") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 cannot update tidb_query_log_max_len when enabling dynamic configs")) - tk.MustQuery("select @@tidb_query_log_max_len").Check(testkit.Rows("123")) - - tk.MustExec("set tidb_record_plan_in_slow_log=0") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 cannot update tidb_record_plan_in_slow_log when enabling dynamic configs")) - tk.MustQuery("select @@tidb_record_plan_in_slow_log").Check(testkit.Rows("1")) - - tk.MustExec("set tidb_check_mb4_value_in_utf8=0") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 cannot update tidb_check_mb4_value_in_utf8 when enabling dynamic configs")) - tk.MustQuery("select @@tidb_check_mb4_value_in_utf8").Check(testkit.Rows("1")) - - tk.MustExec("set tidb_enable_slow_log=0") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 cannot update tidb_enable_slow_log when enabling dynamic configs")) - tk.MustQuery("select @@tidb_enable_slow_log").Check(testkit.Rows("1")) - - config.GetGlobalConfig().EnableDynamicConfig = false - tk.MustExec("set tidb_slow_log_threshold=222") - tk.MustQuery("select @@tidb_slow_log_threshold").Check(testkit.Rows("222")) - tk.MustExec("set tidb_query_log_max_len=222") - tk.MustQuery("select @@tidb_query_log_max_len").Check(testkit.Rows("222")) - tk.MustExec("set tidb_record_plan_in_slow_log=0") - tk.MustQuery("select @@tidb_record_plan_in_slow_log").Check(testkit.Rows("0")) - tk.MustExec("set tidb_check_mb4_value_in_utf8=0") - tk.MustQuery("select @@tidb_check_mb4_value_in_utf8").Check(testkit.Rows("0")) - tk.MustExec("set tidb_enable_slow_log=0") - tk.MustQuery("select @@tidb_enable_slow_log").Check(testkit.Rows("0")) +func (s *testSuite5) TestSetClusterConfigJSONData(c *C) { + var d types.MyDecimal + c.Assert(d.FromFloat64(123.456), IsNil) + cases := []struct { + val expression.Expression + result string + succ bool + }{ + {&expression.Constant{Value: types.NewIntDatum(2333), RetType: types.NewFieldType(mysql.TypeLong)}, `{"k":2333}`, true}, + {&expression.Constant{Value: types.NewFloat64Datum(23.33), RetType: types.NewFieldType(mysql.TypeDouble)}, `{"k":23.33}`, true}, + {&expression.Constant{Value: types.NewStringDatum("abcd"), RetType: types.NewFieldType(mysql.TypeString)}, `{"k":"abcd"}`, true}, + {&expression.Constant{Value: types.NewDecimalDatum(&d), RetType: types.NewFieldType(mysql.TypeNewDecimal)}, `{"k":123.456}`, true}, + {&expression.Constant{Value: types.NewDatum(nil), RetType: types.NewFieldType(mysql.TypeLonglong)}, "", false}, + {&expression.Constant{RetType: types.NewFieldType(mysql.TypeJSON)}, "", false}, // unsupported type + {nil, "", false}, + } + + ctx := mock.NewContext() + for _, t := range cases { + result, err := executor.ConvertConfigItem2JSON(ctx, "k", t.val) + if t.succ { + c.Assert(t.result, Equals, result) + } else { + c.Assert(err, NotNil) + } + } } diff --git a/executor/show.go b/executor/show.go index 26898b2c745b7..a2eb830fc409d 100644 --- a/executor/show.go +++ b/executor/show.go @@ -58,6 +58,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/set" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stringutil" ) @@ -127,6 +128,8 @@ func (e *ShowExec) fetchAll(ctx context.Context) error { return e.fetchShowCollation() case ast.ShowColumns: return e.fetchShowColumns(ctx) + case ast.ShowConfig: + return e.fetchShowClusterConfigs(ctx) case ast.ShowCreateTable: return e.fetchShowCreateTable() case ast.ShowCreateSequence: @@ -197,6 +200,10 @@ func (e *ShowExec) fetchAll(ctx context.Context) error { return e.fetchShowTableRegions() case ast.ShowBuiltins: return e.fetchShowBuiltins() + case ast.ShowBackups: + return e.fetchShowBRIE(ast.BRIEKindBackup) + case ast.ShowRestores: + return e.fetchShowBRIE(ast.BRIEKindRestore) } return nil } @@ -802,6 +809,8 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T if col.Tp == mysql.TypeBit { defaultValBinaryLiteral := types.BinaryLiteral(defaultValStr) fmt.Fprintf(buf, " DEFAULT %s", defaultValBinaryLiteral.ToBitLiteralString(true)) + } else if types.IsTypeNumeric(col.Tp) || col.DefaultIsExpr { + fmt.Fprintf(buf, " DEFAULT %s", format.OutputFormat(defaultValStr)) } else { fmt.Fprintf(buf, " DEFAULT '%s'", format.OutputFormat(defaultValStr)) } @@ -903,6 +912,14 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } } + if tableInfo.AutoIdCache != 0 { + fmt.Fprintf(buf, " /*T![auto_id_cache] AUTO_ID_CACHE=%d */", tableInfo.AutoIdCache) + } + + if tableInfo.AutoRandID != 0 { + fmt.Fprintf(buf, " /*T![auto_rand_base] AUTO_RANDOM_BASE=%d */", tableInfo.AutoRandID) + } + if tableInfo.ShardRowIDBits > 0 { fmt.Fprintf(buf, "/*!90000 SHARD_ROW_ID_BITS=%d ", tableInfo.ShardRowIDBits) if tableInfo.PreSplitRegions > 0 { @@ -959,6 +976,34 @@ func (e *ShowExec) fetchShowCreateSequence() error { return nil } +// TestShowClusterConfigKey is the key used to store TestShowClusterConfigFunc. +var TestShowClusterConfigKey stringutil.StringerStr = "TestShowClusterConfigKey" + +// TestShowClusterConfigFunc is used to test 'show config ...'. +type TestShowClusterConfigFunc func() ([][]types.Datum, error) + +func (e *ShowExec) fetchShowClusterConfigs(ctx context.Context) error { + emptySet := set.NewStringSet() + var confItems [][]types.Datum + var err error + if f := e.ctx.Value(TestShowClusterConfigKey); f != nil { + confItems, err = f.(TestShowClusterConfigFunc)() + } else { + confItems, err = fetchClusterConfig(e.ctx, emptySet, emptySet) + } + if err != nil { + return err + } + for _, items := range confItems { + row := make([]interface{}, 0, 4) + for _, item := range items { + row = append(row, item.GetString()) + } + e.appendRow(row) + } + return nil +} + func (e *ShowExec) fetchShowCreateTable() error { tb, err := e.getTable() if err != nil { @@ -1060,9 +1105,32 @@ func ConstructResultOfShowCreateDatabase(ctx sessionctx.Context, dbInfo *model.D ifNotExistsStr = "/*!32312 IF NOT EXISTS*/ " } fmt.Fprintf(buf, "CREATE DATABASE %s%s", ifNotExistsStr, stringutil.Escape(dbInfo.Name.O, sqlMode)) - if s := dbInfo.Charset; len(s) > 0 { - fmt.Fprintf(buf, " /*!40100 DEFAULT CHARACTER SET %s */", s) + if dbInfo.Charset != "" { + fmt.Fprintf(buf, " /*!40100 DEFAULT CHARACTER SET %s ", dbInfo.Charset) + defaultCollate, err := charset.GetDefaultCollation(dbInfo.Charset) + if err != nil { + return errors.Trace(err) + } + if dbInfo.Collate != "" && dbInfo.Collate != defaultCollate { + fmt.Fprintf(buf, "COLLATE %s ", dbInfo.Collate) + } + fmt.Fprint(buf, "*/") + return nil + } + if dbInfo.Collate != "" { + collInfo, err := collate.GetCollationByName(dbInfo.Collate) + if err != nil { + return errors.Trace(err) + } + fmt.Fprintf(buf, " /*!40100 DEFAULT CHARACTER SET %s ", collInfo.CharsetName) + if !collInfo.IsDefault { + fmt.Fprintf(buf, "COLLATE %s ", dbInfo.Collate) + } + fmt.Fprint(buf, "*/") + return nil } + // MySQL 5.7 always show the charset info but TiDB may ignore it, which makes a slight difference. We keep this + // behavior unchanged because it is trivial enough. return nil } diff --git a/executor/show_test.go b/executor/show_test.go index 6fc30ac0d6bb1..ffe2d5eadf9c6 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -27,6 +27,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" ) @@ -675,6 +676,31 @@ func (s *testSuite5) TestShowCreateTable(c *C) { " `a` varchar(10) DEFAULT NULL\n"+ ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", // non-binary collate is kept. )) + // Test for issue #17 in bug competition, default num and sequence should be shown without quote. + tk.MustExec(`drop table if exists default_num`) + tk.MustExec("create table default_num(a int default 11)") + tk.MustQuery("show create table default_num").Check(testutil.RowsWithSep("|", + ""+ + "default_num CREATE TABLE `default_num` (\n"+ + " `a` int(11) DEFAULT 11\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", + )) + tk.MustExec(`drop table if exists default_varchar`) + tk.MustExec("create table default_varchar(a varchar(10) default \"haha\")") + tk.MustQuery("show create table default_varchar").Check(testutil.RowsWithSep("|", + ""+ + "default_varchar CREATE TABLE `default_varchar` (\n"+ + " `a` varchar(10) DEFAULT 'haha'\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", + )) + tk.MustExec(`drop table if exists default_sequence`) + tk.MustExec("create table default_sequence(a int default nextval(seq))") + tk.MustQuery("show create table default_sequence").Check(testutil.RowsWithSep("|", + ""+ + "default_sequence CREATE TABLE `default_sequence` (\n"+ + " `a` int(11) DEFAULT nextval(`test`.`seq`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", + )) } func (s *testAutoRandomSuite) TestShowCreateTableAutoRandom(c *C) { @@ -715,6 +741,71 @@ func (s *testAutoRandomSuite) TestShowCreateTableAutoRandom(c *C) { " PRIMARY KEY (`a`)\n"+ ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", )) + // Test show auto_random table option. + tk.MustExec("create table auto_random_tbl4 (a bigint primary key auto_random(5), b varchar(255)) auto_random_base = 100") + tk.MustQuery("show create table `auto_random_tbl4`").Check(testutil.RowsWithSep("|", + ""+ + "auto_random_tbl4 CREATE TABLE `auto_random_tbl4` (\n"+ + " `a` bigint(20) NOT NULL /*T![auto_rand] AUTO_RANDOM(5) */,\n"+ + " `b` varchar(255) DEFAULT NULL,\n"+ + " PRIMARY KEY (`a`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_rand_base] AUTO_RANDOM_BASE=100 */", + )) + // Test implicit auto_random with auto_random table option. + tk.MustExec("create table auto_random_tbl5 (a bigint auto_random primary key, b char) auto_random_base 50") + tk.MustQuery("show create table auto_random_tbl5").Check(testutil.RowsWithSep("|", + ""+ + "auto_random_tbl5 CREATE TABLE `auto_random_tbl5` (\n"+ + " `a` bigint(20) NOT NULL /*T![auto_rand] AUTO_RANDOM(5) */,\n"+ + " `b` char(1) DEFAULT NULL,\n"+ + " PRIMARY KEY (`a`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_rand_base] AUTO_RANDOM_BASE=50 */", + )) + // Test auto_random table option already with special comment. + tk.MustExec("create table auto_random_tbl6 (a bigint /*T![auto_rand] auto_random */ primary key) auto_random_base 200") + tk.MustQuery("show create table auto_random_tbl6").Check(testutil.RowsWithSep("|", + ""+ + "auto_random_tbl6 CREATE TABLE `auto_random_tbl6` (\n"+ + " `a` bigint(20) NOT NULL /*T![auto_rand] AUTO_RANDOM(5) */,\n"+ + " PRIMARY KEY (`a`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_rand_base] AUTO_RANDOM_BASE=200 */", + )) +} + +// Override testAutoRandomSuite to test auto id cache. +func (s *testAutoRandomSuite) TestAutoIdCache(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 auto_increment key) auto_id_cache = 10") + tk.MustQuery("show create table t").Check(testutil.RowsWithSep("|", + ""+ + "t CREATE TABLE `t` (\n"+ + " `a` int(11) NOT NULL AUTO_INCREMENT,\n"+ + " PRIMARY KEY (`a`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_id_cache] AUTO_ID_CACHE=10 */", + )) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int auto_increment unique, b int key) auto_id_cache 100") + tk.MustQuery("show create table t").Check(testutil.RowsWithSep("|", + ""+ + "t CREATE TABLE `t` (\n"+ + " `a` int(11) NOT NULL AUTO_INCREMENT,\n"+ + " `b` int(11) NOT NULL,\n"+ + " PRIMARY KEY (`b`),\n"+ + " UNIQUE KEY `a` (`a`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_id_cache] AUTO_ID_CACHE=100 */", + )) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int key) auto_id_cache 5") + tk.MustQuery("show create table t").Check(testutil.RowsWithSep("|", + ""+ + "t CREATE TABLE `t` (\n"+ + " `a` int(11) NOT NULL,\n"+ + " PRIMARY KEY (`a`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_id_cache] AUTO_ID_CACHE=5 */", + )) } func (s *testSuite5) TestShowEscape(c *C) { @@ -755,3 +846,37 @@ func (s *testSuite5) TestShowBuiltin(c *C) { c.Assert("abs", Equals, rows[0][0].(string)) c.Assert("yearweek", Equals, rows[267][0].(string)) } + +func (s *testSuite5) TestShowClusterConfig(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + var confItems [][]types.Datum + var confErr error + var confFunc executor.TestShowClusterConfigFunc = func() ([][]types.Datum, error) { + return confItems, confErr + } + tk.Se.SetValue(executor.TestShowClusterConfigKey, confFunc) + strs2Items := func(strs ...string) []types.Datum { + items := make([]types.Datum, 0, len(strs)) + for _, s := range strs { + items = append(items, types.NewStringDatum(s)) + } + return items + } + confItems = append(confItems, strs2Items("tidb", "127.0.0.1:1111", "log.level", "info")) + confItems = append(confItems, strs2Items("pd", "127.0.0.1:2222", "log.level", "info")) + confItems = append(confItems, strs2Items("tikv", "127.0.0.1:3333", "log.level", "info")) + tk.MustQuery("show config").Check(testkit.Rows( + "tidb 127.0.0.1:1111 log.level info", + "pd 127.0.0.1:2222 log.level info", + "tikv 127.0.0.1:3333 log.level info")) + tk.MustQuery("show config where type='tidb'").Check(testkit.Rows( + "tidb 127.0.0.1:1111 log.level info")) + tk.MustQuery("show config where type like '%ti%'").Check(testkit.Rows( + "tidb 127.0.0.1:1111 log.level info", + "tikv 127.0.0.1:3333 log.level info")) + + confErr = fmt.Errorf("something unknown error") + c.Assert(tk.QueryToErr("show config"), ErrorMatches, confErr.Error()) +} diff --git a/executor/simple.go b/executor/simple.go index 3e40dffc1edcd..6640b0e84f11e 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" @@ -529,9 +528,16 @@ func (e *SimpleExec) executeUse(s *ast.UseStmt) error { // The server sets this variable whenever the default database changes. // See http://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_character_set_database sessionVars := e.ctx.GetSessionVars() - terror.Log(sessionVars.SetSystemVar(variable.CharsetDatabase, dbinfo.Charset)) - terror.Log(sessionVars.SetSystemVar(variable.CollationDatabase, dbinfo.Collate)) - return nil + err := sessionVars.SetSystemVar(variable.CharsetDatabase, dbinfo.Charset) + if err != nil { + return err + } + dbCollate := dbinfo.Collate + if dbCollate == "" { + // Since we have checked the charset, the dbCollate here shouldn't be "". + dbCollate = getDefaultCollate(dbinfo.Charset) + } + return sessionVars.SetSystemVar(variable.CollationDatabase, dbCollate) } func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { diff --git a/executor/slow_query.go b/executor/slow_query.go index cf946afcdbd12..30ea85a432033 100644 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -98,8 +98,8 @@ func (e *slowQueryRetriever) initialize(sctx sessionctx.Context) error { } if e.extractor != nil { e.checker.enableTimeCheck = e.extractor.Enable - e.checker.startTime = e.extractor.StartTime - e.checker.endTime = e.extractor.EndTime + e.checker.startTime = types.NewTime(types.FromGoTime(e.extractor.StartTime), mysql.TypeDatetime, types.MaxFsp) + e.checker.endTime = types.NewTime(types.FromGoTime(e.extractor.EndTime), mysql.TypeDatetime, types.MaxFsp) } e.initialized = true e.files, err = e.getAllFiles(sctx, sctx.GetSessionVars().SlowQueryFile) @@ -171,16 +171,16 @@ type slowLogChecker struct { user *auth.UserIdentity // Below fields is used to check slow log time valid. enableTimeCheck bool - startTime time.Time - endTime time.Time + startTime types.Time + endTime types.Time } func (sc *slowLogChecker) hasPrivilege(userName string) bool { return sc.hasProcessPriv || sc.user == nil || userName == sc.user.Username } -func (sc *slowLogChecker) isTimeValid(t time.Time) bool { - if sc.enableTimeCheck && (t.Before(sc.startTime) || t.After(sc.endTime)) { +func (sc *slowLogChecker) isTimeValid(t types.Time) bool { + if sc.enableTimeCheck && (t.Compare(sc.startTime) < 0 || t.Compare(sc.endTime) > 0) { return false } return true @@ -298,7 +298,7 @@ func getOneLine(reader *bufio.Reader) ([]byte, error) { } type slowQueryTuple struct { - time time.Time + time types.Time txnStartTs uint64 user string host string @@ -318,6 +318,7 @@ type slowQueryTuple struct { writeSize uint64 prewriteRegion uint64 txnRetry uint64 + copTime float64 processTime float64 waitTime float64 backOffTime float64 @@ -350,13 +351,15 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string, valid = true switch field { case variable.SlowLogTimeStr: - st.time, err = ParseTime(value) + var t time.Time + t, err = ParseTime(value) if err != nil { break } - if st.time.Location() != tz { - st.time = st.time.In(tz) + if t.Location() != tz { + t = t.In(tz) } + st.time = types.NewTime(types.FromGoTime(t), mysql.TypeDatetime, types.MaxFsp) if checker != nil { valid = checker.isTimeValid(st.time) } @@ -405,6 +408,8 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string, st.prewriteRegion, err = strconv.ParseUint(value, 10, 64) case execdetails.TxnRetryStr: st.txnRetry, err = strconv.ParseUint(value, 10, 64) + case execdetails.CopTimeStr: + st.copTime, err = strconv.ParseFloat(value, 64) case execdetails.ProcessTimeStr: st.processTime, err = strconv.ParseFloat(value, 64) case execdetails.WaitTimeStr: @@ -464,7 +469,7 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string, func (st *slowQueryTuple) convertToDatumRow() []types.Datum { record := make([]types.Datum, 0, 64) - record = append(record, types.NewTimeDatum(types.NewTime(types.FromGoTime(st.time), mysql.TypeDatetime, types.MaxFsp))) + record = append(record, types.NewTimeDatum(st.time)) record = append(record, types.NewUintDatum(st.txnStartTs)) record = append(record, types.NewStringDatum(st.user)) record = append(record, types.NewStringDatum(st.host)) @@ -484,6 +489,7 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { record = append(record, types.NewUintDatum(st.writeSize)) record = append(record, types.NewUintDatum(st.prewriteRegion)) record = append(record, types.NewUintDatum(st.txnRetry)) + record = append(record, types.NewFloat64Datum(st.copTime)) record = append(record, types.NewFloat64Datum(st.processTime)) record = append(record, types.NewFloat64Datum(st.waitTime)) record = append(record, types.NewFloat64Datum(st.backOffTime)) @@ -660,6 +666,17 @@ func (e *slowQueryRetriever) getFileEndTime(file *os.File) (time.Time, error) { cursor := int64(0) line := make([]byte, 0, 64) maxLineNum := 128 + tryGetTime := func(line []byte) string { + for i, j := 0, len(line)-1; i < j; i, j = i+1, j-1 { + line[i], line[j] = line[j], line[i] + } + lineStr := string(line) + lineStr = strings.TrimSpace(lineStr) + if strings.HasPrefix(lineStr, variable.SlowLogStartPrefixStr) { + return lineStr[len(variable.SlowLogStartPrefixStr):] + } + return "" + } for { cursor -= 1 _, err := file.Seek(cursor, io.SeekEnd) @@ -674,19 +691,17 @@ func (e *slowQueryRetriever) getFileEndTime(file *os.File) (time.Time, error) { } // If find a line. if cursor != -1 && (char[0] == '\n' || char[0] == '\r') { - for i, j := 0, len(line)-1; i < j; i, j = i+1, j-1 { - line[i], line[j] = line[j], line[i] - } - lineStr := string(line) - lineStr = strings.TrimSpace(lineStr) - if strings.HasPrefix(lineStr, variable.SlowLogStartPrefixStr) { - return ParseTime(lineStr[len(variable.SlowLogStartPrefixStr):]) + if timeStr := tryGetTime(line); len(timeStr) > 0 { + return ParseTime(timeStr) } line = line[:0] maxLineNum -= 1 } line = append(line, char[0]) if cursor == -fileSize || maxLineNum <= 0 { + if timeStr := tryGetTime(line); len(timeStr) > 0 { + return ParseTime(timeStr) + } return t, errors.Errorf("malform slow query file %v", file.Name()) } } diff --git a/executor/slow_query_test.go b/executor/slow_query_test.go index 5e9d10e057c60..daa05a46acd16 100644 --- a/executor/slow_query_test.go +++ b/executor/slow_query_test.go @@ -47,7 +47,7 @@ func (s *testExecSuite) TestParseSlowLogFile(c *C) { `# Time: 2019-04-28T15:24:04.309074+08:00 # Txn_start_ts: 405888132465033227 # Query_time: 0.216905 -# Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 +# Cop_time: 0.38 Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 # Is_internal: true # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 # Stats: t1:1,t2:2 @@ -75,7 +75,7 @@ select * from t;` } recordString += str } - expectRecordString := "2019-04-28 15:24:04.309074,405888132465033227,,,0,0.216905,0,0,0,0,0,0,0,,0,0,0,0,0,0,0.021,0,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,0,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,update t set i = 1;,select * from t;" + expectRecordString := "2019-04-28 15:24:04.309074,405888132465033227,,,0,0.216905,0,0,0,0,0,0,0,,0,0,0,0,0,0,0.38,0.021,0,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,0,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,update t set i = 1;,select * from t;" c.Assert(expectRecordString, Equals, recordString) // fix sql contain '# ' bug @@ -244,7 +244,9 @@ select 4;` # Time: 2020-02-16T19:00:00.000000+08:00 select 5; # Time: 2020-02-17T18:00:05.000000+08:00 -select 6;` +select 6; +# Time: 2020-04-15T18:00:05.299063744+08:00 +select 7;` fileName0 := "tidb-slow-2020-02-14T19-04-05.01.log" fileName1 := "tidb-slow-2020-02-15T19-04-05.01.log" @@ -333,6 +335,15 @@ select 6;` querys: []string{ "select 5;", "select 6;", + "select 7;", + }, + }, + { + startTime: "2020-04-15T18:00:05.299063744+08:00", + endTime: "2020-04-15T18:00:05.299063744+08:00", + files: []string{fileName3}, + querys: []string{ + "select 7;", }, }, } diff --git a/executor/split.go b/executor/split.go index a446f27e3ffa9..42da3237d45fb 100755 --- a/executor/split.go +++ b/executor/split.go @@ -162,7 +162,7 @@ func (e *SplitIndexRegionExec) getSplitIdxPhysicalKeysFromValueList(physicalID i keys = e.getSplitIdxPhysicalStartAndOtherIdxKeys(physicalID, keys) index := tables.NewIndex(physicalID, e.tableInfo, e.indexInfo) for _, v := range e.valueLists { - idxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, v, math.MinInt64, nil) + idxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, v, kv.IntHandle(math.MinInt64), nil) if err != nil { return nil, err } @@ -226,13 +226,13 @@ func (e *SplitIndexRegionExec) getSplitIdxPhysicalKeysFromBound(physicalID int64 keys = e.getSplitIdxPhysicalStartAndOtherIdxKeys(physicalID, keys) index := tables.NewIndex(physicalID, e.tableInfo, e.indexInfo) // Split index regions by lower, upper value and calculate the step by (upper - lower)/num. - lowerIdxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, e.lower, math.MinInt64, nil) + lowerIdxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, e.lower, kv.IntHandle(math.MinInt64), nil) if err != nil { return nil, err } // Use math.MinInt64 as handle_id for the upper index key to avoid affecting calculate split point. // If use math.MaxInt64 here, test of `TestSplitIndex` will report error. - upperIdxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, e.upper, math.MinInt64, nil) + upperIdxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, e.upper, kv.IntHandle(math.MinInt64), nil) if err != nil { return nil, err } @@ -482,7 +482,7 @@ func (e *SplitTableRegionExec) getSplitTableKeysFromValueList() ([][]byte, error func (e *SplitTableRegionExec) getSplitTablePhysicalKeysFromValueList(physicalID int64, keys [][]byte) [][]byte { recordPrefix := tablecodec.GenTableRecordPrefix(physicalID) for _, v := range e.valueLists { - key := tablecodec.EncodeRecordKey(recordPrefix, v[0].GetInt64()) + key := tablecodec.EncodeRecordKey(recordPrefix, kv.IntHandle(v[0].GetInt64())) keys = append(keys, key) } return keys @@ -534,7 +534,7 @@ func (e *SplitTableRegionExec) calculateBoundValue() (lowerValue int64, step int if upperRecordID <= lowerRecordID { return 0, 0, errors.Errorf("Split table `%s` region lower value %v should less than the upper value %v", e.tableInfo.Name, lowerRecordID, upperRecordID) } - step = int64(uint64(upperRecordID-lowerRecordID) / uint64(e.num)) + step = int64((upperRecordID - lowerRecordID) / uint64(e.num)) lowerValue = int64(lowerRecordID) } else { lowerRecordID := e.lower.GetInt64() @@ -560,7 +560,7 @@ func (e *SplitTableRegionExec) getSplitTablePhysicalKeysFromBound(physicalID, lo recordID := low for i := 1; i < e.num; i++ { recordID += step - key := tablecodec.EncodeRecordKey(recordPrefix, recordID) + key := tablecodec.EncodeRecordKey(recordPrefix, kv.IntHandle(recordID)) keys = append(keys, key) } return keys diff --git a/executor/split_test.go b/executor/split_test.go index 3ce0c3a9d6866..e2b89ae31b9eb 100644 --- a/executor/split_test.go +++ b/executor/split_test.go @@ -22,6 +22,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -160,13 +161,13 @@ func (s *testSplitIndex) TestSplitIndex(c *C) { index := tables.NewIndex(tbInfo.ID, tbInfo, idxInfo) for _, ca := range cases { // test for minInt64 handle - idxValue, _, err := index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(ca.value)}, math.MinInt64, nil) + idxValue, _, err := index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(ca.value)}, kv.IntHandle(math.MinInt64), nil) c.Assert(err, IsNil) idx := searchLessEqualIdx(valueList, idxValue) c.Assert(idx, Equals, ca.lessEqualIdx, Commentf("%#v", ca)) // Test for max int64 handle. - idxValue, _, err = index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(ca.value)}, math.MaxInt64, nil) + idxValue, _, err = index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(ca.value)}, kv.IntHandle(math.MaxInt64), nil) c.Assert(err, IsNil) idx = searchLessEqualIdx(valueList, idxValue) c.Assert(idx, Equals, ca.lessEqualIdx, Commentf("%#v", ca)) @@ -207,13 +208,13 @@ func (s *testSplitIndex) TestSplitIndex(c *C) { for _, ca := range cases2 { // test for minInt64 handle - idxValue, _, err := index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(ca.value)}, math.MinInt64, nil) + idxValue, _, err := index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(ca.value)}, kv.IntHandle(math.MinInt64), nil) c.Assert(err, IsNil) idx := searchLessEqualIdx(valueList, idxValue) c.Assert(idx, Equals, ca.lessEqualIdx, Commentf("%#v", ca)) // Test for max int64 handle. - idxValue, _, err = index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(ca.value)}, math.MaxInt64, nil) + idxValue, _, err = index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(ca.value)}, kv.IntHandle(math.MaxInt64), nil) c.Assert(err, IsNil) idx = searchLessEqualIdx(valueList, idxValue) c.Assert(idx, Equals, ca.lessEqualIdx, Commentf("%#v", ca)) @@ -264,13 +265,13 @@ func (s *testSplitIndex) TestSplitIndex(c *C) { for _, ca := range cases3 { value := types.NewTime(ca.value, mysql.TypeTimestamp, types.DefaultFsp) // test for min int64 handle - idxValue, _, err := index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(value)}, math.MinInt64, nil) + idxValue, _, err := index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(value)}, kv.IntHandle(math.MinInt64), nil) c.Assert(err, IsNil) idx := searchLessEqualIdx(valueList, idxValue) c.Assert(idx, Equals, ca.lessEqualIdx, Commentf("%#v", ca)) // Test for max int64 handle. - idxValue, _, err = index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(value)}, math.MaxInt64, nil) + idxValue, _, err = index.GenIndexKey(ctx.GetSessionVars().StmtCtx, []types.Datum{types.NewDatum(value)}, kv.IntHandle(math.MaxInt64), nil) c.Assert(err, IsNil) idx = searchLessEqualIdx(valueList, idxValue) c.Assert(idx, Equals, ca.lessEqualIdx, Commentf("%#v", ca)) @@ -345,7 +346,7 @@ func (s *testSplitIndex) TestSplitTable(c *C) { recordPrefix := tablecodec.GenTableRecordPrefix(e.tableInfo.ID) for _, ca := range cases { // test for minInt64 handle - key := tablecodec.EncodeRecordKey(recordPrefix, int64(ca.value)) + key := tablecodec.EncodeRecordKey(recordPrefix, kv.IntHandle(ca.value)) c.Assert(err, IsNil) idx := searchLessEqualIdx(valueList, key) c.Assert(idx, Equals, ca.lessEqualIdx, Commentf("%#v", ca)) diff --git a/executor/union_scan.go b/executor/union_scan.go index dd5a9ae589e15..3425356541e52 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -17,9 +17,9 @@ import ( "context" "sync" - "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -44,8 +44,8 @@ func (udb *DirtyDB) GetDirtyTable(tid int64) *DirtyTable { if !ok { dt = &DirtyTable{ tid: tid, - addedRows: make(map[int64]struct{}), - deletedRows: make(map[int64]struct{}), + addedRows: kv.NewHandleMap(), + deletedRows: kv.NewHandleMap(), } udb.tables[tid] = dt } @@ -58,24 +58,24 @@ type DirtyTable struct { tid int64 // addedRows ... // the key is handle. - addedRows map[int64]struct{} - deletedRows map[int64]struct{} + addedRows *kv.HandleMap + deletedRows *kv.HandleMap } // AddRow adds a row to the DirtyDB. -func (dt *DirtyTable) AddRow(handle int64) { - dt.addedRows[handle] = struct{}{} +func (dt *DirtyTable) AddRow(handle kv.Handle) { + dt.addedRows.Set(handle, true) } // DeleteRow deletes a row from the DirtyDB. -func (dt *DirtyTable) DeleteRow(handle int64) { - delete(dt.addedRows, handle) - dt.deletedRows[handle] = struct{}{} +func (dt *DirtyTable) DeleteRow(handle kv.Handle) { + dt.addedRows.Delete(handle) + dt.deletedRows.Set(handle, true) } // IsEmpty checks whether the table is empty. func (dt *DirtyTable) IsEmpty() bool { - return len(dt.addedRows)+len(dt.deletedRows) == 0 + return dt.addedRows.Len()+dt.deletedRows.Len() == 0 } // GetDirtyDB returns the DirtyDB bind to the context. @@ -244,11 +244,11 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) ([]types.Datum, err } iter := chunk.NewIterator4Chunk(us.snapshotChunkBuffer) for row := iter.Begin(); row != iter.End(); row = iter.Next() { - snapshotHandle := row.GetInt64(us.belowHandleIndex) - if _, ok := us.dirty.deletedRows[snapshotHandle]; ok { + snapshotHandle := kv.IntHandle(row.GetInt64(us.belowHandleIndex)) + if _, ok := us.dirty.deletedRows.Get(snapshotHandle); ok { continue } - if _, ok := us.dirty.addedRows[snapshotHandle]; ok { + if _, ok := us.dirty.addedRows.Get(snapshotHandle); ok { // If src handle appears in added rows, it means there is conflict and the transaction will fail to // commit, but for simplicity, we don't handle it here. continue @@ -313,23 +313,3 @@ func (us *UnionScanExec) compare(a, b []types.Datum) (int, error) { } return cmp, nil } - -// Len implements sort.Interface interface. -func (us *UnionScanExec) Len() int { - return len(us.addedRows) -} - -// Less implements sort.Interface interface. -func (us *UnionScanExec) Less(i, j int) bool { - cmp, err := us.compare(us.addedRows[i], us.addedRows[j]) - if err != nil { - us.sortErr = errors.Trace(err) - return true - } - return cmp < 0 -} - -// Swap implements sort.Interface interface. -func (us *UnionScanExec) Swap(i, j int) { - us.addedRows[i], us.addedRows[j] = us.addedRows[j], us.addedRows[i] -} diff --git a/executor/union_scan_test.go b/executor/union_scan_test.go index 5834ca499c9e9..6cf82482d4baa 100644 --- a/executor/union_scan_test.go +++ b/executor/union_scan_test.go @@ -290,3 +290,41 @@ func (s *testSuite7) TestUnionScanForMemBufferReader(c *C) { tk.MustQuery("select * from t1 use index(idx2);").Check(testkit.Rows("1 2 1")) tk.MustExec("admin check table t1;") } + +func (s *testSuite7) TestForUpdateUntouchedIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + + checkFunc := func() { + tk.MustExec("begin") + tk.MustExec("insert into t values ('a', 1), ('b', 3), ('a', 2) on duplicate key update b = b + 1;") + tk.MustExec("commit") + tk.MustExec("admin check table t") + + // Test for autocommit + tk.MustExec("set autocommit=0") + tk.MustExec("insert into t values ('a', 1), ('b', 3), ('a', 2) on duplicate key update b = b + 1;") + tk.MustExec("set autocommit=1") + tk.MustExec("admin check table t") + } + + // Test for primary key. + tk.MustExec("create table t (a varchar(10) primary key,b int)") + checkFunc() + + // Test for unique key. + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a varchar(10),b int, unique index(a))") + checkFunc() + + // Test for on duplicate update also conflict too. + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int,b int, unique index(a))") + tk.MustExec("begin") + _, err := tk.Exec("insert into t values (1, 1), (2, 2), (1, 3) on duplicate key update a = a + 1;") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '2' for key 'a'") + tk.MustExec("commit") + tk.MustExec("admin check table t") +} diff --git a/executor/update.go b/executor/update.go index 58ee206219089..aeee211e4dbac 100644 --- a/executor/update.go +++ b/executor/update.go @@ -36,7 +36,7 @@ type UpdateExec struct { // updatedRowKeys is a map for unique (Table, handle) pair. // The value is true if the row is changed, or false otherwise - updatedRowKeys map[int64]map[int64]bool + updatedRowKeys map[int64]*kv.HandleMap tblID2table map[int64]table.Table matched uint64 // a counter of matched rows during update @@ -55,18 +55,18 @@ func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema, row, n return err } if e.updatedRowKeys == nil { - e.updatedRowKeys = make(map[int64]map[int64]bool) + e.updatedRowKeys = make(map[int64]*kv.HandleMap) } for _, content := range e.tblColPosInfos { tbl := e.tblID2table[content.TblID] if e.updatedRowKeys[content.TblID] == nil { - e.updatedRowKeys[content.TblID] = make(map[int64]bool) + e.updatedRowKeys[content.TblID] = kv.NewHandleMap() } handleDatum := row[content.HandleOrdinal] if e.canNotUpdate(handleDatum) { continue } - handle := row[content.HandleOrdinal].GetInt64() + handle := kv.IntHandle(row[content.HandleOrdinal].GetInt64()) oldData := row[content.Start:content.End] newTableData := newData[content.Start:content.End] updatable := false @@ -81,10 +81,13 @@ func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema, row, n // If there's nothing to update, we can just skip current row continue } - changed, ok := e.updatedRowKeys[content.TblID][handle] + var changed bool + v, ok := e.updatedRowKeys[content.TblID].Get(handle) if !ok { // Row is matched for the first time, increment `matched` counter e.matched++ + } else { + changed = v.(bool) } if changed { // Each matched row is updated once, even if it matches the conditions multiple times. @@ -92,9 +95,9 @@ func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema, row, n } // Update row - changed, _, _, err1 := updateRecord(ctx, e.ctx, handle, oldData, newTableData, flags, tbl, false, e.memTracker) + changed, err1 := updateRecord(ctx, e.ctx, handle, oldData, newTableData, flags, tbl, false, e.memTracker) if err1 == nil { - e.updatedRowKeys[content.TblID][handle] = changed + e.updatedRowKeys[content.TblID].Set(handle, changed) continue } diff --git a/executor/update_test.go b/executor/update_test.go index 257ebb2a9ae28..87ef2ffb11d99 100644 --- a/executor/update_test.go +++ b/executor/update_test.go @@ -23,16 +23,16 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" ) type testUpdateSuite struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - domain *domain.Domain + cluster cluster.Cluster + store kv.Storage + domain *domain.Domain *parser.Parser ctx *mock.Context } @@ -42,12 +42,15 @@ func (s *testUpdateSuite) SetUpSuite(c *C) { flag.Lookup("mockTikv") useMockTikv := *mockTikv if useMockTikv { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) s.store = store diff --git a/executor/window_test.go b/executor/window_test.go index 423dbb7bfb27e..a3a31ca315147 100644 --- a/executor/window_test.go +++ b/executor/window_test.go @@ -15,6 +15,7 @@ package executor_test import ( "fmt" + . "github.com/pingcap/check" "github.com/pingcap/tidb/util/testkit" ) @@ -328,4 +329,28 @@ func baseTestSlidingWindowFunctions(tk *testkit.TestKit) { result.Check(testkit.Rows("M 2", "F 2.5", "F 3.5", "F 4", "M 4.5", " 10.5", " 10.5")) result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") result.Check(testkit.Rows(" 10.5", " 10.5", "M 4", "F 3.5", "F 2.5", "F 2", "M 1.5")) + + // BIT_XOR ROWS + result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") + result.Check(testkit.Rows("M 1", "F 7", "F 1", "F 15", "M 1", " 11", " 0")) + result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") + result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") + result.Check(testkit.Rows("M 0", "F 1", "F 3", "F 1", "M 7", " 1", " 15")) + result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;") + result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;") + result.Check(testkit.Rows("M 3", "F 0", "F 4", "F 1", "M 11", " 0", " 0")) + + // BIT_XOR RANGE + result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") + result.Check(testkit.Rows("M 1", "F 7", "F 1", "F 5", "M 0", " 11", " 0")) + result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") + result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") + result.Check(testkit.Rows("M 0", "F 1", "F 3", "F 1", "M 7", " 0", " 10")) + result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") + result.Check(testkit.Rows("M 0", "F 4", "F 0", "F 2", "M 1", " 1", " 1")) + result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") + result.Check(testkit.Rows(" 1", " 1", "M 2", "F 0", "F 4", "F 0", "M 3")) } diff --git a/executor/write.go b/executor/write.go index f82ca040c50d3..2d3252f8a3a3f 100644 --- a/executor/write.go +++ b/executor/write.go @@ -21,15 +21,14 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" - "go.uber.org/zap" ) var ( @@ -45,11 +44,9 @@ var ( // Length of `oldData` and `newData` equals to length of `t.WritableCols()`. // The return values: // 1. changed (bool) : does the update really change the row values. e.g. update set i = 1 where i = 1; -// 2. handleChanged (bool) : is the handle changed after the update. -// 3. newHandle (int64) : if handleChanged == true, the newHandle means the new handle after update. -// 4. err (error) : error in the update. -func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData, newData []types.Datum, modified []bool, t table.Table, - onDup bool, memTracker *memory.Tracker) (bool, bool, int64, error) { +// 2. err (error) : error in the update. +func updateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, oldData, newData []types.Datum, modified []bool, t table.Table, + onDup bool, memTracker *memory.Tracker) (bool, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("executor.updateRecord", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -57,7 +54,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData } txn, err := sctx.Txn(false) if err != nil { - return false, false, 0, err + return false, err } memUsageOfTxnState := txn.Size() defer memTracker.Consume(int64(txn.Size() - memUsageOfTxnState)) @@ -66,7 +63,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData // onUpdateSpecified is for "UPDATE SET ts_field = old_value", the // timestamp field is explicitly set, but not changed in fact. onUpdateSpecified := make(map[int]bool) - var newHandle int64 + var newHandle kv.Handle // We can iterate on public columns not writable columns, // because all of them are sorted by their `Offset`, which @@ -78,7 +75,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData // Cast changed fields with respective columns. v, err := table.CastValue(sctx, newData[i], col.ToInfo()) if err != nil { - return false, false, 0, err + return false, err } newData[i] = v } @@ -88,7 +85,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData for i, col := range t.Cols() { var err error if newData[i], err = col.HandleBadNull(newData[i], sc); err != nil { - return false, false, 0, err + return false, err } } @@ -96,7 +93,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData for i, col := range t.Cols() { cmp, err := newData[i].CompareDatum(sc, &oldData[i]) if err != nil { - return false, false, 0, err + return false, err } if cmp != 0 { changed = true @@ -105,18 +102,18 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData if mysql.HasAutoIncrementFlag(col.Flag) { recordID, err := getAutoRecordID(newData[i], &col.FieldType, false) if err != nil { - return false, false, 0, err + return false, err } - if err = t.RebaseAutoID(sctx, recordID, true); err != nil { - return false, false, 0, err + if err = t.RebaseAutoID(sctx, recordID, true, autoid.RowIDAllocType); err != nil { + return false, err } } if col.IsPKHandleColumn(t.Meta()) { handleChanged = true - newHandle = newData[i].GetInt64() + newHandle = kv.IntHandle(newData[i].GetInt64()) // Rebase auto random id if the field is changed. if err := rebaseAutoRandomValue(sctx, t, &newData[i], col); err != nil { - return false, false, 0, err + return false, err } } } else { @@ -140,7 +137,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData if pt, ok := t.(table.PartitionedTable); ok { p, err := pt.GetPartitionByRow(sctx, oldData) if err != nil { - return false, false, 0, err + return false, err } physicalID = p.GetPhysicalID() } @@ -150,7 +147,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData if txnCtx.IsPessimistic { txnCtx.AddUnchangedRowKey(unchangedRowKey) } - return false, false, 0, nil + return false, nil } // 4. Fill values into on-update-now fields, only if they are really changed. @@ -160,7 +157,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData newData[i] = v modified[i] = true } else { - return false, false, 0, err + return false, err } } } @@ -172,21 +169,21 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData // If the new handle exists, this will avoid to remove the record. err = tables.CheckHandleExists(ctx, sctx, t, newHandle, newData) if err != nil { - return false, handleChanged, newHandle, err + return false, err } } if err = t.RemoveRecord(sctx, h, oldData); err != nil { - return false, false, 0, err + return false, err } // the `affectedRows` is increased when adding new record. if sc.DupKeyAsWarning { - newHandle, err = t.AddRecord(sctx, newData, table.IsUpdate, table.SkipHandleCheck, table.WithCtx(ctx)) + _, err = t.AddRecord(sctx, newData, table.IsUpdate, table.SkipHandleCheck, table.WithCtx(ctx)) } else { - newHandle, err = t.AddRecord(sctx, newData, table.IsUpdate, table.WithCtx(ctx)) + _, err = t.AddRecord(sctx, newData, table.IsUpdate, table.WithCtx(ctx)) } if err != nil { - return false, false, 0, err + return false, err } if onDup { sc.AddAffectedRows(1) @@ -194,7 +191,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData } else { // Update record to new value and update index. if err = t.UpdateRecord(sctx, h, oldData, newData, modified); err != nil { - return false, false, 0, err + return false, err } if onDup { sc.AddAffectedRows(2) @@ -205,7 +202,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData sc.AddUpdatedRows(1) sc.AddCopiedRows(1) - return true, handleChanged, newHandle, nil + return true, nil } func rebaseAutoRandomValue(sctx sessionctx.Context, t table.Table, newData *types.Datum, col *table.Column) error { @@ -227,6 +224,5 @@ func rebaseAutoRandomValue(sctx sessionctx.Context, t table.Table, newData *type // so we reset the error msg here, and wrap old err with errors.Wrap. func resetErrDataTooLong(colName string, rowIdx int, err error) error { newErr := types.ErrDataTooLong.GenWithStack("Data too long for column '%v' at row %v", colName, rowIdx) - logutil.BgLogger().Error("data too long for column", zap.String("colName", colName), zap.Int("rowIndex", rowIdx)) return newErr } diff --git a/executor/write_test.go b/executor/write_test.go index dedd50a3017cc..757bf1350b137 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -300,6 +300,13 @@ func (s *testSuite) TestInsert(c *C) { _, err = tk.Exec("replace into v values(1,2)") c.Assert(err.Error(), Equals, "replace into view v is not supported now.") tk.MustExec("drop view v") + + tk.MustExec("create sequence seq") + _, err = tk.Exec("insert into seq values()") + c.Assert(err.Error(), Equals, "insert into sequence seq is not supported now.") + _, err = tk.Exec("replace into seq values()") + c.Assert(err.Error(), Equals, "replace into sequence seq is not supported now.") + tk.MustExec("drop sequence seq") } func (s *testSuiteP2) TestMultiBatch(c *C) { @@ -1511,6 +1518,11 @@ func (s *testSuite8) TestUpdate(c *C) { c.Assert(err.Error(), Equals, core.ErrViewInvalid.GenWithStackByArgs("test", "v").Error()) tk.MustExec("drop view v") + tk.MustExec("create sequence seq") + _, err = tk.Exec("update seq set minvalue=1") + c.Assert(err.Error(), Equals, "update sequence seq is not supported now.") + tk.MustExec("drop sequence seq") + tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int, c int, d int, e int, index idx(a))") tk.MustExec("create table t2(a int, b int, c int)") @@ -1811,6 +1823,11 @@ func (s *testSuite) TestDelete(c *C) { _, err = tk.Exec("delete from v where name = 'aaa'") c.Assert(err.Error(), Equals, core.ErrViewInvalid.GenWithStackByArgs("test", "v").Error()) tk.MustExec("drop view v") + + tk.MustExec("create sequence seq") + _, err = tk.Exec("delete from seq") + c.Assert(err.Error(), Equals, "delete sequence seq is not supported now.") + tk.MustExec("drop sequence seq") } func (s *testSuite4) TestPartitionedTableDelete(c *C) { @@ -2165,6 +2182,7 @@ func (s *testSuite4) TestLoadDataEscape(c *C) { {nil, []byte("6\t\\r\\t\\n\\0\\Z\\b\n"), []string{"6|" + string([]byte{'\r', '\t', '\n', 0, 26, '\b'})}, nil, trivialMsg}, {nil, []byte("7\trtn0ZbN\n"), []string{"7|" + string([]byte{'r', 't', 'n', '0', 'Z', 'b', 'N'})}, nil, trivialMsg}, {nil, []byte("8\trtn0Zb\\N\n"), []string{"8|" + string([]byte{'r', 't', 'n', '0', 'Z', 'b', 'N'})}, nil, trivialMsg}, + {nil, []byte("9\ttab\\ tab\n"), []string{"9|tab tab"}, nil, trivialMsg}, } deleteSQL := "delete from load_data_test" selectSQL := "select * from load_data_test;" @@ -2555,7 +2573,7 @@ func (s *testSuite7) TestReplaceLog(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(1), 1) + _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(1), kv.IntHandle(1)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/expression/aggregation/aggregation.go b/expression/aggregation/aggregation.go index 4fed3d8e4c8a7..5a47f6ac01777 100644 --- a/expression/aggregation/aggregation.go +++ b/expression/aggregation/aggregation.go @@ -15,6 +15,7 @@ package aggregation import ( "bytes" + "strings" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" @@ -190,12 +191,15 @@ func IsAllFirstRow(aggFuncs []*AggFuncDesc) bool { // CheckAggPushDown checks whether an agg function can be pushed to storage. func CheckAggPushDown(aggFunc *AggFuncDesc, storeType kv.StoreType) bool { + ret := true switch storeType { case kv.TiFlash: - return CheckAggPushFlash(aggFunc) - default: - return true + ret = CheckAggPushFlash(aggFunc) + } + if ret { + ret = expression.IsPushDownEnabled(strings.ToLower(aggFunc.Name), storeType) } + return ret } // CheckAggPushFlash checks whether an agg function can be pushed to flash storage. diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index e3b77211b1636..58b2e98502a86 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -14,6 +14,7 @@ package aggregation import ( + "bytes" "fmt" "math" "strconv" @@ -44,6 +45,23 @@ func NewAggFuncDesc(ctx sessionctx.Context, name string, args []expression.Expre return &AggFuncDesc{baseFuncDesc: b, HasDistinct: hasDistinct}, nil } +// String implements the fmt.Stringer interface. +func (a *AggFuncDesc) String() string { + buffer := bytes.NewBufferString(a.Name) + buffer.WriteString("(") + if a.HasDistinct { + buffer.WriteString("distinct ") + } + for i, arg := range a.Args { + buffer.WriteString(arg.String()) + if i+1 != len(a.Args) { + buffer.WriteString(", ") + } + } + buffer.WriteString(")") + return buffer.String() +} + // Equal checks whether two aggregation function signatures are equal. func (a *AggFuncDesc) Equal(ctx sessionctx.Context, other *AggFuncDesc) bool { if a.HasDistinct != other.HasDistinct { diff --git a/expression/bench_test.go b/expression/bench_test.go index e350a49181b97..133639f5486d9 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -190,8 +190,8 @@ func BenchmarkVectorizedExecute(b *testing.B) { func BenchmarkScalarFunctionClone(b *testing.B) { col := &Column{RetType: types.NewFieldType(mysql.TypeLonglong)} - con1 := One.Clone() - con2 := Zero.Clone() + con1 := NewOne() + con2 := NewZero() add := NewFunctionInternal(mock.NewContext(), ast.Plus, types.NewFieldType(mysql.TypeLonglong), col, con1) sub := NewFunctionInternal(mock.NewContext(), ast.Plus, types.NewFieldType(mysql.TypeLonglong), add, con2) b.ResetTimer() @@ -1308,12 +1308,12 @@ func testVectorizedBuiltinFunc(c *C, vecExprCases vecExprBenchCases) { types.NewTimeDatum(types.NewTime(types.FromGoTime(testTime), mysql.TypeTimestamp, 6)), types.NewDurationDatum(types.ZeroDuration), types.NewStringDatum("{}"), - types.NewBinaryLiteralDatum(types.BinaryLiteral([]byte{1})), + types.NewBinaryLiteralDatum([]byte{1}), types.NewBytesDatum([]byte{'b'}), types.NewFloat32Datum(1.1), types.NewFloat64Datum(2.1), types.NewUintDatum(100), - types.NewMysqlBitDatum(types.BinaryLiteral([]byte{1})), + types.NewMysqlBitDatum([]byte{1}), types.NewMysqlEnumDatum(types.Enum{Name: "n", Value: 2}), } } @@ -1527,12 +1527,12 @@ func benchmarkVectorizedBuiltinFunc(b *testing.B, vecExprCases vecExprBenchCases types.NewTimeDatum(types.NewTime(types.FromGoTime(testTime), mysql.TypeTimestamp, 6)), types.NewDurationDatum(types.ZeroDuration), types.NewStringDatum("{}"), - types.NewBinaryLiteralDatum(types.BinaryLiteral([]byte{1})), + types.NewBinaryLiteralDatum([]byte{1}), types.NewBytesDatum([]byte{'b'}), types.NewFloat32Datum(1.1), types.NewFloat64Datum(2.1), types.NewUintDatum(100), - types.NewMysqlBitDatum(types.BinaryLiteral([]byte{1})), + types.NewMysqlBitDatum([]byte{1}), types.NewMysqlEnumDatum(types.Enum{Name: "n", Value: 2}), } } @@ -1751,7 +1751,7 @@ func genVecEvalBool(numCols int, colTypes, eTypes []types.EvalType) (CNFExprs, * func generateRandomSel() []int { randGen := newDefaultRandGen() - randGen.Seed(int64(time.Now().UnixNano())) + randGen.Seed(time.Now().UnixNano()) var sel []int count := 0 // Use constant 256 to make it faster to generate randomly arranged sel slices @@ -1796,21 +1796,6 @@ func (s *testVectorizeSuite2) TestVecEvalBool(c *C) { } } -func (s *testVectorizeSuite2) TestVecToBool(c *C) { - ctx := mock.NewContext() - buf := chunk.NewColumn(eType2FieldType(types.ETString), 2) - buf.ReserveString(1) - buf.AppendString("999999999999999999923") - c.Assert(toBool(ctx.GetSessionVars().StmtCtx, types.ETString, buf, []int{0, 1}, []int8{0, 0}), NotNil) - buf.ReserveString(1) - buf.AppendString("23") - c.Assert(toBool(ctx.GetSessionVars().StmtCtx, types.ETString, buf, []int{0, 1}, []int8{0, 0}), IsNil) - buf.ReserveString(2) - buf.AppendString("999999999999999999923") - buf.AppendString("23") - c.Assert(toBool(ctx.GetSessionVars().StmtCtx, types.ETString, buf, []int{0, 1}, []int8{0, 0}), NotNil) -} - func BenchmarkVecEvalBool(b *testing.B) { ctx := mock.NewContext() selected := make([]bool, 0, 1024) diff --git a/expression/builtin.go b/expression/builtin.go index 193abab63e545..feaf74b8f6698 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -86,10 +86,13 @@ func (b *baseBuiltinFunc) collator() collate.Collator { return b.ctor } -func newBaseBuiltinFunc(ctx sessionctx.Context, args []Expression) baseBuiltinFunc { +func newBaseBuiltinFunc(ctx sessionctx.Context, funcName string, args []Expression) (baseBuiltinFunc, error) { if ctx == nil { panic("ctx should not be nil") } + if err := checkIllegalMixCollation(funcName, args); err != nil { + return baseBuiltinFunc{}, err + } derivedCharset, derivedCollate, derivedFlen := DeriveCollationFromExprs(ctx, args...) bf := baseBuiltinFunc{ bufAllocator: newLocalSliceBuffer(len(args)), @@ -102,13 +105,30 @@ func newBaseBuiltinFunc(ctx sessionctx.Context, args []Expression) baseBuiltinFu } bf.SetCharsetAndCollation(derivedCharset, derivedCollate, derivedFlen) bf.setCollator(collate.GetCollator(derivedCollate)) - return bf + return bf, nil +} + +func checkIllegalMixCollation(funcName string, args []Expression) error { + firstExplicitCollation := "" + for _, arg := range args { + if arg.GetType().EvalType() != types.ETString { + continue + } + if arg.Coercibility() == CoercibilityExplicit { + if firstExplicitCollation == "" { + firstExplicitCollation = arg.GetType().Collate + } else if firstExplicitCollation != arg.GetType().Collate { + return collate.ErrIllegalMixCollation.GenWithStackByArgs(firstExplicitCollation, "EXPLICIT", arg.GetType().Collate, "EXPLICIT", funcName) + } + } + } + return nil } // newBaseBuiltinFuncWithTp creates a built-in function signature with specified types of arguments and the return type of the function. // argTps indicates the types of the args, retType indicates the return type of the built-in function. // Every built-in function needs determined argTps and retType when we create it. -func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, args []Expression, retType types.EvalType, argTps ...types.EvalType) (bf baseBuiltinFunc) { +func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, funcName string, args []Expression, retType types.EvalType, argTps ...types.EvalType) (bf baseBuiltinFunc, err error) { if len(args) != len(argTps) { panic("unexpected length of args and argTps") } @@ -137,6 +157,10 @@ func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, args []Expression, retType } } + if err = checkIllegalMixCollation(funcName, args); err != nil { + return + } + // derive collation information for string function, and we must do it // before doing implicit cast. derivedCharset, derivedCollate, derivedFlen := DeriveCollationFromExprs(ctx, args...) @@ -216,7 +240,7 @@ func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, args []Expression, retType } bf.SetCharsetAndCollation(derivedCharset, derivedCollate, derivedFlen) bf.setCollator(collate.GetCollator(derivedCollate)) - return bf + return bf, nil } func (b *baseBuiltinFunc) getArgs() []Expression { diff --git a/expression/builtin_arithmetic.go b/expression/builtin_arithmetic.go index 8b12367c855c9..7bd59ec95347a 100644 --- a/expression/builtin_arithmetic.go +++ b/expression/builtin_arithmetic.go @@ -160,19 +160,28 @@ func (c *arithmeticPlusFunctionClass) getFunction(ctx sessionctx.Context, args [ lhsTp, rhsTp := args[0].GetType(), args[1].GetType() lhsEvalTp, rhsEvalTp := numericContextResultType(lhsTp), numericContextResultType(rhsTp) if lhsEvalTp == types.ETReal || rhsEvalTp == types.ETReal { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } setFlenDecimal4RealOrDecimal(bf.tp, args[0].GetType(), args[1].GetType(), true, false) sig := &builtinArithmeticPlusRealSig{bf} sig.setPbCode(tipb.ScalarFuncSig_PlusReal) return sig, nil } else if lhsEvalTp == types.ETDecimal || rhsEvalTp == types.ETDecimal { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) + if err != nil { + return nil, err + } setFlenDecimal4RealOrDecimal(bf.tp, args[0].GetType(), args[1].GetType(), false, false) sig := &builtinArithmeticPlusDecimalSig{bf} sig.setPbCode(tipb.ScalarFuncSig_PlusDecimal) return sig, nil } else { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } if mysql.HasUnsignedFlag(args[0].GetType().Flag) || mysql.HasUnsignedFlag(args[1].GetType().Flag) { bf.tp.Flag |= mysql.UnsignedFlag } @@ -301,20 +310,29 @@ func (c *arithmeticMinusFunctionClass) getFunction(ctx sessionctx.Context, args lhsTp, rhsTp := args[0].GetType(), args[1].GetType() lhsEvalTp, rhsEvalTp := numericContextResultType(lhsTp), numericContextResultType(rhsTp) if lhsEvalTp == types.ETReal || rhsEvalTp == types.ETReal { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } setFlenDecimal4RealOrDecimal(bf.tp, args[0].GetType(), args[1].GetType(), true, false) sig := &builtinArithmeticMinusRealSig{bf} sig.setPbCode(tipb.ScalarFuncSig_MinusReal) return sig, nil } else if lhsEvalTp == types.ETDecimal || rhsEvalTp == types.ETDecimal { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) + if err != nil { + return nil, err + } setFlenDecimal4RealOrDecimal(bf.tp, args[0].GetType(), args[1].GetType(), false, false) sig := &builtinArithmeticMinusDecimalSig{bf} sig.setPbCode(tipb.ScalarFuncSig_MinusDecimal) return sig, nil } else { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } setFlenDecimal4Int(bf.tp, args[0].GetType(), args[1].GetType()) if (mysql.HasUnsignedFlag(args[0].GetType().Flag) || mysql.HasUnsignedFlag(args[1].GetType().Flag)) && !ctx.GetSessionVars().SQLMode.HasNoUnsignedSubtractionMode() { bf.tp.Flag |= mysql.UnsignedFlag @@ -450,19 +468,28 @@ func (c *arithmeticMultiplyFunctionClass) getFunction(ctx sessionctx.Context, ar lhsTp, rhsTp := args[0].GetType(), args[1].GetType() lhsEvalTp, rhsEvalTp := numericContextResultType(lhsTp), numericContextResultType(rhsTp) if lhsEvalTp == types.ETReal || rhsEvalTp == types.ETReal { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } setFlenDecimal4RealOrDecimal(bf.tp, args[0].GetType(), args[1].GetType(), true, true) sig := &builtinArithmeticMultiplyRealSig{bf} sig.setPbCode(tipb.ScalarFuncSig_MultiplyReal) return sig, nil } else if lhsEvalTp == types.ETDecimal || rhsEvalTp == types.ETDecimal { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) + if err != nil { + return nil, err + } setFlenDecimal4RealOrDecimal(bf.tp, args[0].GetType(), args[1].GetType(), false, true) sig := &builtinArithmeticMultiplyDecimalSig{bf} sig.setPbCode(tipb.ScalarFuncSig_MultiplyDecimal) return sig, nil } else { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } if mysql.HasUnsignedFlag(lhsTp.Flag) || mysql.HasUnsignedFlag(rhsTp.Flag) { bf.tp.Flag |= mysql.UnsignedFlag setFlenDecimal4Int(bf.tp, args[0].GetType(), args[1].GetType()) @@ -587,13 +614,19 @@ func (c *arithmeticDivideFunctionClass) getFunction(ctx sessionctx.Context, args lhsTp, rhsTp := args[0].GetType(), args[1].GetType() lhsEvalTp, rhsEvalTp := numericContextResultType(lhsTp), numericContextResultType(rhsTp) if lhsEvalTp == types.ETReal || rhsEvalTp == types.ETReal { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } c.setType4DivReal(bf.tp) sig := &builtinArithmeticDivideRealSig{bf} sig.setPbCode(tipb.ScalarFuncSig_DivideReal) return sig, nil } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) + if err != nil { + return nil, err + } c.setType4DivDecimal(bf.tp, lhsTp, rhsTp) sig := &builtinArithmeticDivideDecimalSig{bf} sig.setPbCode(tipb.ScalarFuncSig_DivideDecimal) @@ -674,7 +707,10 @@ func (c *arithmeticIntDivideFunctionClass) getFunction(ctx sessionctx.Context, a lhsTp, rhsTp := args[0].GetType(), args[1].GetType() lhsEvalTp, rhsEvalTp := numericContextResultType(lhsTp), numericContextResultType(rhsTp) if lhsEvalTp == types.ETInt && rhsEvalTp == types.ETInt { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } if mysql.HasUnsignedFlag(lhsTp.Flag) || mysql.HasUnsignedFlag(rhsTp.Flag) { bf.tp.Flag |= mysql.UnsignedFlag } @@ -682,7 +718,10 @@ func (c *arithmeticIntDivideFunctionClass) getFunction(ctx sessionctx.Context, a sig.setPbCode(tipb.ScalarFuncSig_IntDivideInt) return sig, nil } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDecimal, types.ETDecimal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDecimal, types.ETDecimal) + if err != nil { + return nil, err + } if mysql.HasUnsignedFlag(lhsTp.Flag) || mysql.HasUnsignedFlag(rhsTp.Flag) { bf.tp.Flag |= mysql.UnsignedFlag } @@ -839,7 +878,10 @@ func (c *arithmeticModFunctionClass) getFunction(ctx sessionctx.Context, args [] lhsTp, rhsTp := args[0].GetType(), args[1].GetType() lhsEvalTp, rhsEvalTp := numericContextResultType(lhsTp), numericContextResultType(rhsTp) if lhsEvalTp == types.ETReal || rhsEvalTp == types.ETReal { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } c.setType4ModRealOrDecimal(bf.tp, lhsTp, rhsTp, false) if mysql.HasUnsignedFlag(lhsTp.Flag) { bf.tp.Flag |= mysql.UnsignedFlag @@ -848,7 +890,10 @@ func (c *arithmeticModFunctionClass) getFunction(ctx sessionctx.Context, args [] sig.setPbCode(tipb.ScalarFuncSig_ModReal) return sig, nil } else if lhsEvalTp == types.ETDecimal || rhsEvalTp == types.ETDecimal { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDecimal, types.ETDecimal, types.ETDecimal) + if err != nil { + return nil, err + } c.setType4ModRealOrDecimal(bf.tp, lhsTp, rhsTp, true) if mysql.HasUnsignedFlag(lhsTp.Flag) { bf.tp.Flag |= mysql.UnsignedFlag @@ -857,7 +902,10 @@ func (c *arithmeticModFunctionClass) getFunction(ctx sessionctx.Context, args [] sig.setPbCode(tipb.ScalarFuncSig_ModDecimal) return sig, nil } else { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } if mysql.HasUnsignedFlag(lhsTp.Flag) { bf.tp.Flag |= mysql.UnsignedFlag } diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index d3933f10a865c..bf80d05178b7d 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -118,7 +118,11 @@ func (c *castAsIntFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinCastFunc(newBaseBuiltinFunc(ctx, args), ctx.Value(inUnionCastContext) != nil) + b, err := newBaseBuiltinFunc(ctx, c.funcName, args) + if err != nil { + return nil, err + } + bf := newBaseBuiltinCastFunc(b, ctx.Value(inUnionCastContext) != nil) bf.tp = c.tp if args[0].GetType().Hybrid() || IsBinaryLiteral(args[0]) { sig = &builtinCastIntAsIntSig{bf} @@ -164,7 +168,11 @@ func (c *castAsRealFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinCastFunc(newBaseBuiltinFunc(ctx, args), ctx.Value(inUnionCastContext) != nil) + b, err := newBaseBuiltinFunc(ctx, c.funcName, args) + if err != nil { + return nil, err + } + bf := newBaseBuiltinCastFunc(b, ctx.Value(inUnionCastContext) != nil) bf.tp = c.tp if IsBinaryLiteral(args[0]) { sig = &builtinCastRealAsRealSig{bf} @@ -215,7 +223,11 @@ func (c *castAsDecimalFunctionClass) getFunction(ctx sessionctx.Context, args [] if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinCastFunc(newBaseBuiltinFunc(ctx, args), ctx.Value(inUnionCastContext) != nil) + b, err := newBaseBuiltinFunc(ctx, c.funcName, args) + if err != nil { + return nil, err + } + bf := newBaseBuiltinCastFunc(b, ctx.Value(inUnionCastContext) != nil) bf.tp = c.tp if IsBinaryLiteral(args[0]) { sig = &builtinCastDecimalAsDecimalSig{bf} @@ -266,7 +278,10 @@ func (c *castAsStringFunctionClass) getFunction(ctx sessionctx.Context, args []E if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFunc(ctx, args) + bf, err := newBaseBuiltinFunc(ctx, c.funcName, args) + if err != nil { + return nil, err + } bf.tp = c.tp if args[0].GetType().Hybrid() || IsBinaryLiteral(args[0]) { sig = &builtinCastStringAsStringSig{bf} @@ -312,7 +327,10 @@ func (c *castAsTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFunc(ctx, args) + bf, err := newBaseBuiltinFunc(ctx, c.funcName, args) + if err != nil { + return nil, err + } bf.tp = c.tp argTp := args[0].GetType().EvalType() switch argTp { @@ -353,7 +371,10 @@ func (c *castAsDurationFunctionClass) getFunction(ctx sessionctx.Context, args [ if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFunc(ctx, args) + bf, err := newBaseBuiltinFunc(ctx, c.funcName, args) + if err != nil { + return nil, err + } bf.tp = c.tp argTp := args[0].GetType().EvalType() switch argTp { @@ -394,7 +415,10 @@ func (c *castAsJSONFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFunc(ctx, args) + bf, err := newBaseBuiltinFunc(ctx, c.funcName, args) + if err != nil { + return nil, err + } bf.tp = c.tp argTp := args[0].GetType().EvalType() switch argTp { diff --git a/expression/builtin_cast_bench_test.go b/expression/builtin_cast_bench_test.go index e68d0bfdf175d..90f19dba195ba 100644 --- a/expression/builtin_cast_bench_test.go +++ b/expression/builtin_cast_bench_test.go @@ -25,7 +25,10 @@ import ( func genCastIntAsInt() (*builtinCastIntAsIntSig, *chunk.Chunk, *chunk.Column) { col := &Column{RetType: types.NewFieldType(mysql.TypeLonglong), Index: 0} - baseFunc := newBaseBuiltinFunc(mock.NewContext(), []Expression{col}) + baseFunc, err := newBaseBuiltinFunc(mock.NewContext(), "", []Expression{col}) + if err != nil { + panic(err) + } baseCast := newBaseBuiltinCastFunc(baseFunc, false) cast := &builtinCastIntAsIntSig{baseCast} input := chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeLonglong)}, 1024) diff --git a/expression/builtin_cast_test.go b/expression/builtin_cast_test.go index 97d0c6caa5b2e..1789febe31e99 100644 --- a/expression/builtin_cast_test.go +++ b/expression/builtin_cast_test.go @@ -167,7 +167,7 @@ func (s *testEvaluatorSuite) TestCastXXX(c *C) { // cast('18446744073709551616' as signed); mask := ^mysql.UnsignedFlag - tp1.Flag &= uint(mask) + tp1.Flag &= mask f = BuildCastFunction(ctx, &Constant{Value: types.NewDatum("18446744073709551616"), RetType: types.NewFieldType(mysql.TypeString)}, tp1) res, err = f.Eval(chunk.Row{}) c.Assert(err, IsNil) @@ -270,7 +270,7 @@ func (s *testEvaluatorSuite) TestCastXXX(c *C) { var ( year, month, day = time.Now().In(time.UTC).Date() curDateInt = int64(year*10000 + int(month)*100 + day) - curTimeInt = int64(curDateInt*1000000 + 125959) + curTimeInt = curDateInt*1000000 + 125959 curTimeWithFspReal = float64(curTimeInt) + 0.555 curTimeString = fmt.Sprintf("%4d-%02d-%02d 12:59:59", year, int(month), day) curTimeWithFspString = fmt.Sprintf("%4d-%02d-%02d 12:59:59.555000", year, int(month), day) @@ -281,12 +281,12 @@ var ( // timeWithFspDatum indicates datetime "curYear-curMonth-curDay 12:59:59.555000". timeWithFspDatum = types.NewDatum(tmWithFsp) duration = types.Duration{ - Duration: time.Duration(12*time.Hour + 59*time.Minute + 59*time.Second), + Duration: 12*time.Hour + 59*time.Minute + 59*time.Second, Fsp: types.DefaultFsp} // durationDatum indicates duration "12:59:59". durationDatum = types.NewDatum(duration) durationWithFsp = types.Duration{ - Duration: time.Duration(12*time.Hour + 59*time.Minute + 59*time.Second + 555*time.Millisecond), + Duration: 12*time.Hour + 59*time.Minute + 59*time.Second + 555*time.Millisecond, Fsp: 3} // durationWithFspDatum indicates duration "12:59:59.555" durationWithFspDatum = types.NewDatum(durationWithFsp) @@ -355,7 +355,9 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { } for i, t := range castToDecCases { args := []Expression{t.before} - decFunc := newBaseBuiltinCastFunc(newBaseBuiltinFunc(ctx, args), false) + b, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) + decFunc := newBaseBuiltinCastFunc(b, false) decFunc.tp = types.NewFieldType(mysql.TypeNewDecimal) switch i { case 0: @@ -439,7 +441,9 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { args := []Expression{t.before} tp := types.NewFieldType(mysql.TypeNewDecimal) tp.Flen, tp.Decimal = t.flen, t.decimal - decFunc := newBaseBuiltinCastFunc(newBaseBuiltinFunc(ctx, args), false) + b, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) + decFunc := newBaseBuiltinCastFunc(b, false) decFunc.tp = tp switch i { case 0: @@ -507,7 +511,9 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { } for i, t := range castToIntCases { args := []Expression{t.before} - intFunc := newBaseBuiltinCastFunc(newBaseBuiltinFunc(ctx, args), false) + b, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) + intFunc := newBaseBuiltinCastFunc(b, false) switch i { case 0: sig = &builtinCastStringAsIntSig{intFunc} @@ -573,7 +579,9 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { } for i, t := range castToRealCases { args := []Expression{t.before} - realFunc := newBaseBuiltinCastFunc(newBaseBuiltinFunc(ctx, args), false) + b, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) + realFunc := newBaseBuiltinCastFunc(b, false) switch i { case 0: sig = &builtinCastStringAsRealSig{realFunc} @@ -647,7 +655,8 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { tp := types.NewFieldType(mysql.TypeVarString) tp.Charset = charset.CharsetBin args := []Expression{t.before} - stringFunc := newBaseBuiltinFunc(ctx, args) + stringFunc, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) stringFunc.tp = tp switch i { case 0: @@ -725,7 +734,8 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { args := []Expression{t.before} tp := types.NewFieldType(mysql.TypeVarString) tp.Flen, tp.Charset = t.flen, charset.CharsetBin - stringFunc := newBaseBuiltinFunc(ctx, args) + stringFunc, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) stringFunc.tp = tp switch i { case 0: @@ -800,7 +810,8 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { args := []Expression{t.before} tp := types.NewFieldType(mysql.TypeDatetime) tp.Decimal = int(types.DefaultFsp) - timeFunc := newBaseBuiltinFunc(ctx, args) + timeFunc, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) timeFunc.tp = tp switch i { case 0: @@ -884,7 +895,8 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { args := []Expression{t.before} tp := types.NewFieldType(t.tp) tp.Decimal = int(t.fsp) - timeFunc := newBaseBuiltinFunc(ctx, args) + timeFunc, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) timeFunc.tp = tp switch i { case 0: @@ -965,7 +977,8 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { args := []Expression{t.before} tp := types.NewFieldType(mysql.TypeDuration) tp.Decimal = int(types.DefaultFsp) - durationFunc := newBaseBuiltinFunc(ctx, args) + durationFunc, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) durationFunc.tp = tp switch i { case 0: @@ -1042,7 +1055,8 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { args := []Expression{t.before} tp := types.NewFieldType(mysql.TypeDuration) tp.Decimal = t.fsp - durationFunc := newBaseBuiltinFunc(ctx, args) + durationFunc, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) durationFunc.tp = tp switch i { case 0: @@ -1074,7 +1088,8 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { // null case args := []Expression{&Column{RetType: types.NewFieldType(mysql.TypeDouble), Index: 0}} row := chunk.MutRowFromDatums([]types.Datum{types.NewDatum(nil)}) - bf := newBaseBuiltinFunc(ctx, args) + bf, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) bf.tp = types.NewFieldType(mysql.TypeVarString) sig = &builtinCastRealAsStringSig{bf} sRes, isNull, err := sig.evalString(row.ToRow()) @@ -1084,7 +1099,9 @@ func (s *testEvaluatorSuite) TestCastFuncSig(c *C) { // test hybridType case. args = []Expression{&Constant{Value: types.NewDatum(types.Enum{Name: "a", Value: 0}), RetType: types.NewFieldType(mysql.TypeEnum)}} - sig = &builtinCastStringAsIntSig{newBaseBuiltinCastFunc(newBaseBuiltinFunc(ctx, args), false)} + b, err := newBaseBuiltinFunc(ctx, "", args) + c.Assert(err, IsNil) + sig = &builtinCastStringAsIntSig{newBaseBuiltinCastFunc(b, false)} iRes, isNull, err := sig.evalInt(chunk.Row{}) c.Assert(isNull, Equals, false) c.Assert(err, IsNil) @@ -1100,7 +1117,9 @@ func (s *testEvaluatorSuite) TestCastJSONAsDecimalSig(c *C) { }() col := &Column{RetType: types.NewFieldType(mysql.TypeJSON), Index: 0} - decFunc := newBaseBuiltinCastFunc(newBaseBuiltinFunc(ctx, []Expression{col}), false) + b, err := newBaseBuiltinFunc(ctx, "", []Expression{col}) + c.Assert(err, IsNil) + decFunc := newBaseBuiltinCastFunc(b, false) decFunc.tp = types.NewFieldType(mysql.TypeNewDecimal) decFunc.tp.Flen = 60 decFunc.tp.Decimal = 2 @@ -1187,7 +1206,7 @@ func (s *testEvaluatorSuite) TestWrapWithCastAsTypesClasses(c *C) { { &Column{RetType: types.NewFieldType(mysql.TypeDatetime), Index: 0}, chunk.MutRowFromDatums([]types.Datum{timeWithFspDatum}), - int64(curDateInt*1000000 + 130000), curTimeWithFspReal, types.NewDecFromFloatForTest(curTimeWithFspReal), curTimeWithFspString, + curDateInt*1000000 + 130000, curTimeWithFspReal, types.NewDecFromFloatForTest(curTimeWithFspReal), curTimeWithFspString, }, { durationColumn0, diff --git a/expression/builtin_cast_vec_test.go b/expression/builtin_cast_vec_test.go index 149ac7e19a65c..1fbf9380d20a0 100644 --- a/expression/builtin_cast_vec_test.go +++ b/expression/builtin_cast_vec_test.go @@ -121,7 +121,7 @@ type randJSONDuration struct{} func (g *randJSONDuration) gen() interface{} { d := types.Duration{ - Duration: time.Duration(time.Duration(rand.Intn(12))*time.Hour + time.Duration(rand.Intn(60))*time.Minute + time.Duration(rand.Intn(60))*time.Second + time.Duration(rand.Intn(1000))*time.Millisecond), + Duration: time.Duration(rand.Intn(12))*time.Hour + time.Duration(rand.Intn(60))*time.Minute + time.Duration(rand.Intn(60))*time.Second + time.Duration(rand.Intn(1000))*time.Millisecond, Fsp: 3} return json.CreateBinary(d.String()) } diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index f227cd572779a..9f9dbac00e425 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -125,7 +125,10 @@ func (c *coalesceFunctionClass) getFunction(ctx sessionctx.Context, args []Expre fieldEvalTps = append(fieldEvalTps, retEvalTp) } - bf := newBaseBuiltinFuncWithTp(ctx, args, retEvalTp, fieldEvalTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, retEvalTp, fieldEvalTps...) + if err != nil { + return nil, err + } bf.tp.Flag |= resultFieldType.Flag resultFieldType.Flen, resultFieldType.Decimal = 0, types.UnspecifiedLength @@ -430,7 +433,10 @@ func (c *greatestFunctionClass) getFunction(ctx sessionctx.Context, args []Expre for i := range args { argTps[i] = tp } - bf := newBaseBuiltinFuncWithTp(ctx, args, tp, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, tp, argTps...) + if err != nil { + return nil, err + } if cmpAsDatetime { tp = types.ETDatetime } @@ -629,7 +635,10 @@ func (c *leastFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi for i := range args { argTps[i] = tp } - bf := newBaseBuiltinFuncWithTp(ctx, args, tp, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, tp, argTps...) + if err != nil { + return nil, err + } if cmpAsDatetime { tp = types.ETDatetime } @@ -840,7 +849,10 @@ func (c *intervalFunctionClass) getFunction(ctx sessionctx.Context, args []Expre for range args { argTps = append(argTps, argTp) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } var sig builtinFunc if allInt { sig = &builtinIntervalIntSig{bf} @@ -1249,21 +1261,21 @@ func (c *compareFunctionClass) refineArgs(ctx sessionctx.Context, args []Express } if isExceptional && (c.op == opcode.EQ || c.op == opcode.NullEQ) { // This will always be false. - return []Expression{Zero, One} + return []Expression{NewZero(), NewOne()} } if isPositiveInfinite { // If the op is opcode.LT, opcode.LE // This will always be true. // If the op is opcode.GT, opcode.GE // This will always be false. - return []Expression{Zero, One} + return []Expression{NewZero(), NewOne()} } if isNegativeInfinite { // If the op is opcode.GT, opcode.GE // This will always be true. // If the op is opcode.LT, opcode.LE // This will always be false. - return []Expression{One, Zero} + return []Expression{NewOne(), NewZero()} } return []Expression{finalArg0, finalArg1} @@ -1282,7 +1294,10 @@ func (c *compareFunctionClass) getFunction(ctx sessionctx.Context, rawArgs []Exp // generateCmpSigs generates compare function signatures. func (c *compareFunctionClass) generateCmpSigs(ctx sessionctx.Context, args []Expression, tp types.EvalType) (sig builtinFunc, err error) { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, tp, tp) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, tp, tp) + if err != nil { + return nil, err + } if tp == types.ETJson { // In compare, if we cast string to JSON, we shouldn't parse it. for i := range args { diff --git a/expression/builtin_compare_test.go b/expression/builtin_compare_test.go index efa4edab8d059..8f3516c655f1e 100644 --- a/expression/builtin_compare_test.go +++ b/expression/builtin_compare_test.go @@ -82,7 +82,7 @@ func (s *testEvaluatorSuite) TestCompareFunctionWithRefine(c *C) { func (s *testEvaluatorSuite) TestCompare(c *C) { intVal, uintVal, realVal, stringVal, decimalVal := 1, uint64(1), 1.1, "123", types.NewDecFromFloatForTest(123.123) timeVal := types.NewTime(types.FromGoTime(time.Now()), mysql.TypeDatetime, 6) - durationVal := types.Duration{Duration: time.Duration(12*time.Hour + 1*time.Minute + 1*time.Second)} + durationVal := types.Duration{Duration: 12*time.Hour + 1*time.Minute + 1*time.Second} jsonVal := json.CreateBinary("123") // test cases for generating function signatures. tests := []struct { @@ -201,7 +201,7 @@ func (s *testEvaluatorSuite) TestCoalesce(c *C) { } } - _, err := funcs[ast.Length].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Length].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -331,8 +331,8 @@ func (s *testEvaluatorSuite) TestGreatestLeastFuncs(c *C) { } } } - _, err := funcs[ast.Greatest].getFunction(s.ctx, []Expression{Zero, One}) + _, err := funcs[ast.Greatest].getFunction(s.ctx, []Expression{NewZero(), NewOne()}) c.Assert(err, IsNil) - _, err = funcs[ast.Least].getFunction(s.ctx, []Expression{Zero, One}) + _, err = funcs[ast.Least].getFunction(s.ctx, []Expression{NewZero(), NewOne()}) c.Assert(err, IsNil) } diff --git a/expression/builtin_compare_vec.go b/expression/builtin_compare_vec.go index 214fe4b2838b3..5e08511a09fad 100644 --- a/expression/builtin_compare_vec.go +++ b/expression/builtin_compare_vec.go @@ -725,8 +725,8 @@ func (b *builtinLeastTimeSig) vecEvalString(input *chunk.Chunk, result *chunk.Co } var argTime types.Time - var findInvalidTime []bool = make([]bool, n) - var invalidValue []string = make([]string, n) + findInvalidTime := make([]bool, n) + invalidValue := make([]string, n) for j := 0; j < len(b.args); j++ { if err := b.args[j].VecEvalString(b.ctx, input, result); err != nil { diff --git a/expression/builtin_control.go b/expression/builtin_control.go index 929236eec4ff3..c869c284aaf44 100644 --- a/expression/builtin_control.go +++ b/expression/builtin_control.go @@ -143,14 +143,22 @@ func (c *caseWhenFunctionClass) getFunction(ctx sessionctx.Context, args []Expre for i := 1; i < l; i += 2 { fieldTps = append(fieldTps, args[i].GetType()) decimal = mathutil.Max(decimal, args[i].GetType().Decimal) - flen = mathutil.Max(flen, args[i].GetType().Flen) + if args[i].GetType().Flen == -1 { + flen = -1 + } else if flen != -1 { + flen = mathutil.Max(flen, args[i].GetType().Flen) + } isBinaryStr = isBinaryStr || types.IsBinaryStr(args[i].GetType()) isBinaryFlag = isBinaryFlag || !types.IsNonBinaryStr(args[i].GetType()) } if l%2 == 1 { fieldTps = append(fieldTps, args[l-1].GetType()) decimal = mathutil.Max(decimal, args[l-1].GetType().Decimal) - flen = mathutil.Max(flen, args[l-1].GetType().Flen) + if args[l-1].GetType().Flen == -1 { + flen = -1 + } else if flen != -1 { + flen = mathutil.Max(flen, args[l-1].GetType().Flen) + } isBinaryStr = isBinaryStr || types.IsBinaryStr(args[l-1].GetType()) isBinaryFlag = isBinaryFlag || !types.IsNonBinaryStr(args[l-1].GetType()) } @@ -175,7 +183,7 @@ func (c *caseWhenFunctionClass) getFunction(ctx sessionctx.Context, args []Expre } argTps := make([]types.EvalType, 0, l) for i := 0; i < l-1; i += 2 { - if args[i], err = wrapWithIsTrue(ctx, true, args[i]); err != nil { + if args[i], err = wrapWithIsTrue(ctx, true, args[i], false); err != nil { return nil, err } argTps = append(argTps, types.ETInt, tp) @@ -183,7 +191,10 @@ func (c *caseWhenFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if l%2 == 1 { argTps = append(argTps, tp) } - bf := newBaseBuiltinFuncWithTp(ctx, args, tp, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, tp, argTps...) + if err != nil { + return nil, err + } bf.tp = fieldTp switch tp { @@ -475,11 +486,14 @@ func (c *ifFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) } retTp := InferType4ControlFuncs(args[1].GetType(), args[2].GetType()) evalTps := retTp.EvalType() - args[0], err = wrapWithIsTrue(ctx, true, args[0]) + args[0], err = wrapWithIsTrue(ctx, true, args[0], false) + if err != nil { + return nil, err + } + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, evalTps, types.ETInt, evalTps, evalTps) if err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, evalTps, types.ETInt, evalTps, evalTps) retTp.Flag |= bf.tp.Flag bf.tp = retTp switch evalTps { @@ -695,7 +709,10 @@ func (c *ifNullFunctionClass) getFunction(ctx sessionctx.Context, args []Express types.SetBinChsClnFlag(retTp) } evalTps := retTp.EvalType() - bf := newBaseBuiltinFuncWithTp(ctx, args, evalTps, evalTps, evalTps) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, evalTps, evalTps, evalTps) + if err != nil { + return nil, err + } bf.tp = retTp switch evalTps { case types.ETInt: diff --git a/expression/builtin_control_test.go b/expression/builtin_control_test.go index 5c4ee439e3177..6ea1655e1c874 100644 --- a/expression/builtin_control_test.go +++ b/expression/builtin_control_test.go @@ -139,9 +139,9 @@ func (s *testEvaluatorSuite) TestIfNull(c *C) { } } - _, err := funcs[ast.Ifnull].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err := funcs[ast.Ifnull].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) - _, err = funcs[ast.Ifnull].getFunction(s.ctx, []Expression{Zero}) + _, err = funcs[ast.Ifnull].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, NotNil) } diff --git a/expression/builtin_encryption.go b/expression/builtin_encryption.go index 4d4574dac2225..9f423814d232d 100644 --- a/expression/builtin_encryption.go +++ b/expression/builtin_encryption.go @@ -111,7 +111,10 @@ func (c *aesDecryptFunctionClass) getFunction(ctx sessionctx.Context, args []Exp for range args { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen = args[0].GetType().Flen // At most. types.SetBinChsClnFlag(bf.tp) @@ -242,7 +245,10 @@ func (c *aesEncryptFunctionClass) getFunction(ctx sessionctx.Context, args []Exp for range args { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen = aes.BlockSize * (args[0].GetType().Flen/aes.BlockSize + 1) // At most. types.SetBinChsClnFlag(bf.tp) @@ -370,7 +376,10 @@ func (c *decodeFunctionClass) getFunction(ctx sessionctx.Context, args []Express return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = args[0].GetType().Flen sig := &builtinDecodeSig{bf} @@ -430,7 +439,10 @@ func (c *encodeFunctionClass) getFunction(ctx sessionctx.Context, args []Express return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = args[0].GetType().Flen sig := &builtinEncodeSig{bf} @@ -489,7 +501,10 @@ func (c *passwordFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = mysql.PWDHashLen + 1 sig := &builtinPasswordSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Password) @@ -533,7 +548,10 @@ func (c *randomBytesFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen = 1024 // Max allowed random bytes types.SetBinChsClnFlag(bf.tp) sig := &builtinRandomBytesSig{bf} @@ -577,7 +595,10 @@ func (c *md5FunctionClass) getFunction(ctx sessionctx.Context, args []Expression if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 32 sig := &builtinMD5Sig{bf} @@ -615,7 +636,10 @@ func (c *sha1FunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 40 sig := &builtinSHA1Sig{bf} @@ -657,7 +681,10 @@ func (c *sha2FunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 128 // sha512 sig := &builtinSHA2Sig{bf} @@ -753,7 +780,10 @@ func (c *compressFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } srcLen := args[0].GetType().Flen compressBound := srcLen + (srcLen >> 12) + (srcLen >> 14) + (srcLen >> 25) + 13 if compressBound > mysql.MaxBlobWidth { @@ -821,7 +851,10 @@ func (c *uncompressFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = mysql.MaxBlobWidth types.SetBinChsClnFlag(bf.tp) sig := &builtinUncompressSig{bf} @@ -876,7 +909,10 @@ func (c *uncompressedLengthFunctionClass) getFunction(ctx sessionctx.Context, ar if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 10 sig := &builtinUncompressedLengthSig{bf} sig.setPbCode(tipb.ScalarFuncSig_UncompressedLength) diff --git a/expression/builtin_encryption_test.go b/expression/builtin_encryption_test.go index c9ad35a353ece..c3b7f34878ab4 100644 --- a/expression/builtin_encryption_test.go +++ b/expression/builtin_encryption_test.go @@ -321,7 +321,7 @@ func (s *testEvaluatorSuite) TestMD5Hash(c *C) { } } } - _, err := funcs[ast.MD5].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.MD5].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -491,6 +491,6 @@ func (s *testEvaluatorSuite) TestPassword(c *C) { } } - _, err := funcs[ast.PasswordFunc].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.PasswordFunc].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } diff --git a/expression/builtin_info.go b/expression/builtin_info.go index 0f5c2d277a2d1..252d043f19bc2 100644 --- a/expression/builtin_info.go +++ b/expression/builtin_info.go @@ -91,7 +91,10 @@ func (c *databaseFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 64 sig := &builtinDatabaseSig{bf} @@ -123,7 +126,10 @@ func (c *foundRowsFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flag |= mysql.UnsignedFlag sig := &builtinFoundRowsSig{bf} return sig, nil @@ -158,7 +164,10 @@ func (c *currentUserFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 64 sig := &builtinCurrentUserSig{bf} @@ -193,7 +202,10 @@ func (c *currentRoleFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 64 sig := &builtinCurrentRoleSig{bf} @@ -243,7 +255,10 @@ func (c *userFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 64 sig := &builtinUserSig{bf} @@ -279,7 +294,10 @@ func (c *connectionIDFunctionClass) getFunction(ctx sessionctx.Context, args []E if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flag |= mysql.UnsignedFlag sig := &builtinConnectionIDSig{bf} return sig, nil @@ -316,7 +334,10 @@ func (c *lastInsertIDFunctionClass) getFunction(ctx sessionctx.Context, args []E if len(args) == 1 { argsTp = append(argsTp, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argsTp...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argsTp...) + if err != nil { + return nil, err + } bf.tp.Flag |= mysql.UnsignedFlag if len(args) == 1 { @@ -376,7 +397,10 @@ func (c *versionFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 64 sig := &builtinVersionSig{bf} @@ -407,7 +431,10 @@ func (c *tidbVersionFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = len(printer.GetTiDBInfo()) sig := &builtinTiDBVersionSig{bf} @@ -438,7 +465,10 @@ func (c *tidbIsDDLOwnerFunctionClass) getFunction(ctx sessionctx.Context, args [ if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt) + if err != nil { + return nil, err + } sig := &builtinTiDBIsDDLOwnerSig{bf} return sig, nil } @@ -484,7 +514,10 @@ func (c *benchmarkFunctionClass) getFunction(ctx sessionctx.Context, args []Expr constLoopCount = lc } } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, sameEvalType) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, sameEvalType) + if err != nil { + return nil, err + } sig := &builtinBenchmarkSig{bf, constLoopCount} return sig, nil } @@ -601,7 +634,10 @@ func (c *coercibilityFunctionClass) getFunction(ctx sessionctx.Context, args []E if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, args[0].GetType().EvalType()) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, args[0].GetType().EvalType()) + if err != nil { + return nil, err + } sig := &builtinCoercibilitySig{bf} sig.setPbCode(tipb.ScalarFuncSig_Unspecified) return sig, nil @@ -633,7 +669,10 @@ func (c *collationFunctionClass) getFunction(ctx sessionctx.Context, args []Expr for _, arg := range args { argsTps = append(argsTps, arg.GetType().EvalType()) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argsTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argsTps...) + if err != nil { + return nil, err + } sig := &builtinCollationSig{bf} return sig, nil } @@ -660,7 +699,10 @@ func (c *rowCountFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err = c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt) + if err != nil { + return nil, err + } sig = &builtinRowCountSig{bf} sig.setPbCode(tipb.ScalarFuncSig_RowCount) return sig, nil @@ -679,7 +721,7 @@ func (b *builtinRowCountSig) Clone() builtinFunc { // evalInt evals ROW_COUNT(). // See https://dev.mysql.com/doc/refman/5.7/en/information-functions.html#function_row-count. func (b *builtinRowCountSig) evalInt(_ chunk.Row) (res int64, isNull bool, err error) { - res = int64(b.ctx.GetSessionVars().StmtCtx.PrevAffectedRows) + res = b.ctx.GetSessionVars().StmtCtx.PrevAffectedRows return res, false, nil } @@ -691,7 +733,10 @@ func (c *tidbDecodeKeyFunctionClass) getFunction(ctx sessionctx.Context, args [] if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } sig := &builtinTiDBDecodeKeySig{bf} return sig, nil } @@ -722,14 +767,14 @@ func decodeKey(ctx sessionctx.Context, s string) string { return s } // Auto decode byte if needed. - _, bs, err := codec.DecodeBytes([]byte(key), nil) + _, bs, err := codec.DecodeBytes(key, nil) if err == nil { key = bs } // Try to decode it as a record key. tableID, handle, err := tablecodec.DecodeRecordKey(key) if err == nil { - return "tableID=" + strconv.FormatInt(tableID, 10) + ", _tidb_rowid=" + strconv.FormatInt(handle, 10) + return "tableID=" + strconv.FormatInt(tableID, 10) + ", _tidb_rowid=" + strconv.FormatInt(handle.IntValue(), 10) } // Try decode as table index key. tableID, indexID, indexValues, err := tablecodec.DecodeIndexKeyPrefix(key) @@ -751,7 +796,10 @@ func (c *tidbDecodePlanFunctionClass) getFunction(ctx sessionctx.Context, args [ if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } sig := &builtinTiDBDecodePlanSig{bf} return sig, nil } @@ -783,7 +831,10 @@ func (c *nextValFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } sig := &builtinNextValSig{bf} bf.tp.Flen = 10 return sig, nil @@ -836,7 +887,10 @@ func (c *lastValFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } sig := &builtinLastValSig{bf} bf.tp.Flen = 10 return sig, nil @@ -883,7 +937,10 @@ func (c *setValFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString, types.ETInt) + if err != nil { + return nil, err + } sig := &builtinSetValSig{bf} bf.tp.Flen = args[1].GetType().Flen return sig, nil @@ -942,7 +999,10 @@ func (c *formatBytesFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETReal) + if err != nil { + return nil, err + } bf.tp.Flag |= mysql.UnsignedFlag sig := &builtinFormatBytesSig{bf} return sig, nil @@ -976,7 +1036,10 @@ func (c *formatNanoTimeFunctionClass) getFunction(ctx sessionctx.Context, args [ if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETReal) + if err != nil { + return nil, err + } bf.tp.Flag |= mysql.UnsignedFlag sig := &builtinFormatNanoTimeSig{bf} return sig, nil diff --git a/expression/builtin_info_test.go b/expression/builtin_info_test.go index 45ad65ac6337d..54f5305a8b442 100644 --- a/expression/builtin_info_test.go +++ b/expression/builtin_info_test.go @@ -259,7 +259,7 @@ func (s *testEvaluatorSuite) TestLastInsertID(c *C) { c.Assert(tp.Tp, Equals, mysql.TypeLonglong) c.Assert(tp.Charset, Equals, charset.CharsetBin) c.Assert(tp.Collate, Equals, charset.CollationBin) - c.Assert(tp.Flag&mysql.BinaryFlag, Equals, uint(mysql.BinaryFlag)) + c.Assert(tp.Flag&mysql.BinaryFlag, Equals, mysql.BinaryFlag) c.Assert(tp.Flen, Equals, mysql.MaxIntWidth) d, err := f.Eval(chunk.Row{}) if t.getErr { @@ -274,7 +274,7 @@ func (s *testEvaluatorSuite) TestLastInsertID(c *C) { } } - _, err := funcs[ast.LastInsertId].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.LastInsertId].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } diff --git a/expression/builtin_info_vec.go b/expression/builtin_info_vec.go index 690f2f47ca6ab..272dc1833a167 100644 --- a/expression/builtin_info_vec.go +++ b/expression/builtin_info_vec.go @@ -90,7 +90,7 @@ func (b *builtinRowCountSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column n := input.NumRows() result.ResizeInt64(n, false) i64s := result.Int64s() - res := int64(b.ctx.GetSessionVars().StmtCtx.PrevAffectedRows) + res := b.ctx.GetSessionVars().StmtCtx.PrevAffectedRows for i := 0; i < n; i++ { i64s[i] = res } diff --git a/expression/builtin_info_vec_test.go b/expression/builtin_info_vec_test.go index ba5efae457100..474d1c80d9320 100644 --- a/expression/builtin_info_vec_test.go +++ b/expression/builtin_info_vec_test.go @@ -21,6 +21,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" ) @@ -35,7 +36,7 @@ func (g *tidbKeyGener) gen() interface{} { if rand.Intn(2) == 1 { // Generate a record key handle := g.inner.gen().(int64) - result = tablecodec.EncodeRowKeyWithHandle(tableID, handle) + result = tablecodec.EncodeRowKeyWithHandle(tableID, kv.IntHandle(handle)) } else { // Generate an index key idx := g.inner.gen().(int64) diff --git a/expression/builtin_json.go b/expression/builtin_json.go index f2f2bb0198d4c..bb2eee747606b 100644 --- a/expression/builtin_json.go +++ b/expression/builtin_json.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hack" - "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" ) @@ -100,7 +99,10 @@ func (c *jsonTypeFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETJson) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETJson) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 51 // Flen of JSON_TYPE is length of UNSIGNED INTEGER. sig := &builtinJSONTypeSig{bf} @@ -140,7 +142,10 @@ func (c *jsonExtractFunctionClass) getFunction(ctx sessionctx.Context, args []Ex for range args[1:] { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } sig := &builtinJSONExtractSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonExtractSig) return sig, nil @@ -199,7 +204,10 @@ func (c *jsonUnquoteFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = mysql.MaxFieldVarCharLength DisableParseJSONFlag4Expr(args[0]) sig := &builtinJSONUnquoteSig{bf} @@ -245,7 +253,10 @@ func (c *jsonSetFunctionClass) getFunction(ctx sessionctx.Context, args []Expres for i := 1; i < len(args)-1; i += 2 { argTps = append(argTps, types.ETString, types.ETJson) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } for i := 2; i < len(args); i += 2 { DisableParseJSONFlag4Expr(args[i]) } @@ -285,7 +296,10 @@ func (c *jsonInsertFunctionClass) getFunction(ctx sessionctx.Context, args []Exp for i := 1; i < len(args)-1; i += 2 { argTps = append(argTps, types.ETString, types.ETJson) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } for i := 2; i < len(args); i += 2 { DisableParseJSONFlag4Expr(args[i]) } @@ -325,7 +339,10 @@ func (c *jsonReplaceFunctionClass) getFunction(ctx sessionctx.Context, args []Ex for i := 1; i < len(args)-1; i += 2 { argTps = append(argTps, types.ETString, types.ETJson) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } for i := 2; i < len(args); i += 2 { DisableParseJSONFlag4Expr(args[i]) } @@ -362,7 +379,10 @@ func (c *jsonRemoveFunctionClass) getFunction(ctx sessionctx.Context, args []Exp for range args[1:] { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } sig := &builtinJSONRemoveSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonRemoveSig) return sig, nil @@ -416,7 +436,10 @@ func (c *jsonMergeFunctionClass) getFunction(ctx sessionctx.Context, args []Expr for range args { argTps = append(argTps, types.ETJson) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } sig := &builtinJSONMergeSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonMergeSig) return sig, nil @@ -466,7 +489,10 @@ func (c *jsonObjectFunctionClass) getFunction(ctx sessionctx.Context, args []Exp for i := 0; i < len(args)-1; i += 2 { argTps = append(argTps, types.ETString, types.ETJson) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } for i := 1; i < len(args); i += 2 { DisableParseJSONFlag4Expr(args[i]) } @@ -529,7 +555,10 @@ func (c *jsonArrayFunctionClass) getFunction(ctx sessionctx.Context, args []Expr for range args { argTps = append(argTps, types.ETJson) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } for i := range args { DisableParseJSONFlag4Expr(args[i]) } @@ -575,7 +604,10 @@ func (c *jsonContainsPathFunctionClass) getFunction(ctx sessionctx.Context, args for i := 3; i <= len(args); i++ { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } sig := &builtinJSONContainsPathSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonContainsPathSig) return sig, nil @@ -692,7 +724,10 @@ func (c *jsonContainsFunctionClass) getFunction(ctx sessionctx.Context, args []E if len(args) == 3 { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } sig := &builtinJSONContainsSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonContainsSig) return sig, nil @@ -746,15 +781,24 @@ func (c *jsonValidFunctionClass) getFunction(ctx sessionctx.Context, args []Expr argType := args[0].GetType().EvalType() switch argType { case types.ETJson: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETJson) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETJson) + if err != nil { + return nil, err + } sig = &builtinJSONValidJSONSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonValidJsonSig) case types.ETString: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } sig = &builtinJSONValidStringSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonValidStringSig) default: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argType) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argType) + if err != nil { + return nil, err + } sig = &builtinJSONValidOthersSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonValidOthersSig) } @@ -845,7 +889,10 @@ func (c *jsonArrayAppendFunctionClass) getFunction(ctx sessionctx.Context, args for i := 1; i < len(args)-1; i += 2 { argTps = append(argTps, types.ETString, types.ETJson) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } for i := 2; i < len(args); i += 2 { DisableParseJSONFlag4Expr(args[i]) } @@ -868,53 +915,53 @@ func (b *builtinJSONArrayAppendSig) evalJSON(row chunk.Row) (res json.BinaryJSON for i := 1; i < len(b.args)-1; i += 2 { // If JSON path is NULL, MySQL breaks and returns NULL. - s, isNull, err := b.args[i].EvalString(b.ctx, row) - if isNull || err != nil { + s, sNull, err := b.args[i].EvalString(b.ctx, row) + if sNull || err != nil { return res, true, err } - - // We should do the following checks to get correct values in res.Extract - pathExpr, err := json.ParseJSONPathExpr(s) - if err != nil { - return res, true, json.ErrInvalidJSONPath.GenWithStackByArgs(s) - } - if pathExpr.ContainsAnyAsterisk() { - return res, true, json.ErrInvalidJSONPathWildcard.GenWithStackByArgs(s) - } - - obj, exists := res.Extract([]json.PathExpression{pathExpr}) - if !exists { - // If path not exists, just do nothing and no errors. - continue - } - - if obj.TypeCode != json.TypeCodeArray { - // res.Extract will return a json object instead of an array if there is an object at path pathExpr. - // JSON_ARRAY_APPEND({"a": "b"}, "$", {"b": "c"}) => [{"a": "b"}, {"b", "c"}] - // We should wrap them to a single array first. - obj = json.CreateBinary([]interface{}{obj}) - } - - value, isnull, err := b.args[i+1].EvalJSON(b.ctx, row) + value, vNull, err := b.args[i+1].EvalJSON(b.ctx, row) if err != nil { return res, true, err } - - if isnull { + if vNull { value = json.CreateBinary(nil) } - - obj = json.MergeBinary([]json.BinaryJSON{obj, value}) - res, err = res.Modify([]json.PathExpression{pathExpr}, []json.BinaryJSON{obj}, json.ModifySet) - if err != nil { - // We checked pathExpr in the same way as res.Modify do. - // So err should always be nil, the function should never return here. - return res, true, err + res, isNull, err = b.appendJSONArray(res, s, value) + if isNull || err != nil { + return res, isNull, err } } return res, false, nil } +func (b *builtinJSONArrayAppendSig) appendJSONArray(res json.BinaryJSON, p string, v json.BinaryJSON) (json.BinaryJSON, bool, error) { + // We should do the following checks to get correct values in res.Extract + pathExpr, err := json.ParseJSONPathExpr(p) + if err != nil { + return res, true, json.ErrInvalidJSONPath.GenWithStackByArgs(p) + } + if pathExpr.ContainsAnyAsterisk() { + return res, true, json.ErrInvalidJSONPathWildcard.GenWithStackByArgs(p) + } + + obj, exists := res.Extract([]json.PathExpression{pathExpr}) + if !exists { + // If path not exists, just do nothing and no errors. + return res, false, nil + } + + if obj.TypeCode != json.TypeCodeArray { + // res.Extract will return a json object instead of an array if there is an object at path pathExpr. + // JSON_ARRAY_APPEND({"a": "b"}, "$", {"b": "c"}) => [{"a": "b"}, {"b", "c"}] + // We should wrap them to a single array first. + obj = json.CreateBinary([]interface{}{obj}) + } + + obj = json.MergeBinary([]json.BinaryJSON{obj, v}) + res, err = res.Modify([]json.PathExpression{pathExpr}, []json.BinaryJSON{obj}, json.ModifySet) + return res, false, err +} + type jsonArrayInsertFunctionClass struct { baseFunctionClass } @@ -936,7 +983,10 @@ func (c *jsonArrayInsertFunctionClass) getFunction(ctx sessionctx.Context, args for i := 1; i < len(args)-1; i += 2 { argTps = append(argTps, types.ETString, types.ETJson) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } for i := 2; i < len(args); i += 2 { DisableParseJSONFlag4Expr(args[i]) } @@ -1009,7 +1059,10 @@ func (c *jsonMergePreserveFunctionClass) getFunction(ctx sessionctx.Context, arg for range args { argTps = append(argTps, types.ETJson) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } sig := &builtinJSONMergeSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonMergePreserveSig) return sig, nil @@ -1051,7 +1104,10 @@ func (c *jsonQuoteFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } DisableParseJSONFlag4Expr(args[0]) sig := &builtinJSONQuoteSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonQuoteSig) @@ -1090,7 +1146,10 @@ func (c *jsonSearchFunctionClass) getFunction(ctx sessionctx.Context, args []Exp for range args[1:] { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } sig := &builtinJSONSearchSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonSearchSig) return sig, nil @@ -1110,6 +1169,7 @@ func (b *builtinJSONSearchSig) evalJSON(row chunk.Row) (res json.BinaryJSON, isN if isNull || err != nil { return res, isNull, err } + containType = strings.ToLower(containType) if containType != json.ContainsPathAll && containType != json.ContainsPathOne { return res, true, errors.AddStack(json.ErrInvalidJSONContainsPathType) } @@ -1130,26 +1190,11 @@ func (b *builtinJSONSearchSig) evalJSON(row chunk.Row) (res json.BinaryJSON, isN if isNull || len(escapeStr) == 0 { escape = byte('\\') } else if len(escapeStr) == 1 { - escape = byte(escapeStr[0]) + escape = escapeStr[0] } else { return res, true, errIncorrectArgs.GenWithStackByArgs("ESCAPE") } } - patChars, patTypes := stringutil.CompilePattern(searchStr, escape) - - // result - result := make([]interface{}, 0) - - // walk json_doc - walkFn := func(fullpath json.PathExpression, bj json.BinaryJSON) (stop bool, err error) { - if bj.TypeCode == json.TypeCodeString && stringutil.DoMatch(string(bj.GetString()), patChars, patTypes) { - result = append(result, fullpath.String()) - if containType == json.ContainsPathOne { - return true, nil - } - } - return false, nil - } if len(b.args) >= 5 { // path... pathExprs := make([]json.PathExpression, 0, len(b.args)-4) for i := 4; i < len(b.args); i++ { @@ -1165,26 +1210,9 @@ func (b *builtinJSONSearchSig) evalJSON(row chunk.Row) (res json.BinaryJSON, isN } pathExprs = append(pathExprs, pathExpr) } - err = obj.Walk(walkFn, pathExprs...) - if err != nil { - return res, true, err - } - } else { - err = obj.Walk(walkFn) - if err != nil { - return res, true, err - } - } - - // return - switch len(result) { - case 0: - return res, true, nil - case 1: - return json.CreateBinary(result[0]), false, nil - default: - return json.CreateBinary(result), false, nil + return obj.Search(containType, searchStr, escape, pathExprs) } + return obj.Search(containType, searchStr, escape, nil) } type jsonStorageSizeFunctionClass struct { @@ -1206,7 +1234,10 @@ func (c *jsonStorageSizeFunctionClass) getFunction(ctx sessionctx.Context, args return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETJson) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETJson) + if err != nil { + return nil, err + } sig := &builtinJSONStorageSizeSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonStorageSizeSig) return sig, nil @@ -1245,7 +1276,10 @@ func (c *jsonDepthFunctionClass) getFunction(ctx sessionctx.Context, args []Expr return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETJson) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETJson) + if err != nil { + return nil, err + } sig := &builtinJSONDepthSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonDepthSig) return sig, nil @@ -1272,7 +1306,10 @@ func (c *jsonKeysFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if len(args) == 2 { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETJson, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETJson, argTps...) + if err != nil { + return nil, err + } var sig builtinFunc switch len(args) { case 1: @@ -1371,7 +1408,10 @@ func (c *jsonLengthFunctionClass) getFunction(ctx sessionctx.Context, args []Exp argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } sig := &builtinJSONLengthSig{bf} sig.setPbCode(tipb.ScalarFuncSig_JsonLengthSig) return sig, nil diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index 3a6481f8bf93e..0302e4f3c2d3d 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -791,6 +791,7 @@ func (s *testEvaluatorSuite) TestJSONSearch(c *C) { {[]interface{}{jsonString, `one`, `abc`}, `"$[0]"`, true}, {[]interface{}{jsonString, `all`, `abc`}, `["$[0]", "$[2].x"]`, true}, {[]interface{}{jsonString, `all`, `ghi`}, nil, true}, + {[]interface{}{jsonString, `ALL`, `ghi`}, nil, true}, {[]interface{}{jsonString, `all`, `10`}, `"$[1][0].k"`, true}, {[]interface{}{jsonString, `all`, `10`, nil, `$`}, `"$[1][0].k"`, true}, {[]interface{}{jsonString, `all`, `10`, nil, `$[*]`}, `"$[1][0].k"`, true}, @@ -825,6 +826,7 @@ func (s *testEvaluatorSuite) TestJSONSearch(c *C) { {[]interface{}{jsonString, `all`, `abc`, `??`}, nil, false}, // wrong escape_char {[]interface{}{jsonString, `all`, `abc`, nil, nil}, nil, true}, // NULL path {[]interface{}{jsonString, `all`, `abc`, nil, `$xx`}, nil, false}, // wrong path + {[]interface{}{jsonString, nil, `abc`}, nil, true}, } for _, t := range tbl { args := types.MakeDatums(t.input...) diff --git a/expression/builtin_json_vec.go b/expression/builtin_json_vec.go index 0a6213788f56c..db512fc81359f 100644 --- a/expression/builtin_json_vec.go +++ b/expression/builtin_json_vec.go @@ -357,11 +357,104 @@ func (b *builtinJSONQuoteSig) vecEvalString(input *chunk.Chunk, result *chunk.Co } func (b *builtinJSONSearchSig) vectorized() bool { - return false + return true } func (b *builtinJSONSearchSig) vecEvalJSON(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + nr := input.NumRows() + jsonBuf, err := b.bufAllocator.get(types.ETJson, nr) + if err != nil { + return err + } + defer b.bufAllocator.put(jsonBuf) + if err := b.args[0].VecEvalJSON(b.ctx, input, jsonBuf); err != nil { + return err + } + typeBuf, err := b.bufAllocator.get(types.ETString, nr) + if err != nil { + return err + } + defer b.bufAllocator.put(typeBuf) + if err := b.args[1].VecEvalString(b.ctx, input, typeBuf); err != nil { + return err + } + searchBuf, err := b.bufAllocator.get(types.ETString, nr) + if err != nil { + return err + } + defer b.bufAllocator.put(searchBuf) + if err := b.args[2].VecEvalString(b.ctx, input, searchBuf); err != nil { + return err + } + + var escapeBuf *chunk.Column + if len(b.args) >= 4 { + escapeBuf, err = b.bufAllocator.get(types.ETString, nr) + if err != nil { + return err + } + defer b.bufAllocator.put(escapeBuf) + if err := b.args[3].VecEvalString(b.ctx, input, escapeBuf); err != nil { + return err + } + } + + var pathBufs []*chunk.Column + if len(b.args) >= 5 { + pathBufs = make([]*chunk.Column, (len(b.args) - 4)) + for i := 4; i < len(b.args); i++ { + index := i - 4 + pathBufs[index], err = b.bufAllocator.get(types.ETString, nr) + if err != nil { + return err + } + defer b.bufAllocator.put(pathBufs[index]) + if err := b.args[i].VecEvalString(b.ctx, input, pathBufs[index]); err != nil { + return err + } + } + } + + result.ReserveJSON(nr) + + for i := 0; i < nr; i++ { + if jsonBuf.IsNull(i) || searchBuf.IsNull(i) || typeBuf.IsNull(i) { + result.AppendNull() + continue + } + containType := strings.ToLower(typeBuf.GetString(i)) + escape := byte('\\') + if escapeBuf != nil && !escapeBuf.IsNull(i) { + escapeStr := escapeBuf.GetString(i) + if len(escapeStr) == 0 { + escape = byte('\\') + } else if len(escapeStr) == 1 { + escape = escapeStr[0] + } else { + return errIncorrectArgs.GenWithStackByArgs("ESCAPE") + } + } + var pathExprs []json.PathExpression + if pathBufs != nil { + pathExprs = make([]json.PathExpression, 0, len(b.args)-4) + for j := 0; j < len(b.args)-4; j++ { + if pathBufs[j].IsNull(i) { + break + } + pathExpr, err := json.ParseJSONPathExpr(pathBufs[j].GetString(i)) + if err != nil { + return json.ErrInvalidJSONPath.GenWithStackByArgs(pathBufs[j].GetString(i)) + } + pathExprs = append(pathExprs, pathExpr) + } + } + bj, _, err := jsonBuf.GetJSON(i).Search(containType, searchBuf.GetString(i), escape, pathExprs) + if err != nil { + return err + } + result.AppendJSON(bj) + } + return nil } func (b *builtinJSONSetSig) vectorized() bool { @@ -967,11 +1060,87 @@ func (b *builtinJSONContainsPathSig) vecEvalInt(input *chunk.Chunk, result *chun } func (b *builtinJSONArrayAppendSig) vectorized() bool { - return false + return true } func (b *builtinJSONArrayAppendSig) vecEvalJSON(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + n := input.NumRows() + m := (len(b.args) - 1) / 2 + + jsonBufs, err := b.bufAllocator.get(types.ETJson, n) + if err != nil { + return err + } + defer b.bufAllocator.put(jsonBufs) + if err := b.args[0].VecEvalJSON(b.ctx, input, jsonBufs); err != nil { + return err + } + + pathBufs := make([]*chunk.Column, 0, m) + valBufs := make([]*chunk.Column, 0, m) + defer func() { + for _, buf := range pathBufs { + b.bufAllocator.put(buf) + } + for _, buf := range valBufs { + b.bufAllocator.put(buf) + } + }() + for i := 1; i < len(b.args)-1; i += 2 { + pathBuf, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + pathBufs = append(pathBufs, pathBuf) + if err := b.args[i].VecEvalString(b.ctx, input, pathBuf); err != nil { + return err + } + valBuf, err := b.bufAllocator.get(types.ETJson, n) + if err != nil { + return err + } + if err := b.args[i+1].VecEvalJSON(b.ctx, input, valBuf); err != nil { + return err + } + valBufs = append(valBufs, valBuf) + } + + result.ReserveJSON(n) + for i := 0; i < n; i++ { + if jsonBufs.IsNull(i) { + result.AppendNull() + continue + } + res := jsonBufs.GetJSON(i) + isNull := false + for j := 0; j < m; j++ { + if pathBufs[j].IsNull(i) { + isNull = true + break + } + s := pathBufs[j].GetString(i) + v, vNull := json.BinaryJSON{}, valBufs[j].IsNull(i) + if !vNull { + v = valBufs[j].GetJSON(i) + } + if vNull { + v = json.CreateBinary(nil) + } + res, isNull, err = b.appendJSONArray(res, s, v) + if err != nil { + return err + } + if isNull { + break + } + } + if isNull { + result.AppendNull() + } else { + result.AppendJSON(res) + } + } + return nil } func (b *builtinJSONUnquoteSig) vectorized() bool { diff --git a/expression/builtin_json_vec_test.go b/expression/builtin_json_vec_test.go index 663659582c384..cfbb039b65cc2 100644 --- a/expression/builtin_json_vec_test.go +++ b/expression/builtin_json_vec_test.go @@ -26,7 +26,26 @@ var vecBuiltinJSONCases = map[string][]vecExprBenchCase{ {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson}}, {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString}, geners: []dataGenerator{&constJSONGener{"{\"a\": {\"c\": 3}, \"b\": 2}"}, &constStrGener{"$.a"}}}, }, - ast.JSONArrayAppend: {}, + ast.JSONArrayAppend: { + {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETJson}, + geners: []dataGenerator{newNullWrappedGener(0.1, &constJSONGener{"{\"a\": {\"c\": 3}, \"b\": 2}"}), + newNullWrappedGener(0.1, &constStrGener{"$.a"}), + newNullWrappedGener(0.1, &constJSONGener{"{\"b\": 2}"})}}, + {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETJson, types.ETString, types.ETJson}, + geners: []dataGenerator{newNullWrappedGener(0.1, &constJSONGener{"{\"a\": {\"c\": 3}, \"b\": 2}"}), + newNullWrappedGener(0.1, &constStrGener{"$.a"}), + newNullWrappedGener(0.1, &constJSONGener{"{\"b\": 2}"}), + newNullWrappedGener(0.1, &constStrGener{"$.b"}), + newNullWrappedGener(0.1, &constJSONGener{"{\"x\": 3}"}), + }}, + {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETJson, types.ETString, types.ETJson}, + geners: []dataGenerator{newNullWrappedGener(0.1, &constJSONGener{"{\"a\": {\"c\": 3}, \"b\": 2}"}), + newNullWrappedGener(0.1, &constStrGener{"$.a"}), + newNullWrappedGener(0.1, &constJSONGener{"{\"b\": 2}"}), + newNullWrappedGener(0.1, &constStrGener{"$.x"}), // not exists + newNullWrappedGener(0.1, &constJSONGener{"{\"x\": 3}"}), + }}, + }, ast.JSONContainsPath: { {retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETString}, geners: []dataGenerator{&constJSONGener{"{\"a\": {\"c\": {\"d\": 4}}, \"b\": 2}"}, &constStrGener{"one"}, &constStrGener{"$.c"}}}, {retEvalType: types.ETInt, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETString}, geners: []dataGenerator{&constJSONGener{"{\"a\": {\"c\": {\"d\": 4}}, \"b\": 2}"}, &constStrGener{"all"}, &constStrGener{"$.a"}, &constStrGener{"$.c"}}}, @@ -93,7 +112,14 @@ var vecBuiltinJSONCases = map[string][]vecExprBenchCase{ ast.JSONSet: { {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETJson, types.ETString, types.ETJson}, geners: []dataGenerator{nil, &constStrGener{"$.key"}, nil, &constStrGener{"$.aaa"}, nil}}, }, - ast.JSONSearch: {}, + ast.JSONSearch: { + {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETString}, geners: []dataGenerator{&constJSONGener{"[\"abc\", [{\"k\": \"10\"}, \"def\"], {\"x\":\"abc\"}, {\"y\":\"bcd\"}]"}, &constStrGener{"one"}, &constStrGener{"abc"}}}, + {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETString}, geners: []dataGenerator{&constJSONGener{"[\"abc\", [{\"k\": \"10\"}, \"def\"], {\"x\":\"abc\"}, {\"y\":\"bcd\"}]"}, &constStrGener{"all"}, &constStrGener{"abc"}}}, + {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETString}, geners: []dataGenerator{&constJSONGener{"[\"abc\", [{\"k\": \"10\"}, \"def\"], {\"x\":\"abc\"}, {\"y\":\"bcd\"}]"}, &constStrGener{"all"}, &constStrGener{"%a%"}}}, + {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETString, types.ETString, types.ETString}, geners: []dataGenerator{&constJSONGener{"[\"abc\", [{\"k\": \"10\"}, \"def\"], {\"x\":\"abc\"}, {\"y\":\"bcd\"}]"}, &constStrGener{"all"}, &constStrGener{"%a%"}, &constStrGener{}, &constStrGener{"$"}}}, + {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETString, types.ETString}, geners: []dataGenerator{&constJSONGener{"[\"abc\", [{\"k\": \"10\"}, \"def\"], {\"x\":\"abc\"}, {\"y\":\"bcd\"}]"}, &constStrGener{"all"}, &constStrGener{"%a%"}, &constStrGener{}}}, + {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETString, types.ETString, types.ETString}, geners: []dataGenerator{&constJSONGener{"[\"abc\", [{\"k\": \"10\"}, \"def\"], {\"x\":\"abc\"}, {\"y\":\"bcd\"}]"}, &constStrGener{"all"}, &constStrGener{"10"}, &constStrGener{""}, &constStrGener{"$[1][0]"}}}, + }, ast.JSONReplace: { {retEvalType: types.ETJson, childrenTypes: []types.EvalType{types.ETJson, types.ETString, types.ETJson}, geners: []dataGenerator{nil, &constStrGener{"$.key"}, nil}}, }, diff --git a/expression/builtin_like.go b/expression/builtin_like.go index 01bb996c602c3..8a65625bf07ac 100644 --- a/expression/builtin_like.go +++ b/expression/builtin_like.go @@ -43,7 +43,10 @@ func (c *likeFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio return nil, err } argTp := []types.EvalType{types.ETString, types.ETString, types.ETInt} - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTp...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTp...) + if err != nil { + return nil, err + } bf.tp.Flen = 1 sig := &builtinLikeSig{bf, nil, false} sig.setPbCode(tipb.ScalarFuncSig_LikeSig) @@ -103,7 +106,10 @@ func (c *regexpFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 1 var sig builtinFunc if types.IsBinaryStr(args[0].GetType()) || types.IsBinaryStr(args[1].GetType()) { diff --git a/expression/builtin_like_vec.go b/expression/builtin_like_vec.go index fd2da02200d3c..0a4eea71aaf4c 100644 --- a/expression/builtin_like_vec.go +++ b/expression/builtin_like_vec.go @@ -86,6 +86,10 @@ func (b *builtinRegexpSharedSig) isMemorizedRegexpInitialized() bool { func (b *builtinRegexpSharedSig) initMemoizedRegexp(patterns *chunk.Column, n int) { // Precondition: patterns is generated from a constant expression + if n == 0 { + // If the input rownum is zero, the Regexp error shouldn't be generated. + return + } for i := 0; i < n; i++ { if patterns.IsNull(i) { continue diff --git a/expression/builtin_math.go b/expression/builtin_math.go index 5df520e756ae7..2b84cdeb77c49 100644 --- a/expression/builtin_math.go +++ b/expression/builtin_math.go @@ -125,7 +125,10 @@ func (c *absFunctionClass) getFunction(ctx sessionctx.Context, args []Expression if argTp != types.ETInt && argTp != types.ETDecimal { argTp = types.ETReal } - bf := newBaseBuiltinFuncWithTp(ctx, args, argTp, argTp) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, argTp, argTp) + if err != nil { + return nil, err + } if mysql.HasUnsignedFlag(argFieldTp.Flag) { bf.tp.Flag |= mysql.UnsignedFlag } @@ -259,7 +262,10 @@ func (c *roundFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if len(args) > 1 { argTps = append(argTps, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, argTp, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, argTp, argTps...) + if err != nil { + return nil, err + } argFieldTp := args[0].GetType() if mysql.HasUnsignedFlag(argFieldTp.Flag) { bf.tp.Flag |= mysql.UnsignedFlag @@ -466,7 +472,10 @@ func (c *ceilFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio } retTp, argTp := getEvalTp4FloorAndCeil(args[0]) - bf := newBaseBuiltinFuncWithTp(ctx, args, retTp, argTp) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, retTp, argTp) + if err != nil { + return nil, err + } setFlag4FloorAndCeil(bf.tp, args[0]) argFieldTp := args[0].GetType() bf.tp.Flen, bf.tp.Decimal = argFieldTp.Flen, 0 @@ -653,7 +662,10 @@ func (c *floorFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi } retTp, argTp := getEvalTp4FloorAndCeil(args[0]) - bf := newBaseBuiltinFuncWithTp(ctx, args, retTp, argTp) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, retTp, argTp) + if err != nil { + return nil, err + } setFlag4FloorAndCeil(bf.tp, args[0]) bf.tp.Flen, bf.tp.Decimal = args[0].GetType().Flen, 0 switch argTp { @@ -814,10 +826,17 @@ func (c *logFunctionClass) getFunction(ctx sessionctx.Context, args []Expression argsLen = len(args) ) + var err error if argsLen == 1 { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } } else { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal, types.ETReal) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } } if argsLen == 1 { @@ -892,7 +911,10 @@ func (c *log2FunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinLog2Sig{bf} sig.setPbCode(tipb.ScalarFuncSig_Log2) return sig, nil @@ -929,7 +951,10 @@ func (c *log10FunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinLog10Sig{bf} sig.setPbCode(tipb.ScalarFuncSig_Log10) return sig, nil @@ -971,7 +996,10 @@ func (c *randFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if len(args) > 0 { argTps = []types.EvalType{types.ETInt} } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, argTps...) + if err != nil { + return nil, err + } bt := bf if len(args) == 0 { sig = &builtinRandSig{bt, &sync.Mutex{}, NewWithTime()} @@ -1057,7 +1085,10 @@ func (c *powFunctionClass) getFunction(ctx sessionctx.Context, args []Expression if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinPowSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Pow) return sig, nil @@ -1104,7 +1135,10 @@ func (c *convFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen = 64 sig := &builtinConvSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Conv) @@ -1211,7 +1245,10 @@ func (c *crc32FunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 10 bf.tp.Flag |= mysql.UnsignedFlag sig := &builtinCRC32Sig{bf} @@ -1248,7 +1285,10 @@ func (c *signFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinSignSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Sign) return sig, nil @@ -1288,7 +1328,10 @@ func (c *sqrtFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinSqrtSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Sqrt) return sig, nil @@ -1325,7 +1368,10 @@ func (c *acosFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinAcosSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Acos) return sig, nil @@ -1363,7 +1409,10 @@ func (c *asinFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinAsinSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Asin) return sig, nil @@ -1408,10 +1457,17 @@ func (c *atanFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio argsLen = len(args) ) + var err error if argsLen == 1 { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } } else { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal, types.ETReal) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } } if argsLen == 1 { @@ -1480,7 +1536,10 @@ func (c *cosFunctionClass) getFunction(ctx sessionctx.Context, args []Expression if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinCosSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Cos) return sig, nil @@ -1514,7 +1573,10 @@ func (c *cotFunctionClass) getFunction(ctx sessionctx.Context, args []Expression if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinCotSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Cot) return sig, nil @@ -1556,7 +1618,10 @@ func (c *degreesFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinDegreesSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Degrees) return sig, nil @@ -1591,7 +1656,10 @@ func (c *expFunctionClass) getFunction(ctx sessionctx.Context, args []Expression if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinExpSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Exp) return sig, nil @@ -1635,7 +1703,11 @@ func (c *piFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) sig builtinFunc ) - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETReal) + var err error + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal) + if err != nil { + return nil, err + } bf.tp.Decimal = 6 bf.tp.Flen = 8 sig = &builtinPISig{bf} @@ -1667,7 +1739,10 @@ func (c *radiansFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinRadiansSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Radians) return sig, nil @@ -1701,7 +1776,10 @@ func (c *sinFunctionClass) getFunction(ctx sessionctx.Context, args []Expression if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinSinSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Sin) return sig, nil @@ -1735,7 +1813,10 @@ func (c *tanFunctionClass) getFunction(ctx sessionctx.Context, args []Expression if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig := &builtinTanSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Tan) return sig, nil @@ -1775,7 +1856,10 @@ func (c *truncateFunctionClass) getFunction(ctx sessionctx.Context, args []Expre argTp = types.ETReal } - bf := newBaseBuiltinFuncWithTp(ctx, args, argTp, argTp, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, argTp, argTp, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Decimal = calculateDecimal4RoundAndTruncate(ctx, args, argTp) bf.tp.Flen = args[0].GetType().Flen - args[0].GetType().Decimal + bf.tp.Decimal diff --git a/expression/builtin_math_test.go b/expression/builtin_math_test.go index 06c227122a3f9..4d792bae3e4be 100644 --- a/expression/builtin_math_test.go +++ b/expression/builtin_math_test.go @@ -159,7 +159,7 @@ func (s *testEvaluatorSuite) TestExp(c *C) { } } - _, err := funcs[ast.Exp].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Exp].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -268,7 +268,7 @@ func (s *testEvaluatorSuite) TestLog(c *C) { } } - _, err := funcs[ast.Log].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Log].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -304,7 +304,7 @@ func (s *testEvaluatorSuite) TestLog2(c *C) { } } - _, err := funcs[ast.Log2].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Log2].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -340,7 +340,7 @@ func (s *testEvaluatorSuite) TestLog10(c *C) { } } - _, err := funcs[ast.Log10].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Log10].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -583,7 +583,7 @@ func (s *testEvaluatorSuite) TestConv(c *C) { c.Assert(r, Equals, t.ret) } - _, err := funcs[ast.Conv].getFunction(s.ctx, []Expression{Zero, Zero, Zero}) + _, err := funcs[ast.Conv].getFunction(s.ctx, []Expression{NewZero(), NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -657,7 +657,7 @@ func (s *testEvaluatorSuite) TestDegrees(c *C) { } } } - _, err := funcs[ast.Degrees].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Degrees].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -740,7 +740,7 @@ func (s *testEvaluatorSuite) TestSin(c *C) { {math.Pi / 6, float64(math.Sin(math.Pi / 6)), false, false}, // Pie/6(30 degrees) ==> 0.5 {-math.Pi / 6, float64(math.Sin(-math.Pi / 6)), false, false}, {math.Pi * 2, float64(math.Sin(math.Pi * 2)), false, false}, - {string("adfsdfgs"), 0, false, true}, + {"adfsdfgs", 0, false, true}, {"0.000", 0, false, false}, } @@ -761,7 +761,7 @@ func (s *testEvaluatorSuite) TestSin(c *C) { } } - _, err := funcs[ast.Sin].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Sin].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -799,7 +799,7 @@ func (s *testEvaluatorSuite) TestCos(c *C) { } } - _, err := funcs[ast.Cos].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Cos].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -835,7 +835,7 @@ func (s *testEvaluatorSuite) TestAcos(c *C) { } } - _, err := funcs[ast.Acos].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Acos].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -871,7 +871,7 @@ func (s *testEvaluatorSuite) TestAsin(c *C) { } } - _, err := funcs[ast.Asin].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Asin].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -907,7 +907,7 @@ func (s *testEvaluatorSuite) TestAtan(c *C) { } } - _, err := funcs[ast.Atan].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Atan].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -944,7 +944,7 @@ func (s *testEvaluatorSuite) TestTan(c *C) { } } - _, err := funcs[ast.Tan].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Tan].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -986,6 +986,6 @@ func (s *testEvaluatorSuite) TestCot(c *C) { } } - _, err := funcs[ast.Cot].getFunction(s.ctx, []Expression{One}) + _, err := funcs[ast.Cot].getFunction(s.ctx, []Expression{NewOne()}) c.Assert(err, IsNil) } diff --git a/expression/builtin_miscellaneous.go b/expression/builtin_miscellaneous.go index f7a4c1652b52f..9e40a70f11e3e 100644 --- a/expression/builtin_miscellaneous.go +++ b/expression/builtin_miscellaneous.go @@ -91,7 +91,10 @@ func (c *sleepFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETReal) + if err != nil { + return nil, err + } bf.tp.Flen = 21 sig := &builtinSleepSig{bf} return sig, nil @@ -144,7 +147,10 @@ func (c *lockFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString, types.ETInt) + if err != nil { + return nil, err + } sig := &builtinLockSig{bf} bf.tp.Flen = 1 return sig, nil @@ -176,7 +182,10 @@ func (c *releaseLockFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } sig := &builtinReleaseLockSig{bf} bf.tp.Flen = 1 return sig, nil @@ -209,7 +218,10 @@ func (c *anyValueFunctionClass) getFunction(ctx sessionctx.Context, args []Expre return nil, err } argTp := args[0].GetType().EvalType() - bf := newBaseBuiltinFuncWithTp(ctx, args, argTp, argTp) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, argTp, argTp) + if err != nil { + return nil, err + } args[0].GetType().Flag |= bf.tp.Flag *bf.tp = *args[0].GetType() var sig builtinFunc @@ -372,7 +384,10 @@ func (c *inetAtonFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 21 bf.tp.Flag |= mysql.UnsignedFlag sig := &builtinInetAtonSig{bf} @@ -445,7 +460,10 @@ func (c *inetNtoaFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 93 bf.tp.Decimal = 0 @@ -494,7 +512,10 @@ func (c *inet6AtonFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 16 types.SetBinChsClnFlag(bf.tp) bf.tp.Decimal = 0 @@ -563,7 +584,10 @@ func (c *inet6NtoaFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 117 bf.tp.Decimal = 0 @@ -588,7 +612,7 @@ func (b *builtinInet6NtoaSig) evalString(row chunk.Row) (string, bool, error) { if err != nil || isNull { return "", true, err } - ip := net.IP([]byte(val)).String() + ip := net.IP(val).String() if len(val) == net.IPv6len && !strings.Contains(ip, ":") { ip = fmt.Sprintf("::ffff:%s", ip) } @@ -616,7 +640,10 @@ func (c *isIPv4FunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 1 sig := &builtinIsIPv4Sig{bf} return sig, nil @@ -680,7 +707,10 @@ func (c *isIPv4CompatFunctionClass) getFunction(ctx sessionctx.Context, args []E if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 1 sig := &builtinIsIPv4CompatSig{bf} return sig, nil @@ -725,7 +755,10 @@ func (c *isIPv4MappedFunctionClass) getFunction(ctx sessionctx.Context, args []E if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 1 sig := &builtinIsIPv4MappedSig{bf} return sig, nil @@ -770,7 +803,10 @@ func (c *isIPv6FunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 1 sig := &builtinIsIPv6Sig{bf} return sig, nil @@ -825,7 +861,10 @@ func (c *nameConstFunctionClass) getFunction(ctx sessionctx.Context, args []Expr return nil, err } argTp := args[1].GetType().EvalType() - bf := newBaseBuiltinFuncWithTp(ctx, args, argTp, types.ETString, argTp) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, argTp, types.ETString, argTp) + if err != nil { + return nil, err + } *bf.tp = *args[1].GetType() var sig builtinFunc switch argTp { @@ -966,7 +1005,10 @@ func (c *uuidFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 36 sig := &builtinUUIDSig{bf} diff --git a/expression/builtin_miscellaneous_test.go b/expression/builtin_miscellaneous_test.go index b75bb6b352530..0a23283c1dbaa 100644 --- a/expression/builtin_miscellaneous_test.go +++ b/expression/builtin_miscellaneous_test.go @@ -320,7 +320,7 @@ func (s *testEvaluatorSuite) TestIsIPv4Compat(c *C) { func (s *testEvaluatorSuite) TestNameConst(c *C) { dec := types.NewDecFromFloatForTest(123.123) tm := types.NewTime(types.FromGoTime(time.Now()), mysql.TypeDatetime, 6) - du := types.Duration{Duration: time.Duration(12*time.Hour + 1*time.Minute + 1*time.Second), Fsp: types.DefaultFsp} + du := types.Duration{Duration: 12*time.Hour + 1*time.Minute + 1*time.Second, Fsp: types.DefaultFsp} cases := []struct { colName string arg interface{} diff --git a/expression/builtin_miscellaneous_vec.go b/expression/builtin_miscellaneous_vec.go index 681837862cf39..f591336ba7220 100644 --- a/expression/builtin_miscellaneous_vec.go +++ b/expression/builtin_miscellaneous_vec.go @@ -580,7 +580,7 @@ func (b *builtinInet6NtoaSig) vecEvalString(input *chunk.Chunk, result *chunk.Co continue } valI := val.GetString(i) - ip := net.IP([]byte(valI)).String() + ip := net.IP(valI).String() if len(valI) == net.IPv6len && !strings.Contains(ip, ":") { ip = fmt.Sprintf("::ffff:%s", ip) } diff --git a/expression/builtin_op.go b/expression/builtin_op.go index 602582ec74da6..867aebbb25660 100644 --- a/expression/builtin_op.go +++ b/expression/builtin_op.go @@ -67,16 +67,19 @@ func (c *logicAndFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err != nil { return nil, err } - args[0], err = wrapWithIsTrue(ctx, true, args[0]) + args[0], err = wrapWithIsTrue(ctx, true, args[0], false) if err != nil { return nil, errors.Trace(err) } - args[1], err = wrapWithIsTrue(ctx, true, args[1]) + args[1], err = wrapWithIsTrue(ctx, true, args[1], false) if err != nil { return nil, errors.Trace(err) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } sig := &builtinLogicAndSig{bf} sig.setPbCode(tipb.ScalarFuncSig_LogicalAnd) sig.tp.Flen = 1 @@ -117,16 +120,19 @@ func (c *logicOrFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err != nil { return nil, err } - args[0], err = wrapWithIsTrue(ctx, true, args[0]) + args[0], err = wrapWithIsTrue(ctx, true, args[0], false) if err != nil { return nil, errors.Trace(err) } - args[1], err = wrapWithIsTrue(ctx, true, args[1]) + args[1], err = wrapWithIsTrue(ctx, true, args[1], false) if err != nil { return nil, errors.Trace(err) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen = 1 sig := &builtinLogicOrSig{bf} sig.setPbCode(tipb.ScalarFuncSig_LogicalOr) @@ -173,8 +179,19 @@ func (c *logicXorFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err != nil { return nil, err } + args[0], err = wrapWithIsTrue(ctx, true, args[0], false) + if err != nil { + return nil, errors.Trace(err) + } + args[1], err = wrapWithIsTrue(ctx, true, args[1], false) + if err != nil { + return nil, errors.Trace(err) + } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } sig := &builtinLogicXorSig{bf} sig.setPbCode(tipb.ScalarFuncSig_LogicalXor) sig.tp.Flen = 1 @@ -215,7 +232,10 @@ func (c *bitAndFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } sig := &builtinBitAndSig{bf} sig.setPbCode(tipb.ScalarFuncSig_BitAndSig) sig.tp.Flag |= mysql.UnsignedFlag @@ -253,7 +273,10 @@ func (c *bitOrFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } sig := &builtinBitOrSig{bf} sig.setPbCode(tipb.ScalarFuncSig_BitOrSig) sig.tp.Flag |= mysql.UnsignedFlag @@ -291,7 +314,10 @@ func (c *bitXorFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } sig := &builtinBitXorSig{bf} sig.setPbCode(tipb.ScalarFuncSig_BitXorSig) sig.tp.Flag |= mysql.UnsignedFlag @@ -329,7 +355,10 @@ func (c *leftShiftFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } sig := &builtinLeftShiftSig{bf} sig.setPbCode(tipb.ScalarFuncSig_LeftShift) sig.tp.Flag |= mysql.UnsignedFlag @@ -367,7 +396,10 @@ func (c *rightShiftFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } sig := &builtinRightShiftSig{bf} sig.setPbCode(tipb.ScalarFuncSig_RightShift) sig.tp.Flag |= mysql.UnsignedFlag @@ -418,7 +450,10 @@ func (c *isTrueOrFalseFunctionClass) getFunction(ctx sessionctx.Context, args [] argTp = types.ETReal } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTp) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTp) + if err != nil { + return nil, err + } bf.tp.Flen = 1 var sig builtinFunc @@ -427,13 +462,25 @@ func (c *isTrueOrFalseFunctionClass) getFunction(ctx sessionctx.Context, args [] switch argTp { case types.ETReal: sig = &builtinRealIsTrueSig{bf, c.keepNull} - sig.setPbCode(tipb.ScalarFuncSig_RealIsTrue) + if c.keepNull { + sig.setPbCode(tipb.ScalarFuncSig_RealIsTrueWithNull) + } else { + sig.setPbCode(tipb.ScalarFuncSig_RealIsTrue) + } case types.ETDecimal: sig = &builtinDecimalIsTrueSig{bf, c.keepNull} - sig.setPbCode(tipb.ScalarFuncSig_DecimalIsTrue) + if c.keepNull { + sig.setPbCode(tipb.ScalarFuncSig_DecimalIsTrueWithNull) + } else { + sig.setPbCode(tipb.ScalarFuncSig_DecimalIsTrue) + } case types.ETInt: sig = &builtinIntIsTrueSig{bf, c.keepNull} - sig.setPbCode(tipb.ScalarFuncSig_IntIsTrue) + if c.keepNull { + sig.setPbCode(tipb.ScalarFuncSig_IntIsTrueWithNull) + } else { + sig.setPbCode(tipb.ScalarFuncSig_IntIsTrue) + } default: return nil, errors.Errorf("unexpected types.EvalType %v", argTp) } @@ -441,13 +488,25 @@ func (c *isTrueOrFalseFunctionClass) getFunction(ctx sessionctx.Context, args [] switch argTp { case types.ETReal: sig = &builtinRealIsFalseSig{bf, c.keepNull} - sig.setPbCode(tipb.ScalarFuncSig_RealIsFalse) + if c.keepNull { + sig.setPbCode(tipb.ScalarFuncSig_RealIsFalseWithNull) + } else { + sig.setPbCode(tipb.ScalarFuncSig_RealIsFalse) + } case types.ETDecimal: sig = &builtinDecimalIsFalseSig{bf, c.keepNull} - sig.setPbCode(tipb.ScalarFuncSig_DecimalIsFalse) + if c.keepNull { + sig.setPbCode(tipb.ScalarFuncSig_DecimalIsFalseWithNull) + } else { + sig.setPbCode(tipb.ScalarFuncSig_DecimalIsFalse) + } case types.ETInt: sig = &builtinIntIsFalseSig{bf, c.keepNull} - sig.setPbCode(tipb.ScalarFuncSig_IntIsFalse) + if c.keepNull { + sig.setPbCode(tipb.ScalarFuncSig_IntIsFalseWithNull) + } else { + sig.setPbCode(tipb.ScalarFuncSig_IntIsFalse) + } default: return nil, errors.Errorf("unexpected types.EvalType %v", argTp) } @@ -613,7 +672,10 @@ func (c *bitNegFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flag |= mysql.UnsignedFlag sig := &builtinBitNegSig{bf} sig.setPbCode(tipb.ScalarFuncSig_BitNegSig) @@ -654,7 +716,10 @@ func (c *unaryNotFunctionClass) getFunction(ctx sessionctx.Context, args []Expre argTp = types.ETReal } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTp) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTp) + if err != nil { + return nil, err + } bf.tp.Flen = 1 var sig builtinFunc @@ -791,32 +856,50 @@ func (c *unaryMinusFunctionClass) getFunction(ctx sessionctx.Context, args []Exp switch argExprTp.EvalType() { case types.ETInt: if intOverflow { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETDecimal, types.ETDecimal) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDecimal, types.ETDecimal) + if err != nil { + return nil, err + } sig = &builtinUnaryMinusDecimalSig{bf, true} sig.setPbCode(tipb.ScalarFuncSig_UnaryMinusDecimal) } else { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } sig = &builtinUnaryMinusIntSig{bf} sig.setPbCode(tipb.ScalarFuncSig_UnaryMinusInt) } bf.tp.Decimal = 0 case types.ETDecimal: - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETDecimal, types.ETDecimal) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDecimal, types.ETDecimal) + if err != nil { + return nil, err + } bf.tp.Decimal = argExprTp.Decimal sig = &builtinUnaryMinusDecimalSig{bf, false} sig.setPbCode(tipb.ScalarFuncSig_UnaryMinusDecimal) case types.ETReal: - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig = &builtinUnaryMinusRealSig{bf} sig.setPbCode(tipb.ScalarFuncSig_UnaryMinusReal) default: tp := argExpr.GetType().Tp if types.IsTypeTime(tp) || tp == mysql.TypeDuration { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETDecimal, types.ETDecimal) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDecimal, types.ETDecimal) + if err != nil { + return nil, err + } sig = &builtinUnaryMinusDecimalSig{bf, false} sig.setPbCode(tipb.ScalarFuncSig_UnaryMinusDecimal) } else { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETReal, types.ETReal) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETReal, types.ETReal) + if err != nil { + return nil, err + } sig = &builtinUnaryMinusRealSig{bf} sig.setPbCode(tipb.ScalarFuncSig_UnaryMinusReal) } @@ -904,7 +987,10 @@ func (c *isNullFunctionClass) getFunction(ctx sessionctx.Context, args []Express } else if argTp == types.ETJson { argTp = types.ETString } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTp) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTp) + if err != nil { + return nil, err + } bf.tp.Flen = 1 var sig builtinFunc switch argTp { diff --git a/expression/builtin_op_test.go b/expression/builtin_op_test.go index 80df488338fd8..9055f88449b9b 100644 --- a/expression/builtin_op_test.go +++ b/expression/builtin_op_test.go @@ -59,7 +59,7 @@ func (s *testEvaluatorSuite) TestUnary(c *C) { } } - _, err := funcs[ast.UnaryMinus].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.UnaryMinus].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -118,10 +118,10 @@ func (s *testEvaluatorSuite) TestLogicAnd(c *C) { } // Test incorrect parameter count. - _, err := newFunctionForTest(s.ctx, ast.LogicAnd, Zero) + _, err := newFunctionForTest(s.ctx, ast.LogicAnd, NewZero()) c.Assert(err, NotNil) - _, err = funcs[ast.LogicAnd].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err = funcs[ast.LogicAnd].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -187,10 +187,10 @@ func (s *testEvaluatorSuite) TestRightShift(c *C) { } // Test incorrect parameter count. - _, err := newFunctionForTest(s.ctx, ast.RightShift, Zero) + _, err := newFunctionForTest(s.ctx, ast.RightShift, NewZero()) c.Assert(err, NotNil) - _, err = funcs[ast.RightShift].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err = funcs[ast.RightShift].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -225,10 +225,10 @@ func (s *testEvaluatorSuite) TestBitXor(c *C) { } // Test incorrect parameter count. - _, err := newFunctionForTest(s.ctx, ast.Xor, Zero) + _, err := newFunctionForTest(s.ctx, ast.Xor, NewZero()) c.Assert(err, NotNil) - _, err = funcs[ast.Xor].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err = funcs[ast.Xor].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -270,10 +270,10 @@ func (s *testEvaluatorSuite) TestBitOr(c *C) { } // Test incorrect parameter count. - _, err := newFunctionForTest(s.ctx, ast.Or, Zero) + _, err := newFunctionForTest(s.ctx, ast.Or, NewZero()) c.Assert(err, NotNil) - _, err = funcs[ast.Or].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err = funcs[ast.Or].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -336,10 +336,10 @@ func (s *testEvaluatorSuite) TestLogicOr(c *C) { } // Test incorrect parameter count. - _, err := newFunctionForTest(s.ctx, ast.LogicOr, Zero) + _, err := newFunctionForTest(s.ctx, ast.LogicOr, NewZero()) c.Assert(err, NotNil) - _, err = funcs[ast.LogicOr].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err = funcs[ast.LogicOr].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -374,10 +374,10 @@ func (s *testEvaluatorSuite) TestBitAnd(c *C) { } // Test incorrect parameter count. - _, err := newFunctionForTest(s.ctx, ast.And, Zero) + _, err := newFunctionForTest(s.ctx, ast.And, NewZero()) c.Assert(err, NotNil) - _, err = funcs[ast.And].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err = funcs[ast.And].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -419,10 +419,10 @@ func (s *testEvaluatorSuite) TestBitNeg(c *C) { } // Test incorrect parameter count. - _, err := newFunctionForTest(s.ctx, ast.BitNeg, Zero, Zero) + _, err := newFunctionForTest(s.ctx, ast.BitNeg, NewZero(), NewZero()) c.Assert(err, NotNil) - _, err = funcs[ast.BitNeg].getFunction(s.ctx, []Expression{Zero}) + _, err = funcs[ast.BitNeg].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -470,10 +470,10 @@ func (s *testEvaluatorSuite) TestUnaryNot(c *C) { } // Test incorrect parameter count. - _, err := newFunctionForTest(s.ctx, ast.UnaryNot, Zero, Zero) + _, err := newFunctionForTest(s.ctx, ast.UnaryNot, NewZero(), NewZero()) c.Assert(err, NotNil) - _, err = funcs[ast.UnaryNot].getFunction(s.ctx, []Expression{Zero}) + _, err = funcs[ast.UnaryNot].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -589,13 +589,13 @@ func (s *testEvaluatorSuite) TestLogicXor(c *C) { {[]interface{}{0, nil}, 0, true, false}, {[]interface{}{nil, 0}, 0, true, false}, {[]interface{}{nil, 1}, 0, true, false}, - {[]interface{}{0.5000, 0.4999}, 1, false, false}, + {[]interface{}{0.5000, 0.4999}, 0, false, false}, {[]interface{}{0.5000, 1.0}, 0, false, false}, - {[]interface{}{0.4999, 1.0}, 1, false, false}, + {[]interface{}{0.4999, 1.0}, 0, false, false}, {[]interface{}{nil, 0.000}, 0, true, false}, {[]interface{}{nil, 0.001}, 0, true, false}, {[]interface{}{types.NewDecFromStringForTest("0.000001"), 0.00001}, 0, false, false}, - {[]interface{}{types.NewDecFromStringForTest("0.000001"), 1}, 1, false, false}, + {[]interface{}{types.NewDecFromStringForTest("0.000001"), 1}, 0, false, false}, {[]interface{}{types.NewDecFromStringForTest("0.000000"), nil}, 0, true, false}, {[]interface{}{types.NewDecFromStringForTest("0.000001"), nil}, 0, true, false}, @@ -619,9 +619,9 @@ func (s *testEvaluatorSuite) TestLogicXor(c *C) { } // Test incorrect parameter count. - _, err := newFunctionForTest(s.ctx, ast.LogicXor, Zero) + _, err := newFunctionForTest(s.ctx, ast.LogicXor, NewZero()) c.Assert(err, NotNil) - _, err = funcs[ast.LogicXor].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err = funcs[ast.LogicXor].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) } diff --git a/expression/builtin_other.go b/expression/builtin_other.go index 5d2f9de8ad89b..e8f9c0edafdad 100644 --- a/expression/builtin_other.go +++ b/expression/builtin_other.go @@ -78,7 +78,10 @@ func (c *inFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) for i := range args { argTps[i] = args[0].GetType().EvalType() } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen = 1 switch args[0].GetType().EvalType() { case types.ETInt: @@ -639,7 +642,10 @@ func (c *rowFunctionClass) getFunction(ctx sessionctx.Context, args []Expression for i := range argTps { argTps[i] = args[i].GetType().EvalType() } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } sig = &builtinRowSig{bf} return sig, nil } @@ -667,7 +673,10 @@ func (c *setVarFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err = c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = args[1].GetType().Flen // TODO: we should consider the type of the argument, but not take it as string for all situations. sig = &builtinSetVarSig{bf} @@ -719,7 +728,10 @@ func (c *getVarFunctionClass) getFunction(ctx sessionctx.Context, args []Express return nil, err } // TODO: we should consider the type of the argument, but not take it as string for all situations. - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = mysql.MaxFieldVarCharLength if err := c.resolveCollation(ctx, args, &bf); err != nil { return nil, err @@ -786,7 +798,10 @@ func (c *valuesFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err = c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFunc(ctx, args) + bf, err := newBaseBuiltinFunc(ctx, c.funcName, args) + if err != nil { + return nil, err + } bf.tp = c.tp switch c.tp.EvalType() { case types.ETInt: @@ -1058,7 +1073,10 @@ func (c *bitCountFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen = 2 sig := &builtinBitCountSig{bf} return sig, nil @@ -1098,7 +1116,10 @@ func (c *getParamFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen = mysql.MaxFieldVarCharLength sig := &builtinGetParamStringSig{bf} return sig, nil diff --git a/expression/builtin_other_test.go b/expression/builtin_other_test.go index 147656756df82..05e921216d367 100644 --- a/expression/builtin_other_test.go +++ b/expression/builtin_other_test.go @@ -51,7 +51,7 @@ func (s *testEvaluatorSuite) TestBitCount(c *C) { {float64(-1.1), int64(64)}, {float64(-3.1), int64(63)}, {uint64(math.MaxUint64), int64(64)}, - {string("xxx"), int64(0)}, + {"xxx", int64(0)}, {nil, nil}, } for _, test := range bitCountCases { @@ -83,10 +83,10 @@ func (s *testEvaluatorSuite) TestInFunc(c *C) { time2 := types.NewTime(types.FromGoTime(time.Date(2017, 1, 2, 1, 1, 1, 1, time.UTC)), mysql.TypeDatetime, 6) time3 := types.NewTime(types.FromGoTime(time.Date(2017, 1, 3, 1, 1, 1, 1, time.UTC)), mysql.TypeDatetime, 6) time4 := types.NewTime(types.FromGoTime(time.Date(2017, 1, 4, 1, 1, 1, 1, time.UTC)), mysql.TypeDatetime, 6) - duration1 := types.Duration{Duration: time.Duration(12*time.Hour + 1*time.Minute + 1*time.Second)} - duration2 := types.Duration{Duration: time.Duration(12*time.Hour + 1*time.Minute)} - duration3 := types.Duration{Duration: time.Duration(12*time.Hour + 1*time.Second)} - duration4 := types.Duration{Duration: time.Duration(12 * time.Hour)} + duration1 := types.Duration{Duration: 12*time.Hour + 1*time.Minute + 1*time.Second} + duration2 := types.Duration{Duration: 12*time.Hour + 1*time.Minute} + duration3 := types.Duration{Duration: 12*time.Hour + 1*time.Second} + duration4 := types.Duration{Duration: 12 * time.Hour} json1 := json.CreateBinary("123") json2 := json.CreateBinary("123.1") json3 := json.CreateBinary("123.2") diff --git a/expression/builtin_string.go b/expression/builtin_string.go index 1d4c7636f3a31..4ec157a0e9359 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -188,7 +188,10 @@ func (c *lengthFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 10 sig := &builtinLengthSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Length) @@ -223,7 +226,10 @@ func (c *asciiFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 3 sig := &builtinASCIISig{bf} sig.setPbCode(tipb.ScalarFuncSig_ASCII) @@ -265,7 +271,10 @@ func (c *concatFunctionClass) getFunction(ctx sessionctx.Context, args []Express for i := 0; i < len(args); i++ { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen = 0 for i := range args { argType := args[i].GetType() @@ -335,7 +344,10 @@ func (c *concatWSFunctionClass) getFunction(ctx sessionctx.Context, args []Expre argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen = 0 for i := range args { @@ -438,7 +450,10 @@ func (c *leftFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETInt) + if err != nil { + return nil, err + } argType := args[0].GetType() bf.tp.Flen = argType.Flen SetBinFlagOrBinStr(argType, bf.tp) @@ -520,7 +535,10 @@ func (c *rightFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETInt) + if err != nil { + return nil, err + } argType := args[0].GetType() bf.tp.Flen = argType.Flen SetBinFlagOrBinStr(argType, bf.tp) @@ -603,7 +621,10 @@ func (c *repeatFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen = mysql.MaxBlobWidth SetBinFlagOrBinStr(args[0].GetType(), bf.tp) valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket) @@ -666,7 +687,10 @@ func (c *lowerFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } argTp := args[0].GetType() bf.tp.Flen = argTp.Flen SetBinFlagOrBinStr(argTp, bf.tp) @@ -708,7 +732,10 @@ func (c *reverseFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } retTp := *args[0].GetType() retTp.Tp = mysql.TypeVarString retTp.Decimal = types.UnspecifiedLength @@ -774,7 +801,10 @@ func (c *spaceFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = mysql.MaxBlobWidth valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket) @@ -829,7 +859,10 @@ func (c *upperFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } argTp := args[0].GetType() bf.tp.Flen = argTp.Flen SetBinFlagOrBinStr(argTp, bf.tp) @@ -894,7 +927,10 @@ func (c *strcmpFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 2 types.SetBinChsClnFlag(bf.tp) sig := &builtinStrcmpSig{bf} @@ -941,7 +977,10 @@ func (c *replaceFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = c.fixLength(args) for _, a := range args { SetBinFlagOrBinStr(a.GetType(), bf.tp) @@ -1003,7 +1042,10 @@ func (c *convertFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETString) + if err != nil { + return nil, err + } charsetArg, ok := args[1].(*Constant) if !ok { @@ -1018,7 +1060,6 @@ func (c *convertFunctionClass) getFunction(ctx sessionctx.Context, args []Expres // See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert // Here in syntax CONVERT(expr USING transcoding_name), behavior is kept the same, // picking the default collation of target charset. - var err error bf.tp.Collate, err = charset.GetDefaultCollation(bf.tp.Charset) if err != nil { return nil, errUnknownCharacterSet.GenWithStackByArgs(transcodingName) @@ -1081,7 +1122,10 @@ func (c *substringFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if len(args) == 3 { argTps = append(argTps, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } argType := args[0].GetType() bf.tp.Flen = argType.Flen @@ -1270,7 +1314,10 @@ func (c *substringIndexFunctionClass) getFunction(ctx sessionctx.Context, args [ if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETString, types.ETInt) + if err != nil { + return nil, err + } argType := args[0].GetType() bf.tp.Flen = argType.Flen SetBinFlagOrBinStr(argType, bf.tp) @@ -1347,7 +1394,10 @@ func (c *locateFunctionClass) getFunction(ctx sessionctx.Context, args []Express if hasStartPos { argTps = append(argTps, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } var sig builtinFunc // Loacte is multibyte safe, and is case-sensitive only if at least one argument is a binary string. hasBianryInput := types.IsBinaryStr(args[0].GetType()) || types.IsBinaryStr(args[1].GetType()) @@ -1424,7 +1474,7 @@ func (b *builtinLocate2ArgsUTF8Sig) evalInt(row chunk.Row) (int64, bool, error) if int64(len([]rune(subStr))) == 0 { return 1, false, nil } - slice := string([]rune(strings.ToLower(str))) + slice := strings.ToLower(str) ret, idx := 0, strings.Index(slice, strings.ToLower(subStr)) if idx != -1 { ret = utf8.RuneCountInString(slice[:idx]) + 1 @@ -1526,14 +1576,20 @@ func (c *hexFunctionClass) getFunction(ctx sessionctx.Context, args []Expression argTp := args[0].GetType().EvalType() switch argTp { case types.ETString, types.ETDatetime, types.ETTimestamp, types.ETDuration, types.ETJson: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } // Use UTF-8 as default bf.tp.Flen = args[0].GetType().Flen * 3 * 2 sig := &builtinHexStrArgSig{bf} sig.setPbCode(tipb.ScalarFuncSig_HexStrArg) return sig, nil case types.ETInt, types.ETReal, types.ETDecimal: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen = args[0].GetType().Flen * 2 bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() sig := &builtinHexIntArgSig{bf} @@ -1610,7 +1666,10 @@ func (c *unhexFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi return nil, errors.Errorf("Unhex invalid args, need int or string but get %s", argType) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = retFlen types.SetBinChsClnFlag(bf.tp) sig := &builtinUnHexSig{bf} @@ -1664,7 +1723,10 @@ func (c *trimFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio switch len(args) { case 1: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } argType := args[0].GetType() bf.tp.Flen = argType.Flen SetBinFlagOrBinStr(argType, bf.tp) @@ -1673,7 +1735,10 @@ func (c *trimFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio return sig, nil case 2: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETString) + if err != nil { + return nil, err + } argType := args[0].GetType() SetBinFlagOrBinStr(argType, bf.tp) sig := &builtinTrim2ArgsSig{bf} @@ -1681,7 +1746,10 @@ func (c *trimFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio return sig, nil case 3: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETString, types.ETInt) + if err != nil { + return nil, err + } argType := args[0].GetType() bf.tp.Flen = argType.Flen SetBinFlagOrBinStr(argType, bf.tp) @@ -1805,7 +1873,10 @@ func (c *lTrimFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } argType := args[0].GetType() bf.tp.Flen = argType.Flen SetBinFlagOrBinStr(argType, bf.tp) @@ -1842,7 +1913,10 @@ func (c *rTrimFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } argType := args[0].GetType() bf.tp.Flen = argType.Flen SetBinFlagOrBinStr(argType, bf.tp) @@ -1913,7 +1987,10 @@ func (c *lpadFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = getFlen4LpadAndRpad(bf.ctx, args[1]) SetBinFlagOrBinStr(args[0].GetType(), bf.tp) SetBinFlagOrBinStr(args[2].GetType(), bf.tp) @@ -2043,7 +2120,10 @@ func (c *rpadFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = getFlen4LpadAndRpad(bf.ctx, args[1]) SetBinFlagOrBinStr(args[0].GetType(), bf.tp) SetBinFlagOrBinStr(args[2].GetType(), bf.tp) @@ -2172,7 +2252,10 @@ func (c *bitLengthFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 10 sig := &builtinBitLengthSig{bf} sig.setPbCode(tipb.ScalarFuncSig_BitLength) @@ -2213,7 +2296,10 @@ func (c *charFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio argTps = append(argTps, types.ETInt) } argTps = append(argTps, types.ETString) - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen = 4 * (len(args) - 1) types.SetBinChsClnFlag(bf.tp) @@ -2298,7 +2384,10 @@ func (c *charLengthFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if argsErr := c.verifyArgs(args); argsErr != nil { return nil, argsErr } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } if types.IsBinaryStr(args[0].GetType()) { sig := &builtinCharLengthBinarySig{bf} sig.setPbCode(tipb.ScalarFuncSig_CharLength) @@ -2357,7 +2446,10 @@ func (c *findInSetFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 3 sig := &builtinFindInSetSig{bf} sig.setPbCode(tipb.ScalarFuncSig_FindInSet) @@ -2427,7 +2519,10 @@ func (c *fieldFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi for i, length := 0, len(args); i < length; i++ { argTps[i] = argTp } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } var sig builtinFunc switch argTp { case types.ETReal: @@ -2566,7 +2661,10 @@ func (c *makeSetFunctionClass) getFunction(ctx sessionctx.Context, args []Expres for i, length := 1, len(args); i < length; i++ { argTps[i] = types.ETString } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } for i, length := 0, len(args); i < length; i++ { SetBinFlagOrBinStr(args[i].GetType(), bf.tp) } @@ -2624,13 +2722,19 @@ func (c *octFunctionClass) getFunction(ctx sessionctx.Context, args []Expression } var sig builtinFunc if IsBinaryLiteral(args[0]) || args[0].GetType().EvalType() == types.ETInt { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen, bf.tp.Decimal = 64, types.UnspecifiedLength sig = &builtinOctIntSig{bf} sig.setPbCode(tipb.ScalarFuncSig_OctInt) } else { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen, bf.tp.Decimal = 64, types.UnspecifiedLength sig = &builtinOctStringSig{bf} @@ -2710,7 +2814,10 @@ func (c *ordFunctionClass) getFunction(ctx sessionctx.Context, args []Expression if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 10 sig := &builtinOrdSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Ord) @@ -2787,7 +2894,10 @@ func (c *quoteFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } SetBinFlagOrBinStr(args[0].GetType(), bf.tp) bf.tp.Flen = 2*args[0].GetType().Flen + 2 if bf.tp.Flen > mysql.MaxBlobWidth { @@ -2855,7 +2965,10 @@ func (c *binFunctionClass) getFunction(ctx sessionctx.Context, args []Expression if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 64 sig := &builtinBinSig{bf} @@ -2896,7 +3009,10 @@ func (c *eltFunctionClass) getFunction(ctx sessionctx.Context, args []Expression for i := 1; i < len(args); i++ { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } for _, arg := range args[1:] { argType := arg.GetType() if types.IsBinaryStr(argType) { @@ -2954,7 +3070,10 @@ func (c *exportSetFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if len(args) > 4 { argTps = append(argTps, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen = mysql.MaxBlobWidth switch len(args) { case 3: @@ -3117,7 +3236,10 @@ func (c *formatFunctionClass) getFunction(ctx sessionctx.Context, args []Express if len(args) == 3 { argTps = append(argTps, types.ETString) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = mysql.MaxBlobWidth var sig builtinFunc @@ -3280,7 +3402,10 @@ func (c *fromBase64FunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = mysql.MaxBlobWidth valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket) @@ -3353,7 +3478,10 @@ func (c *toBase64FunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = base64NeededEncodedLength(bf.args[0].GetType().Flen) @@ -3453,7 +3581,10 @@ func (c *insertFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err = c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETInt, types.ETInt, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETInt, types.ETInt, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = mysql.MaxBlobWidth SetBinFlagOrBinStr(args[0].GetType(), bf.tp) SetBinFlagOrBinStr(args[3].GetType(), bf.tp) @@ -3586,7 +3717,10 @@ func (c *instrFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 11 if types.IsBinaryStr(bf.args[0].GetType()) || types.IsBinaryStr(bf.args[1].GetType()) { sig := &builtinInstrSig{bf} @@ -3737,7 +3871,10 @@ func (c *weightStringFunctionClass) getFunction(ctx sessionctx.Context, args []E argTps[2] = types.ETInt } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argTps...) + if err != nil { + return nil, err + } var sig builtinFunc if padding == weightStringPaddingNull { sig = &builtinWeightStringNullSig{bf} diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index f7b149fe21872..e5693654f458f 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -49,7 +49,7 @@ func (s *testEvaluatorSuite) TestLengthAndOctetLength(c *C) { {types.NewTime(types.FromGoTime(time.Now()), mysql.TypeDatetime, 6), 26, false, false}, {types.NewBinaryLiteralFromUint(0x01, -1), 1, false, false}, {types.Set{Value: 1, Name: "abc"}, 3, false, false}, - {types.Duration{Duration: time.Duration(12*time.Hour + 1*time.Minute + 1*time.Second), Fsp: types.DefaultFsp}, 8, false, false}, + {types.Duration{Duration: 12*time.Hour + 1*time.Minute + 1*time.Second, Fsp: types.DefaultFsp}, 8, false, false}, {nil, 0, true, false}, {errors.New("must error"), 0, false, true}, } @@ -73,7 +73,7 @@ func (s *testEvaluatorSuite) TestLengthAndOctetLength(c *C) { } } - _, err := funcs[ast.Length].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Length].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -109,7 +109,7 @@ func (s *testEvaluatorSuite) TestASCII(c *C) { } } } - _, err := funcs[ast.Length].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Length].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -133,7 +133,7 @@ func (s *testEvaluatorSuite) TestConcat(c *C) { types.NewDecFromFloatForTest(1.1), types.NewTime(types.FromDate(2000, 1, 1, 12, 01, 01, 0), mysql.TypeDatetime, types.DefaultFsp), types.Duration{ - Duration: time.Duration(12*time.Hour + 1*time.Minute + 1*time.Second), + Duration: 12*time.Hour + 1*time.Minute + 1*time.Second, Fsp: types.DefaultFsp}, }, false, false, "ab121.11.21.12000-01-01 12:01:0112:01:01", @@ -251,7 +251,7 @@ func (s *testEvaluatorSuite) TestConcatWS(c *C) { types.NewDecFromFloatForTest(1.1), types.NewTime(types.FromDate(2000, 1, 1, 12, 01, 01, 0), mysql.TypeDatetime, types.DefaultFsp), types.Duration{ - Duration: time.Duration(12*time.Hour + 1*time.Minute + 1*time.Second), + Duration: 12*time.Hour + 1*time.Minute + 1*time.Second, Fsp: types.DefaultFsp}, }, false, false, "a,b,1,2,1.1,0.11,1.1,2000-01-01 12:01:01,12:01:01", @@ -703,7 +703,7 @@ func (s *testEvaluatorSuite) TestReplace(c *C) { } } - _, err := funcs[ast.Replace].getFunction(s.ctx, []Expression{Zero, Zero, Zero}) + _, err := funcs[ast.Replace].getFunction(s.ctx, []Expression{NewZero(), NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -747,10 +747,10 @@ func (s *testEvaluatorSuite) TestSubstring(c *C) { } } - _, err := funcs[ast.Substring].getFunction(s.ctx, []Expression{Zero, Zero, Zero}) + _, err := funcs[ast.Substring].getFunction(s.ctx, []Expression{NewZero(), NewZero(), NewZero()}) c.Assert(err, IsNil) - _, err = funcs[ast.Substring].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err = funcs[ast.Substring].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -855,7 +855,7 @@ func (s *testEvaluatorSuite) TestSubstringIndex(c *C) { } } - _, err := funcs[ast.SubstringIndex].getFunction(s.ctx, []Expression{Zero, Zero, Zero}) + _, err := funcs[ast.SubstringIndex].getFunction(s.ctx, []Expression{NewZero(), NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -900,7 +900,7 @@ func (s *testEvaluatorSuite) TestSpace(c *C) { } } - _, err := funcs[ast.Space].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Space].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -1044,13 +1044,13 @@ func (s *testEvaluatorSuite) TestTrim(c *C) { } } - _, err := funcs[ast.Trim].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Trim].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) - _, err = funcs[ast.Trim].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err = funcs[ast.Trim].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) - _, err = funcs[ast.Trim].getFunction(s.ctx, []Expression{Zero, Zero, Zero}) + _, err = funcs[ast.Trim].getFunction(s.ctx, []Expression{NewZero(), NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -1091,7 +1091,7 @@ func (s *testEvaluatorSuite) TestLTrim(c *C) { } } - _, err := funcs[ast.LTrim].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.LTrim].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -1130,7 +1130,7 @@ func (s *testEvaluatorSuite) TestRTrim(c *C) { } } - _, err := funcs[ast.RTrim].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.RTrim].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -1212,7 +1212,7 @@ func (s *testEvaluatorSuite) TestUnhexFunc(c *C) { } } - _, err := funcs[ast.Unhex].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Unhex].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -1244,7 +1244,7 @@ func (s *testEvaluatorSuite) TestBitLength(c *C) { } } - _, err := funcs[ast.BitLength].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.BitLength].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -1852,27 +1852,27 @@ func (s *testEvaluatorSuite) TestFromBase64(c *C) { args interface{} expect interface{} }{ - {string(""), string("")}, - {string("YWJj"), string("abc")}, - {string("YWIgYw=="), string("ab c")}, - {string("YWIKYw=="), string("ab\nc")}, - {string("YWIJYw=="), string("ab\tc")}, - {string("cXdlcnR5MTIzNDU2"), string("qwerty123456")}, + {"", ""}, + {"YWJj", "abc"}, + {"YWIgYw==", "ab c"}, + {"YWIKYw==", "ab\nc"}, + {"YWIJYw==", "ab\tc"}, + {"cXdlcnR5MTIzNDU2", "qwerty123456"}, { - string("QUJDREVGR0hJSktMTU5PUFFSU1RVVldYWVphYmNkZWZnaGlqa2xtbm9wcXJzdHV2d3h5ejAxMjM0\nNTY3ODkrL0FCQ0RFRkdISUpLTE1OT1BRUlNUVVZXWFlaYWJjZGVmZ2hpamtsbW5vcHFyc3R1dnd4\neXowMTIzNDU2Nzg5Ky9BQkNERUZHSElKS0xNTk9QUVJTVFVWV1hZWmFiY2RlZmdoaWprbG1ub3Bx\ncnN0dXZ3eHl6MDEyMzQ1Njc4OSsv"), - string("ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/"), + "QUJDREVGR0hJSktMTU5PUFFSU1RVVldYWVphYmNkZWZnaGlqa2xtbm9wcXJzdHV2d3h5ejAxMjM0\nNTY3ODkrL0FCQ0RFRkdISUpLTE1OT1BRUlNUVVZXWFlaYWJjZGVmZ2hpamtsbW5vcHFyc3R1dnd4\neXowMTIzNDU2Nzg5Ky9BQkNERUZHSElKS0xNTk9QUVJTVFVWV1hZWmFiY2RlZmdoaWprbG1ub3Bx\ncnN0dXZ3eHl6MDEyMzQ1Njc4OSsv", + "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/", }, { - string("QUJDREVGR0hJSktMTU5PUFFSU1RVVldYWVphYmNkZWZnaGlqa2xtbm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw=="), - string("ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/"), + "QUJDREVGR0hJSktMTU5PUFFSU1RVVldYWVphYmNkZWZnaGlqa2xtbm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw==", + "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/", }, { - string("QUJDREVGR0hJSktMTU5PUFFSU1RVVldYWVphYmNkZWZnaGlqa2xtbm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw=="), - string("ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/"), + "QUJDREVGR0hJSktMTU5PUFFSU1RVVldYWVphYmNkZWZnaGlqa2xtbm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw==", + "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/", }, { - string("QUJDREVGR0hJSkt\tMTU5PUFFSU1RVVld\nYWVphYmNkZ\rWZnaGlqa2xt bm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw=="), - string("ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/"), + "QUJDREVGR0hJSkt\tMTU5PUFFSU1RVVld\nYWVphYmNkZ\rWZnaGlqa2xt bm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw==", + "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/", }, } fc := funcs[ast.FromBase64] @@ -1902,16 +1902,16 @@ func (s *testEvaluatorSuite) TestFromBase64Sig(c *C) { isNil bool maxAllowPacket uint64 }{ - {string("YWJj"), string("abc"), false, 3}, - {string("YWJj"), "", true, 2}, + {"YWJj", "abc", false, 3}, + {"YWJj", "", true, 2}, { - string("QUJDREVGR0hJSkt\tMTU5PUFFSU1RVVld\nYWVphYmNkZ\rWZnaGlqa2xt bm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw=="), - string("ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/"), + "QUJDREVGR0hJSkt\tMTU5PUFFSU1RVVld\nYWVphYmNkZ\rWZnaGlqa2xt bm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw==", + "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/", false, 70, }, { - string("QUJDREVGR0hJSkt\tMTU5PUFFSU1RVVld\nYWVphYmNkZ\rWZnaGlqa2xt bm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw=="), + "QUJDREVGR0hJSkt\tMTU5PUFFSU1RVVld\nYWVphYmNkZ\rWZnaGlqa2xt bm9wcXJzdHV2d3h5ejAxMjM0NTY3ODkrLw==", "", true, 69, @@ -2024,7 +2024,7 @@ func (s *testEvaluatorSuite) TestOrd(c *C) { } } } - _, err := funcs[ast.Ord].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Ord].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -2203,7 +2203,7 @@ func (s *testEvaluatorSuite) TestToBase64(c *C) { } } - _, err := funcs[ast.ToBase64].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.ToBase64].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } diff --git a/expression/builtin_string_vec.go b/expression/builtin_string_vec.go index a45268ddb1d1c..6d5e713a1e333 100644 --- a/expression/builtin_string_vec.go +++ b/expression/builtin_string_vec.go @@ -2161,7 +2161,7 @@ func (b *builtinLocate2ArgsUTF8Sig) vecEvalInt(input *chunk.Chunk, result *chunk i64s[i] = 1 continue } - slice := string([]rune(str)) + slice := str slice = strings.ToLower(slice) subStr = strings.ToLower(subStr) idx := strings.Index(slice, subStr) diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 811b76a0b418c..fb8a479536cec 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -267,7 +267,10 @@ func (c *dateFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, 10, 0 sig := &builtinDateSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Date) @@ -325,7 +328,10 @@ func (c *dateLiteralFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, []Expression{}, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, []Expression{}, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, 10, 0 sig := &builtinDateLiteralSig{bf, tm} return sig, nil @@ -363,7 +369,10 @@ func (c *dateDiffFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime, types.ETDatetime) + if err != nil { + return nil, err + } sig := &builtinDateDiffSig{bf} sig.setPbCode(tipb.ScalarFuncSig_DateDiff) return sig, nil @@ -422,7 +431,10 @@ func (c *timeDiffFunctionClass) getFunction(ctx sessionctx.Context, args []Expre arg0FieldTp, arg1FieldTp := args[0].GetType(), args[1].GetType() arg0Tp, arg1Tp := c.getArgEvalTp(arg0FieldTp), c.getArgEvalTp(arg1FieldTp) - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, arg0Tp, arg1Tp) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration, arg0Tp, arg1Tp) + if err != nil { + return nil, err + } arg0Dec, err := getExpressionFsp(ctx, args[0]) if err != nil { @@ -777,7 +789,10 @@ func (c *dateFormatFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETDatetime, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETDatetime, types.ETString) + if err != nil { + return nil, err + } // worst case: formatMask=%r%r%r...%r, each %r takes 11 characters bf.tp.Flen = (args[1].GetType().Flen + 1) / 2 * 11 sig := &builtinDateFormatSig{bf} @@ -837,7 +852,10 @@ func (c *fromDaysFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 10, 0 sig := &builtinFromDaysSig{bf} sig.setPbCode(tipb.ScalarFuncSig_FromDays) @@ -873,7 +891,10 @@ func (c *hourFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDuration) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDuration) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 3, 0 sig := &builtinHourSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Hour) @@ -909,7 +930,10 @@ func (c *minuteFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDuration) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDuration) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 2, 0 sig := &builtinMinuteSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Minute) @@ -945,7 +969,10 @@ func (c *secondFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDuration) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDuration) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 2, 0 sig := &builtinSecondSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Second) @@ -981,7 +1008,10 @@ func (c *microSecondFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDuration) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDuration) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 6, 0 sig := &builtinMicroSecondSig{bf} sig.setPbCode(tipb.ScalarFuncSig_MicroSecond) @@ -1017,7 +1047,10 @@ func (c *monthFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 2, 0 sig := &builtinMonthSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Month) @@ -1062,7 +1095,10 @@ func (c *monthNameFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 10 sig := &builtinMonthNameSig{bf} @@ -1102,7 +1138,10 @@ func (c *dayNameFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() bf.tp.Flen = 10 sig := &builtinDayNameSig{bf} @@ -1169,7 +1208,10 @@ func (c *dayOfMonthFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen = 2 sig := &builtinDayOfMonthSig{bf} sig.setPbCode(tipb.ScalarFuncSig_DayOfMonth) @@ -1210,7 +1252,10 @@ func (c *dayOfWeekFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen = 1 sig := &builtinDayOfWeekSig{bf} sig.setPbCode(tipb.ScalarFuncSig_DayOfWeek) @@ -1249,7 +1294,10 @@ func (c *dayOfYearFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen = 3 sig := &builtinDayOfYearSig{bf} sig.setPbCode(tipb.ScalarFuncSig_DayOfYear) @@ -1294,7 +1342,10 @@ func (c *weekFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio argTps = append(argTps, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 2, 0 @@ -1386,7 +1437,10 @@ func (c *weekDayFunctionClass) getFunction(ctx sessionctx.Context, args []Expres return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen = 1 sig := &builtinWeekDaySig{bf} @@ -1426,7 +1480,10 @@ func (c *weekOfYearFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 2, 0 sig := &builtinWeekOfYearSig{bf} sig.setPbCode(tipb.ScalarFuncSig_WeekOfYear) @@ -1468,7 +1525,10 @@ func (c *yearFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 4, 0 sig := &builtinYearSig{bf} sig.setPbCode(tipb.ScalarFuncSig_Year) @@ -1516,7 +1576,10 @@ func (c *yearWeekFunctionClass) getFunction(ctx sessionctx.Context, args []Expre argTps = append(argTps, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 6, 0 @@ -1615,7 +1678,10 @@ func (c *fromUnixTimeFunctionClass) getFunction(ctx sessionctx.Context, args []E } isArg0Str := args[0].GetType().EvalType() == types.ETString - bf := newBaseBuiltinFuncWithTp(ctx, args, retTp, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, retTp, argTps...) + if err != nil { + return nil, err + } if len(args) > 1 { bf.tp.Flen = args[1].GetType().Flen @@ -1739,7 +1805,10 @@ func (c *getFormatFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Flen = 17 sig := &builtinGetFormatSig{bf} sig.setPbCode(tipb.ScalarFuncSig_GetFormat) @@ -1807,12 +1876,18 @@ func (c *strToDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expr retTp, fsp := c.getRetTp(ctx, args[1]) switch retTp { case mysql.TypeDate: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, int(types.MinFsp) sig = &builtinStrToDateDateSig{bf} sig.setPbCode(tipb.ScalarFuncSig_StrToDateDate) case mysql.TypeDatetime: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, types.ETString, types.ETString) + if err != nil { + return nil, err + } if fsp == types.MinFsp { bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeWidthNoFsp, int(types.MinFsp) } else { @@ -1821,7 +1896,10 @@ func (c *strToDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expr sig = &builtinStrToDateDatetimeSig{bf} sig.setPbCode(tipb.ScalarFuncSig_StrToDateDatetime) case mysql.TypeDuration: - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration, types.ETString, types.ETString) + if err != nil { + return nil, err + } if fsp == types.MinFsp { bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthNoFsp, int(types.MinFsp) } else { @@ -1947,7 +2025,10 @@ func (c *sysDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if len(args) == 1 { argTps = append(argTps, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 19, 0 var sig builtinFunc @@ -2018,7 +2099,10 @@ func (c *currentDateFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 10, 0 sig := &builtinCurrentDateSig{bf} return sig, nil @@ -2057,7 +2141,10 @@ func (c *currentTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Ex } if len(args) == 0 { - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthNoFsp, int(types.MinFsp) sig = &builtinCurrentTime0ArgSig{bf} sig.setPbCode(tipb.ScalarFuncSig_CurrentTime0Arg) @@ -2077,7 +2164,10 @@ func (c *currentTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Ex return nil, errors.Errorf("Invalid negative %d specified, must in [0, 6].", fsp) } } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthWithFsp, int(fsp) sig = &builtinCurrentTime1ArgSig{bf} sig.setPbCode(tipb.ScalarFuncSig_CurrentTime1Arg) @@ -2145,7 +2235,10 @@ func (c *timeFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio if err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration, types.ETString) + if err != nil { + return nil, err + } bf.tp.Decimal, err = getExpressionFsp(ctx, args[0]) if err != nil { return nil, err @@ -2216,7 +2309,10 @@ func (c *timeLiteralFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, []Expression{}, types.ETDuration) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, []Expression{}, types.ETDuration) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 10, int(duration.Fsp) if int(duration.Fsp) > 0 { bf.tp.Flen += 1 + int(duration.Fsp) @@ -2250,7 +2346,10 @@ func (c *utcDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = 10, 0 sig := &builtinUTCDateSig{bf} return sig, nil @@ -2309,7 +2408,10 @@ func (c *utcTimestampFunctionClass) getFunction(ctx sessionctx.Context, args []E if len(args) == 1 { argTps = append(argTps, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, argTps...) + if err != nil { + return nil, err + } if len(args) == 1 { bf.tp.Flen, bf.tp.Decimal = getFlenAndDecimal4UTCTimestampAndNow(bf.ctx, args[0]) @@ -2398,7 +2500,10 @@ func (c *nowFunctionClass) getFunction(ctx sessionctx.Context, args []Expression if len(args) == 1 { argTps = append(argTps, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, argTps...) + if err != nil { + return nil, err + } if len(args) == 1 { bf.tp.Flen, bf.tp.Decimal = getFlenAndDecimal4UTCTimestampAndNow(bf.ctx, args[0]) @@ -2524,11 +2629,17 @@ func (c *extractFunctionClass) getFunction(ctx sessionctx.Context, args []Expres } var bf baseBuiltinFunc if isDatetimeUnit { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString, types.ETDatetime) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString, types.ETDatetime) + if err != nil { + return nil, err + } sig = &builtinExtractDatetimeSig{bf} sig.setPbCode(tipb.ScalarFuncSig_ExtractDatetime) } else { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString, types.ETDuration) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString, types.ETDuration) + if err != nil { + return nil, err + } sig = &builtinExtractDurationSig{bf} sig.setPbCode(tipb.ScalarFuncSig_ExtractDuration) } @@ -3163,14 +3274,20 @@ func (c *addDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres } // Otherwise, the fsp should be 0. } - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, argTps...) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration, argTps...) + if err != nil { + return nil, err + } arg0Dec, err := getExpressionFsp(ctx, args[0]) if err != nil { return nil, err } bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthWithFsp, mathutil.Max(arg0Dec, internalFsp) } else { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, argTps...) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeFullWidth, types.UnspecifiedLength } @@ -3831,14 +3948,20 @@ func (c *subDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres } // Otherwise, the fsp should be 0. } - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, argTps...) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration, argTps...) + if err != nil { + return nil, err + } arg0Dec, err := getExpressionFsp(ctx, args[0]) if err != nil { return nil, err } bf.tp.Flen, bf.tp.Decimal = mysql.MaxDurationWidthWithFsp, mathutil.Max(arg0Dec, internalFsp) } else { - bf = newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, argTps...) + bf, err = newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeFullWidth, types.UnspecifiedLength } @@ -4468,7 +4591,10 @@ func (c *timestampDiffFunctionClass) getFunction(ctx sessionctx.Context, args [] if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETString, types.ETDatetime, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETString, types.ETDatetime, types.ETDatetime) + if err != nil { + return nil, err + } sig := &builtinTimestampDiffSig{bf} sig.setPbCode(tipb.ScalarFuncSig_TimestampDiff) return sig, nil @@ -4564,7 +4690,10 @@ func (c *unixTimestampFunctionClass) getFunction(ctx sessionctx.Context, args [] panic("Unexpected retTp") } - bf := newBaseBuiltinFuncWithTp(ctx, args, retTp, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, retTp, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen = retFLen bf.tp.Decimal = retDecimal @@ -4741,7 +4870,10 @@ func (c *timestampFunctionClass) getFunction(ctx sessionctx.Context, args []Expr case mysql.TypeFloat, mysql.TypeDouble, mysql.TypeDecimal: isFloat = true } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, evalTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, evalTps...) + if err != nil { + return nil, err + } bf.tp.Decimal, bf.tp.Flen = -1, 19 if fsp != 0 { bf.tp.Flen += 1 + int(fsp) @@ -4863,7 +4995,10 @@ func (c *timestampLiteralFunctionClass) getFunction(ctx sessionctx.Context, args if err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, []Expression{}, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, []Expression{}, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeWidthNoFsp, int(tm.Fsp()) if tm.Fsp() > 0 { bf.tp.Flen += int(tm.Fsp()) + 1 @@ -4906,7 +5041,7 @@ func getFsp4TimeAddSub(s string) int8 { // getBf4TimeAddSub parses input types, generates baseBuiltinFunc and set related attributes for // builtin function 'ADDTIME' and 'SUBTIME' -func getBf4TimeAddSub(ctx sessionctx.Context, args []Expression) (tp1, tp2 *types.FieldType, bf baseBuiltinFunc, err error) { +func getBf4TimeAddSub(ctx sessionctx.Context, funcName string, args []Expression) (tp1, tp2 *types.FieldType, bf baseBuiltinFunc, err error) { tp1, tp2 = args[0].GetType(), args[1].GetType() var argTp1, argTp2, retTp types.EvalType switch tp1.Tp { @@ -4934,7 +5069,10 @@ func getBf4TimeAddSub(ctx sessionctx.Context, args []Expression) (tp1, tp2 *type return } - bf = newBaseBuiltinFuncWithTp(ctx, args, retTp, argTp1, argTp2) + bf, err = newBaseBuiltinFuncWithTp(ctx, funcName, args, retTp, argTp1, argTp2) + if err != nil { + return + } bf.tp.Decimal = mathutil.Min(mathutil.Max(arg0Dec, arg1Dec), int(types.MaxFsp)) if retTp == types.ETString { bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeString, mysql.MaxDatetimeWidthWithFsp, types.UnspecifiedLength @@ -5039,7 +5177,7 @@ func (c *addTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err = c.verifyArgs(args); err != nil { return nil, err } - tp1, tp2, bf, err := getBf4TimeAddSub(ctx, args) + tp1, tp2, bf, err := getBf4TimeAddSub(ctx, c.funcName, args) if err != nil { return nil, err } @@ -5470,7 +5608,10 @@ func (c *convertTzFunctionClass) getFunction(ctx sessionctx.Context, args []Expr } decimal := c.getDecimal(ctx, args[0]) - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, types.ETDatetime, types.ETString, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, types.ETDatetime, types.ETString, types.ETString) + if err != nil { + return nil, err + } bf.tp.Decimal = decimal sig := &builtinConvertTzSig{ baseBuiltinFunc: bf, @@ -5501,15 +5642,22 @@ func (b *builtinConvertTzSig) evalTime(row chunk.Row) (types.Time, bool, error) } fromTzStr, isNull, err := b.args[1].EvalString(b.ctx, row) - if isNull || err != nil || fromTzStr == "" { + if isNull || err != nil { return types.ZeroTime, true, nil } toTzStr, isNull, err := b.args[2].EvalString(b.ctx, row) - if isNull || err != nil || toTzStr == "" { + if isNull || err != nil { return types.ZeroTime, true, nil } + return b.convertTz(dt, fromTzStr, toTzStr) +} + +func (b *builtinConvertTzSig) convertTz(dt types.Time, fromTzStr, toTzStr string) (types.Time, bool, error) { + if fromTzStr == "" || toTzStr == "" { + return types.ZeroTime, true, nil + } fromTzMatched := b.timezoneRegex.MatchString(fromTzStr) toTzMatched := b.timezoneRegex.MatchString(toTzStr) @@ -5550,7 +5698,10 @@ func (c *makeDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } tp := bf.tp tp.Tp, tp.Flen, tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, 0 sig := &builtinMakeDateSig{bf} @@ -5636,7 +5787,10 @@ func (c *makeTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Expre } else { arg1Type = types.ETInt } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, arg0Type, arg1Type, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration, arg0Type, arg1Type, types.ETReal) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = flen, decimal sig := &builtinMakeTimeSig{bf} sig.setPbCode(tipb.ScalarFuncSig_MakeTime) @@ -5729,7 +5883,10 @@ func (c *periodAddFunctionClass) getFunction(ctx sessionctx.Context, args []Expr return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen = 6 sig := &builtinPeriodAddSig{bf} return sig, nil @@ -5813,7 +5970,10 @@ func (c *periodDiffFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Flen = 6 sig := &builtinPeriodDiffSig{bf} return sig, nil @@ -5862,7 +6022,10 @@ func (c *quarterFunctionClass) getFunction(ctx sessionctx.Context, args []Expres return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Flen = 1 sig := &builtinQuarterSig{bf} @@ -5929,7 +6092,10 @@ func (c *secToTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if retFsp > 0 { retFlen += 1 + retFsp } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, types.ETReal) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration, types.ETReal) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = retFlen, retFsp sig := &builtinSecToTimeSig{bf} sig.setPbCode(tipb.ScalarFuncSig_SecToTime) @@ -5996,7 +6162,7 @@ func (c *subTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err = c.verifyArgs(args); err != nil { return nil, err } - tp1, tp2, bf, err := getBf4TimeAddSub(ctx, args) + tp1, tp2, bf, err := getBf4TimeAddSub(ctx, c.funcName, args) if err != nil { return nil, err } @@ -6410,7 +6576,10 @@ func (c *timeFormatFunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETDuration, types.ETString) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETDuration, types.ETString) + if err != nil { + return nil, err + } // worst case: formatMask=%r%r%r...%r, each %r takes 11 characters bf.tp.Flen = (args[1].GetType().Flen + 1) / 2 * 11 sig := &builtinTimeFormatSig{bf} @@ -6460,7 +6629,10 @@ func (c *timeToSecFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDuration) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDuration) + if err != nil { + return nil, err + } bf.tp.Flen = 10 sig := &builtinTimeToSecSig{bf} sig.setPbCode(tipb.ScalarFuncSig_TimeToSec) @@ -6501,7 +6673,10 @@ func (c *timestampAddFunctionClass) getFunction(ctx sessionctx.Context, args []E if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETString, types.ETString, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp = &types.FieldType{Tp: mysql.TypeString, Flen: mysql.MaxDatetimeWidthNoFsp, Decimal: types.UnspecifiedLength} sig := &builtinTimestampAddSig{bf} sig.setPbCode(tipb.ScalarFuncSig_TimestampAdd) @@ -6595,7 +6770,10 @@ func (c *toDaysFunctionClass) getFunction(ctx sessionctx.Context, args []Express if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } sig := &builtinToDaysSig{bf} sig.setPbCode(tipb.ScalarFuncSig_ToDays) return sig, nil @@ -6634,7 +6812,10 @@ func (c *toSecondsFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETInt, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETDatetime) + if err != nil { + return nil, err + } sig := &builtinToSecondsSig{bf} sig.setPbCode(tipb.ScalarFuncSig_ToSeconds) return sig, nil @@ -6699,7 +6880,10 @@ func (c *utcTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if len(args) == 1 { argTps = append(argTps, types.ETInt) } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDuration, argTps...) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration, argTps...) + if err != nil { + return nil, err + } bf.tp.Flen, bf.tp.Decimal = c.getFlenAndDecimal4UTCTime(bf.ctx, args) var sig builtinFunc @@ -6773,7 +6957,10 @@ func (c *lastDayFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err := c.verifyArgs(args); err != nil { return nil, err } - bf := newBaseBuiltinFuncWithTp(ctx, args, types.ETDatetime, types.ETDatetime) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, types.ETDatetime) + if err != nil { + return nil, err + } bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, int(types.DefaultFsp) sig := &builtinLastDaySig{bf} sig.setPbCode(tipb.ScalarFuncSig_LastDay) @@ -6830,7 +7017,10 @@ func (c *tidbParseTsoFunctionClass) getFunction(ctx sessionctx.Context, args []E return nil, err } argTp := args[0].GetType().EvalType() - bf := newBaseBuiltinFuncWithTp(ctx, args, argTp, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, argTp, types.ETInt) + if err != nil { + return nil, err + } bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, int(types.DefaultFsp) sig := &builtinTidbParseTsoSig{bf} diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index 39c771c3bcde5..575b354faef9c 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -397,7 +397,7 @@ func (s *testEvaluatorSuite) TestMonthName(c *C) { } } - _, err := funcs[ast.MonthName].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.MonthName].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -434,7 +434,7 @@ func (s *testEvaluatorSuite) TestDayName(c *C) { } } - _, err := funcs[ast.DayName].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.DayName].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -469,7 +469,7 @@ func (s *testEvaluatorSuite) TestDayOfWeek(c *C) { } } - _, err := funcs[ast.DayOfWeek].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.DayOfWeek].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -504,7 +504,7 @@ func (s *testEvaluatorSuite) TestDayOfMonth(c *C) { } } - _, err := funcs[ast.DayOfMonth].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.DayOfMonth].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -539,7 +539,7 @@ func (s *testEvaluatorSuite) TestDayOfYear(c *C) { } } - _, err := funcs[ast.DayOfYear].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.DayOfYear].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -738,7 +738,7 @@ func (s *testEvaluatorSuite) TestTime(c *C) { c.Assert(tp.Tp, Equals, mysql.TypeDuration) c.Assert(tp.Charset, Equals, charset.CharsetBin) c.Assert(tp.Collate, Equals, charset.CollationBin) - c.Assert(tp.Flag&uint(mysql.BinaryFlag), Equals, uint(mysql.BinaryFlag)) + c.Assert(tp.Flag&mysql.BinaryFlag, Equals, mysql.BinaryFlag) c.Assert(tp.Flen, Equals, mysql.MaxDurationWidthWithFsp) d, err := f.Eval(chunk.Row{}) if t.getErr { @@ -753,7 +753,7 @@ func (s *testEvaluatorSuite) TestTime(c *C) { } } - _, err := funcs[ast.Time].getFunction(s.ctx, []Expression{Zero}) + _, err := funcs[ast.Time].getFunction(s.ctx, []Expression{NewZero()}) c.Assert(err, IsNil) } @@ -1163,18 +1163,18 @@ func (s *testEvaluatorSuite) TestFromUnixTime(c *C) { format string expect string }{ - {false, 1451606400, 0, 0, "", "2016-01-01 08:00:00"}, - {true, 1451606400, 123456000, 1451606400.123456, "", "2016-01-01 08:00:00.123456"}, - {true, 1451606400, 999999000, 1451606400.999999, "", "2016-01-01 08:00:00.999999"}, - {true, 1451606400, 999999900, 1451606400.9999999, "", "2016-01-01 08:00:01.000000"}, - {false, 1451606400, 0, 0, `%Y %D %M %h:%i:%s %x`, "2016-01-01 08:00:00"}, - {true, 1451606400, 123456000, 1451606400.123456, `%Y %D %M %h:%i:%s %x`, "2016-01-01 08:00:00.123456"}, - {true, 1451606400, 999999000, 1451606400.999999, `%Y %D %M %h:%i:%s %x`, "2016-01-01 08:00:00.999999"}, - {true, 1451606400, 999999900, 1451606400.9999999, `%Y %D %M %h:%i:%s %x`, "2016-01-01 08:00:01.000000"}, + {false, 1451606400, 0, 0, "", "2016-01-01 00:00:00"}, + {true, 1451606400, 123456000, 1451606400.123456, "", "2016-01-01 00:00:00.123456"}, + {true, 1451606400, 999999000, 1451606400.999999, "", "2016-01-01 00:00:00.999999"}, + {true, 1451606400, 999999900, 1451606400.9999999, "", "2016-01-01 00:00:01.000000"}, + {false, 1451606400, 0, 0, `%Y %D %M %h:%i:%s %x`, "2016-01-01 00:00:00"}, + {true, 1451606400, 123456000, 1451606400.123456, `%Y %D %M %h:%i:%s %x`, "2016-01-01 00:00:00.123456"}, + {true, 1451606400, 999999000, 1451606400.999999, `%Y %D %M %h:%i:%s %x`, "2016-01-01 00:00:00.999999"}, + {true, 1451606400, 999999900, 1451606400.9999999, `%Y %D %M %h:%i:%s %x`, "2016-01-01 00:00:01.000000"}, } sc := s.ctx.GetSessionVars().StmtCtx originTZ := sc.TimeZone - sc.TimeZone = time.Local + sc.TimeZone = time.UTC defer func() { sc.TimeZone = originTZ }() @@ -1508,7 +1508,7 @@ func (s *testEvaluatorSuite) TestTimeDiff(c *C) { c.Assert(tp.Tp, Equals, mysql.TypeDuration) c.Assert(tp.Charset, Equals, charset.CharsetBin) c.Assert(tp.Collate, Equals, charset.CollationBin) - c.Assert(tp.Flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(tp.Flag, Equals, mysql.BinaryFlag) c.Assert(tp.Flen, Equals, mysql.MaxDurationWidthWithFsp) d, err := f.Eval(chunk.Row{}) if t.getErr { @@ -1523,7 +1523,7 @@ func (s *testEvaluatorSuite) TestTimeDiff(c *C) { } } } - _, err := funcs[ast.TimeDiff].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err := funcs[ast.TimeDiff].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) } @@ -2062,7 +2062,7 @@ func (s *testEvaluatorSuite) TestMakeDate(c *C) { c.Assert(tp.Tp, Equals, mysql.TypeDate) c.Assert(tp.Charset, Equals, charset.CharsetBin) c.Assert(tp.Collate, Equals, charset.CollationBin) - c.Assert(tp.Flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(tp.Flag, Equals, mysql.BinaryFlag) c.Assert(tp.Flen, Equals, mysql.MaxDateWidth) d, err := f.Eval(chunk.Row{}) if t.getErr { @@ -2077,7 +2077,7 @@ func (s *testEvaluatorSuite) TestMakeDate(c *C) { } } - _, err := funcs[ast.MakeDate].getFunction(s.ctx, []Expression{Zero, Zero}) + _, err := funcs[ast.MakeDate].getFunction(s.ctx, []Expression{NewZero(), NewZero()}) c.Assert(err, IsNil) } diff --git a/expression/builtin_time_vec.go b/expression/builtin_time_vec.go index 9766e903e93d6..976251cd0f075 100644 --- a/expression/builtin_time_vec.go +++ b/expression/builtin_time_vec.go @@ -978,7 +978,7 @@ func (b *builtinWeekWithModeSig) vecEvalInt(input *chunk.Chunk, result *chunk.Co continue } mode := int(ms[i]) - week := date.Week(int(mode)) + week := date.Week(mode) i64s[i] = int64(week) } return nil @@ -1382,7 +1382,7 @@ func (b *builtinTimeToSecSig) vecEvalInt(input *chunk.Chunk, result *chunk.Colum continue } var sign int - duration := buf.GetDuration(i, int(fsp)) + duration := buf.GetDuration(i, fsp) if duration.Duration >= 0 { sign = 1 } else { @@ -2519,11 +2519,50 @@ func (b *builtinUTCTimestampWithoutArgSig) vecEvalTime(input *chunk.Chunk, resul } func (b *builtinConvertTzSig) vectorized() bool { - return false + return true } func (b *builtinConvertTzSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { - return errors.Errorf("not implemented") + n := input.NumRows() + if err := b.args[0].VecEvalTime(b.ctx, input, result); err != nil { + return err + } + + fromTzBuf, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + defer b.bufAllocator.put(fromTzBuf) + if err := b.args[1].VecEvalString(b.ctx, input, fromTzBuf); err != nil { + return err + } + + toTzBuf, err := b.bufAllocator.get(types.ETString, n) + if err != nil { + return err + } + defer b.bufAllocator.put(toTzBuf) + if err := b.args[2].VecEvalString(b.ctx, input, toTzBuf); err != nil { + return err + } + + result.MergeNulls(fromTzBuf, toTzBuf) + ts := result.Times() + var isNull bool + for i := 0; i < n; i++ { + if result.IsNull(i) { + continue + } + + ts[i], isNull, err = b.convertTz(ts[i], fromTzBuf.GetString(i), toTzBuf.GetString(i)) + if err != nil { + return err + } + if isNull { + result.SetNull(i, true) + } + } + return nil } func (b *builtinTimestamp1ArgSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { diff --git a/expression/builtin_time_vec_test.go b/expression/builtin_time_vec_test.go index 1cfb80bdcf253..55a829c13556a 100644 --- a/expression/builtin_time_vec_test.go +++ b/expression/builtin_time_vec_test.go @@ -15,6 +15,7 @@ package expression import ( "math" + "math/rand" "testing" . "github.com/pingcap/check" @@ -71,6 +72,22 @@ func newDateTimeUnitStrGener() *dateTimeUnitStrGener { return &dateTimeUnitStrGener{newDefaultRandGen()} } +// tzStrGener is used to generate strings which are timezones +type tzStrGener struct{} + +func (g *tzStrGener) gen() interface{} { + tzs := []string{ + "", + "GMT", + "MET", + "+00:00", + "+10:00", + } + + n := rand.Int() % len(tzs) + return tzs[n] +} + func (g *dateTimeUnitStrGener) gen() interface{} { dateTimes := []string{ "DAY", @@ -422,6 +439,10 @@ var vecBuiltinTimeCases = map[string][]vecExprBenchCase{ constants: []*Constant{{Value: types.NewStringDatum("HOUR_MINUTE"), RetType: types.NewFieldType(mysql.TypeString)}}, }, }, + ast.ConvertTz: { + {retEvalType: types.ETDatetime, childrenTypes: []types.EvalType{types.ETDatetime, types.ETString, types.ETString}, + geners: []dataGenerator{nil, newNullWrappedGener(0.2, &tzStrGener{}), newNullWrappedGener(0.2, &tzStrGener{})}}, + }, } func (s *testVectorizeSuite2) TestVectorizedBuiltinTimeEvalOneVec(c *C) { diff --git a/expression/builtin_vectorized_test.go b/expression/builtin_vectorized_test.go index f92c7f9c24cbc..4a9d5d10b8cfc 100644 --- a/expression/builtin_vectorized_test.go +++ b/expression/builtin_vectorized_test.go @@ -84,7 +84,10 @@ func genMockVecPlusIntBuiltinFunc() (*mockVecPlusIntBuiltinFunc, *chunk.Chunk, * col1.Index, col1.RetType = 0, tp col2 := newColumn(1) col2.Index, col2.RetType = 1, tp - bf := newBaseBuiltinFuncWithTp(mock.NewContext(), []Expression{col1, col2}, types.ETInt, types.ETInt, types.ETInt) + bf, err := newBaseBuiltinFuncWithTp(mock.NewContext(), "", []Expression{col1, col2}, types.ETInt, types.ETInt, types.ETInt) + if err != nil { + panic(err) + } plus := &mockVecPlusIntBuiltinFunc{bf, nil, false} input := chunk.New([]*types.FieldType{tp, tp}, 1024, 1024) buf := chunk.NewColumn(types.NewFieldType(mysql.TypeLonglong), 1024) @@ -399,7 +402,10 @@ func genMockRowDouble(eType types.EvalType, enableVec bool) (builtinFunc, *chunk col1 := newColumn(1) col1.Index = 0 col1.RetType = tp - bf := newBaseBuiltinFuncWithTp(mock.NewContext(), []Expression{col1}, eType, eType) + bf, err := newBaseBuiltinFuncWithTp(mock.NewContext(), "", []Expression{col1}, eType, eType) + if err != nil { + return nil, nil, nil, err + } rowDouble := &mockBuiltinDouble{bf, eType, enableVec} input := chunk.New([]*types.FieldType{tp}, 1024, 1024) buf := chunk.NewColumn(types.NewFieldType(convertETType(eType)), 1024) diff --git a/expression/collation.go b/expression/collation.go index 6f88b3b18ec6d..f16d0e0f3d627 100644 --- a/expression/collation.go +++ b/expression/collation.go @@ -122,6 +122,18 @@ var ( charset.CollationUTF8MB4: 5, charset.CollationBin: 6, } + + // CollationStrictness indicates the strictness of comparison of the collation. The unequal order in a weak collation also holds in a strict collation. + // For example, if a < b in a weak collation(e.g. general_ci), then there must be a < b in a strict collation(e.g. _bin). + CollationStrictness = map[string]int{ + "utf8_general_ci": 0, + "utf8mb4_general_ci": 0, + charset.CollationASCII: 1, + charset.CollationLatin1: 1, + charset.CollationUTF8: 1, + charset.CollationUTF8MB4: 1, + charset.CollationBin: 2, + } ) func deriveCoercibilityForScarlarFunc(sf *ScalarFunction) Coercibility { diff --git a/expression/column.go b/expression/column.go index 986ed8b7b66a7..b0bfcab631de3 100644 --- a/expression/column.go +++ b/expression/column.go @@ -448,7 +448,7 @@ func (col *Column) HashCode(_ *stmtctx.StatementContext) []byte { } col.hashcode = make([]byte, 0, 9) col.hashcode = append(col.hashcode, columnFlag) - col.hashcode = codec.EncodeInt(col.hashcode, int64(col.UniqueID)) + col.hashcode = codec.EncodeInt(col.hashcode, col.UniqueID) return col.hashcode } diff --git a/expression/constant.go b/expression/constant.go index 28bb4a66f0ab2..121e4829eef78 100644 --- a/expression/constant.go +++ b/expression/constant.go @@ -28,25 +28,29 @@ import ( "go.uber.org/zap" ) -var ( - // One stands for a number 1. - One = &Constant{ +// NewOne stands for a number 1. +func NewOne() *Constant { + return &Constant{ Value: types.NewDatum(1), RetType: types.NewFieldType(mysql.TypeTiny), } +} - // Zero stands for a number 0. - Zero = &Constant{ +// NewZero stands for a number 0. +func NewZero() *Constant { + return &Constant{ Value: types.NewDatum(0), RetType: types.NewFieldType(mysql.TypeTiny), } +} - // Null stands for null constant. - Null = &Constant{ +// NewNull stands for null constant. +func NewNull() *Constant { + return &Constant{ Value: types.NewDatum(nil), RetType: types.NewFieldType(mysql.TypeTiny), } -) +} // Constant stands for a constant value. type Constant struct { @@ -99,20 +103,17 @@ func (c *Constant) MarshalJSON() ([]byte, error) { // Clone implements Expression interface. func (c *Constant) Clone() Expression { - if c.DeferredExpr != nil || c.ParamMarker != nil { - con := *c - return &con - } - return c + con := *c + return &con } // GetType implements Expression interface. func (c *Constant) GetType() *types.FieldType { - if c.ParamMarker != nil { + if p := c.ParamMarker; p != nil && !p.ctx.GetSessionVars().StmtCtx.InExplainStmt { // GetType() may be called in multi-threaded context, e.g, in building inner executors of IndexJoin, // so it should avoid data race. We achieve this by returning different FieldType pointer for each call. tp := types.NewFieldType(mysql.TypeUnspecified) - dt := c.ParamMarker.GetUserVar() + dt := p.GetUserVar() types.DefaultParamTypeForValue(dt.GetValue(), tp) return tp } @@ -176,8 +177,13 @@ func (c *Constant) VecEvalJSON(ctx sessionctx.Context, input *chunk.Chunk, resul } func (c *Constant) getLazyDatum() (dt types.Datum, isLazy bool, err error) { - if c.ParamMarker != nil { - dt = c.ParamMarker.GetUserVar() + if p := c.ParamMarker; p != nil { + if p.ctx.GetSessionVars().StmtCtx.InExplainStmt { + // Since `ParamMarker` is not nil only in prepare/execute context, the query must be `explain for connection` when coming here. + // The PreparedParams may have been reset already, to avoid panic, we just use the pre-evaluated datum for this constant. + return dt, false, nil + } + dt = p.GetUserVar() isLazy = true return } else if c.DeferredExpr != nil { diff --git a/expression/constant_fold.go b/expression/constant_fold.go index 139a6ac87b6f8..83065bae9302f 100644 --- a/expression/constant_fold.go +++ b/expression/constant_fold.go @@ -169,7 +169,7 @@ func foldConstant(expr Expression) (Expression, bool) { if argIsConst[i] { constArgs[i] = arg } else { - constArgs[i] = One + constArgs[i] = NewOne() } } dummyScalarFunc, err := NewFunctionBase(x.GetCtx(), x.FuncName.L, x.GetType(), constArgs...) diff --git a/expression/constant_propagation.go b/expression/constant_propagation.go index 27e8ebd60f00e..9ee6f3bdfee63 100644 --- a/expression/constant_propagation.go +++ b/expression/constant_propagation.go @@ -119,7 +119,7 @@ func validEqualCond(ctx sessionctx.Context, cond Expression) (*Column, *Constant // for 'a, b, a < 3', it returns 'true, false, b < 3' // for 'a, b, sin(a) + cos(a) = 5', it returns 'true, false, returns sin(b) + cos(b) = 5' // for 'a, b, cast(a) < rand()', it returns 'false, true, cast(a) < rand()' -func tryToReplaceCond(ctx sessionctx.Context, src *Column, tgt *Column, cond Expression) (bool, bool, Expression) { +func tryToReplaceCond(ctx sessionctx.Context, src *Column, tgt *Column, cond Expression, rejectControl bool) (bool, bool, Expression) { sf, ok := cond.(*ScalarFunction) if !ok { return false, false, cond @@ -132,10 +132,17 @@ func tryToReplaceCond(ctx sessionctx.Context, src *Column, tgt *Column, cond Exp if _, ok := inequalFunctions[sf.FuncName.L]; ok { return false, true, cond } + // See https://github.com/pingcap/tidb/issues/15782. The control function's result may rely on the original nullable + // information of the outer side column. Its args cannot be replaced easily. + // A more strict check is that after we replace the arg. We check the nullability of the new expression. + // But we haven't maintained it yet, so don't replace the arg of the control function currently. + if rejectControl && (sf.FuncName.L == ast.Ifnull || sf.FuncName.L == ast.If || sf.FuncName.L == ast.Case) { + return false, false, cond + } for idx, expr := range sf.GetArgs() { if src.Equal(nil, expr) { _, coll, _ := cond.CharsetAndCollation(ctx) - if !collate.CompatibleCollate(tgt.GetType().Collate, coll) { + if tgt.GetType().Collate != coll { continue } replaced = true @@ -145,7 +152,7 @@ func tryToReplaceCond(ctx sessionctx.Context, src *Column, tgt *Column, cond Exp } args[idx] = tgt } else { - subReplaced, isNonDeterministic, subExpr := tryToReplaceCond(ctx, src, tgt, expr) + subReplaced, isNonDeterministic, subExpr := tryToReplaceCond(ctx, src, tgt, expr, rejectControl) if isNonDeterministic { return false, true, cond } else if subReplaced { @@ -190,20 +197,6 @@ func (s *propConstSolver) propagateConstantEQ() { } for i, cond := range s.conditions { if !visited[i] { - // Make sure the collations of the argument are equal. - if sf, ok := cond.(*ScalarFunction); ok && len(sf.GetArgs()) > 1 { - baseCollation := sf.GetArgs()[0].GetType().Collate - skip := false - for _, arg := range sf.GetArgs() { - if !collate.CompatibleCollate(arg.GetType().Collate, baseCollation) { - skip = true - break - } - } - if skip { - continue - } - } s.conditions[i] = ColumnSubstitute(cond, NewSchema(cols...), cons) } } @@ -235,7 +228,7 @@ func (s *propConstSolver) propagateColumnEQ() { if fun, ok := s.conditions[i].(*ScalarFunction); ok && fun.FuncName.L == ast.EQ { lCol, lOk := fun.GetArgs()[0].(*Column) rCol, rOk := fun.GetArgs()[1].(*Column) - if lOk && rOk && collate.CompatibleCollate(lCol.GetType().Collate, rCol.GetType().Collate) { + if lOk && rOk && lCol.GetType().Collate == rCol.GetType().Collate { lID := s.getColID(lCol) rID := s.getColID(rCol) s.unionSet.Union(lID, rID) @@ -258,11 +251,11 @@ func (s *propConstSolver) propagateColumnEQ() { continue } cond := s.conditions[k] - replaced, _, newExpr := tryToReplaceCond(s.ctx, coli, colj, cond) + replaced, _, newExpr := tryToReplaceCond(s.ctx, coli, colj, cond, false) if replaced { s.conditions = append(s.conditions, newExpr) } - replaced, _, newExpr = tryToReplaceCond(s.ctx, colj, coli, cond) + replaced, _, newExpr = tryToReplaceCond(s.ctx, colj, coli, cond, false) if replaced { s.conditions = append(s.conditions, newExpr) } @@ -482,7 +475,7 @@ func (s *propOuterJoinConstSolver) validColEqualCond(cond Expression) (*Column, if fun, ok := cond.(*ScalarFunction); ok && fun.FuncName.L == ast.EQ { lCol, lOk := fun.GetArgs()[0].(*Column) rCol, rOk := fun.GetArgs()[1].(*Column) - if lOk && rOk && collate.CompatibleCollate(lCol.GetType().Collate, rCol.GetType().Collate) { + if lOk && rOk && lCol.GetType().Collate == rCol.GetType().Collate { return s.colsFromOuterAndInner(lCol, rCol) } } @@ -512,7 +505,7 @@ func (s *propOuterJoinConstSolver) deriveConds(outerCol, innerCol *Column, schem visited[k+offset] = true continue } - replaced, _, newExpr := tryToReplaceCond(s.ctx, outerCol, innerCol, cond) + replaced, _, newExpr := tryToReplaceCond(s.ctx, outerCol, innerCol, cond, true) if replaced { s.joinConds = append(s.joinConds, newExpr) } diff --git a/expression/constant_test.go b/expression/constant_test.go index e2d8404428088..e79ead10037e1 100644 --- a/expression/constant_test.go +++ b/expression/constant_test.go @@ -238,7 +238,7 @@ func (*testExpressionSuite) TestDeferredExprNullConstantFold(c *C) { nullConst := &Constant{ Value: types.NewDatum(nil), RetType: types.NewFieldType(mysql.TypeTiny), - DeferredExpr: Null, + DeferredExpr: NewNull(), } tests := []struct { condition Expression @@ -270,12 +270,12 @@ func (*testExpressionSuite) TestDeferredParamNotNull(c *C) { types.NewTimeDatum(types.NewTime(types.FromGoTime(testTime), mysql.TypeTimestamp, 6)), types.NewDurationDatum(types.ZeroDuration), types.NewStringDatum("{}"), - types.NewBinaryLiteralDatum(types.BinaryLiteral([]byte{1})), + types.NewBinaryLiteralDatum([]byte{1}), types.NewBytesDatum([]byte{'b'}), types.NewFloat32Datum(1.1), types.NewFloat64Datum(2.1), types.NewUintDatum(100), - types.NewMysqlBitDatum(types.BinaryLiteral([]byte{1})), + types.NewMysqlBitDatum([]byte{1}), types.NewMysqlEnumDatum(types.Enum{Name: "n", Value: 2}), } cstInt := &Constant{ParamMarker: &ParamMarker{ctx: ctx, order: 0}, RetType: newIntFieldType()} diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index 60da5cfd8dc14..202561c676614 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -43,7 +43,10 @@ func PbTypeToFieldType(tp *tipb.FieldType) *types.FieldType { func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *tipb.FieldType, args []Expression) (f builtinFunc, e error) { fieldTp := PbTypeToFieldType(tp) - base := newBaseBuiltinFunc(ctx, args) + base, err := newBaseBuiltinFunc(ctx, fmt.Sprintf("PBSig-%v", sigCode), args) + if err != nil { + return nil, err + } base.tp = fieldTp switch sigCode { case tipb.ScalarFuncSig_CastIntAsInt: @@ -456,6 +459,18 @@ func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *ti f = &builtinRealIsFalseSig{base, false} case tipb.ScalarFuncSig_DecimalIsFalse: f = &builtinDecimalIsFalseSig{base, false} + case tipb.ScalarFuncSig_IntIsTrueWithNull: + f = &builtinIntIsTrueSig{base, true} + case tipb.ScalarFuncSig_RealIsTrueWithNull: + f = &builtinRealIsTrueSig{base, true} + case tipb.ScalarFuncSig_DecimalIsTrueWithNull: + f = &builtinDecimalIsTrueSig{base, true} + case tipb.ScalarFuncSig_IntIsFalseWithNull: + f = &builtinIntIsFalseSig{base, true} + case tipb.ScalarFuncSig_RealIsFalseWithNull: + f = &builtinRealIsFalseSig{base, true} + case tipb.ScalarFuncSig_DecimalIsFalseWithNull: + f = &builtinDecimalIsFalseSig{base, true} case tipb.ScalarFuncSig_LeftShift: f = &builtinLeftShiftSig{base} case tipb.ScalarFuncSig_RightShift: diff --git a/expression/expr_to_pb.go b/expression/expr_to_pb.go index 8ea016c5dead9..fd0aef308ab84 100644 --- a/expression/expr_to_pb.go +++ b/expression/expr_to_pb.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -30,10 +31,14 @@ import ( ) // ExpressionsToPBList converts expressions to tipb.Expr list for new plan. -func ExpressionsToPBList(sc *stmtctx.StatementContext, exprs []Expression, client kv.Client) (pbExpr []*tipb.Expr) { +func ExpressionsToPBList(sc *stmtctx.StatementContext, exprs []Expression, client kv.Client) (pbExpr []*tipb.Expr, err error) { pc := PbConverter{client: client, sc: sc} for _, expr := range exprs { v := pc.ExprToPB(expr) + if v == nil { + return nil, terror.ClassOptimizer.New(mysql.ErrInternal, mysql.MySQLErrName[mysql.ErrInternal]). + GenWithStack("expression %v cannot be pushed down", expr) + } pbExpr = append(pbExpr, v) } return diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index 235543ae86ed6..ac026ce0dc7de 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -114,7 +114,8 @@ func (s *testEvaluatorSuite) TestConstant2Pb(c *C) { c.Assert(len(pushed), Equals, len(constExprs)-3) c.Assert(len(remained), Equals, 3) - pbExprs := ExpressionsToPBList(sc, constExprs, client) + pbExprs, err := ExpressionsToPBList(sc, constExprs, client) + c.Assert(err, IsNil) jsons := []string{ "{\"tp\":0,\"sig\":0}", "{\"tp\":1,\"val\":\"gAAAAAAAAGQ=\",\"sig\":0}", @@ -152,9 +153,9 @@ func (s *testEvaluatorSuite) TestColumn2Pb(c *C) { c.Assert(len(pushed), Equals, 0) c.Assert(len(remained), Equals, len(colExprs)) - pbExprs := ExpressionsToPBList(sc, colExprs, client) - for _, pbExpr := range pbExprs { - c.Assert(pbExpr, IsNil) + for _, col := range colExprs { // cannot be pushed down + _, err := ExpressionsToPBList(sc, []Expression{col}, client) + c.Assert(err, NotNil) } colExprs = colExprs[:0] @@ -184,7 +185,8 @@ func (s *testEvaluatorSuite) TestColumn2Pb(c *C) { c.Assert(len(pushed), Equals, len(colExprs)) c.Assert(len(remained), Equals, 0) - pbExprs = ExpressionsToPBList(sc, colExprs, client) + pbExprs, err := ExpressionsToPBList(sc, colExprs, client) + c.Assert(err, IsNil) jsons := []string{ "{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}", "{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":2,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}", @@ -243,7 +245,8 @@ func (s *testEvaluatorSuite) TestCompareFunc2Pb(c *C) { c.Assert(len(pushed), Equals, len(compareExprs)) c.Assert(len(remained), Equals, 0) - pbExprs := ExpressionsToPBList(sc, compareExprs, client) + pbExprs, err := ExpressionsToPBList(sc, compareExprs, client) + c.Assert(err, IsNil) c.Assert(len(pbExprs), Equals, len(compareExprs)) jsons := []string{ "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":100,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}", @@ -286,7 +289,8 @@ func (s *testEvaluatorSuite) TestLikeFunc2Pb(c *C) { c.Assert(err, IsNil) likeFuncs = append(likeFuncs, fc) - pbExprs := ExpressionsToPBList(sc, likeFuncs, client) + pbExprs, err := ExpressionsToPBList(sc, likeFuncs, client) + c.Assert(err, IsNil) results := []string{ `{"tp":10000,"children":[{"tp":5,"val":"c3RyaW5n","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":5,"val":"cGF0dGVybg==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":10000,"val":"CAA=","children":[{"tp":5,"val":"XA==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}}],"sig":30,"field_type":{"tp":8,"flag":128,"flen":-1,"decimal":0,"collate":63,"charset":"binary"}}],"sig":4310,"field_type":{"tp":8,"flag":128,"flen":1,"decimal":0,"collate":63,"charset":"binary"}}`, `{"tp":10000,"children":[{"tp":5,"val":"c3RyaW5n","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":5,"val":"JWFiYyU=","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":10000,"val":"CAA=","children":[{"tp":5,"val":"XA==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}}],"sig":30,"field_type":{"tp":8,"flag":128,"flen":-1,"decimal":0,"collate":63,"charset":"binary"}}],"sig":4310,"field_type":{"tp":8,"flag":128,"flen":1,"decimal":0,"collate":63,"charset":"binary"}}`, @@ -304,7 +308,7 @@ func (s *testEvaluatorSuite) TestArithmeticalFunc2Pb(c *C) { client := new(mock.Client) dg := new(dataGen4Expr2PbTest) - funcNames := []string{ast.Plus, ast.Minus, ast.Mul, ast.Div, ast.Mod, ast.IntDiv} + funcNames := []string{ast.Plus, ast.Minus, ast.Mul, ast.Div} for _, funcName := range funcNames { fc, err := NewFunction( mock.NewContext(), @@ -322,17 +326,26 @@ func (s *testEvaluatorSuite) TestArithmeticalFunc2Pb(c *C) { jsons[ast.Mul] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":208,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"}}" jsons[ast.Div] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":211,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":23,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"}}" - pbExprs := ExpressionsToPBList(sc, arithmeticalFuncs, client) + pbExprs, err := ExpressionsToPBList(sc, arithmeticalFuncs, client) + c.Assert(err, IsNil) for i, pbExpr := range pbExprs { - switch funcNames[i] { - case ast.Mod, ast.IntDiv: - c.Assert(pbExpr, IsNil, Commentf("%v\n", funcNames[i])) - default: - c.Assert(pbExpr, NotNil) - js, err := json.Marshal(pbExpr) - c.Assert(err, IsNil) - c.Assert(string(js), Equals, jsons[funcNames[i]], Commentf("%v\n", funcNames[i])) - } + c.Assert(pbExpr, NotNil) + js, err := json.Marshal(pbExpr) + c.Assert(err, IsNil) + c.Assert(string(js), Equals, jsons[funcNames[i]], Commentf("%v\n", funcNames[i])) + } + + funcNames = []string{ast.Mod, ast.IntDiv} // cannot be pushed down + for _, funcName := range funcNames { + fc, err := NewFunction( + mock.NewContext(), + funcName, + types.NewFieldType(mysql.TypeUnspecified), + dg.genColumn(mysql.TypeDouble, 1), + dg.genColumn(mysql.TypeDouble, 2)) + c.Assert(err, IsNil) + _, err = ExpressionsToPBList(sc, []Expression{fc}, client) + c.Assert(err, NotNil) } } @@ -348,7 +361,8 @@ func (s *testEvaluatorSuite) TestDateFunc2Pb(c *C) { dg.genColumn(mysql.TypeString, 2)) c.Assert(err, IsNil) funcs := []Expression{fc} - pbExprs := ExpressionsToPBList(sc, funcs, client) + pbExprs, err := ExpressionsToPBList(sc, funcs, client) + c.Assert(err, IsNil) c.Assert(pbExprs[0], NotNil) js, err := json.Marshal(pbExprs[0]) c.Assert(err, IsNil) @@ -377,7 +391,8 @@ func (s *testEvaluatorSuite) TestLogicalFunc2Pb(c *C) { logicalFuncs = append(logicalFuncs, fc) } - pbExprs := ExpressionsToPBList(sc, logicalFuncs, client) + pbExprs, err := ExpressionsToPBList(sc, logicalFuncs, client) + c.Assert(err, IsNil) jsons := []string{ "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":3101,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}", "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":3102,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}", @@ -413,7 +428,8 @@ func (s *testEvaluatorSuite) TestBitwiseFunc2Pb(c *C) { bitwiseFuncs = append(bitwiseFuncs, fc) } - pbExprs := ExpressionsToPBList(sc, bitwiseFuncs, client) + pbExprs, err := ExpressionsToPBList(sc, bitwiseFuncs, client) + c.Assert(err, IsNil) jsons := []string{ "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":3118,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}", "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":3119,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}", @@ -484,7 +500,8 @@ func (s *testEvaluatorSerialSuites) TestPushDownSwitcher(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/PushDownTestSwitcher", `return("all")`), IsNil) defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/expression/PushDownTestSwitcher"), IsNil) }() - pbExprs := ExpressionsToPBList(sc, funcs, client) + pbExprs, err := ExpressionsToPBList(sc, funcs, client) + c.Assert(err, IsNil) c.Assert(len(pbExprs), Equals, len(cases)) for i, pbExpr := range pbExprs { c.Assert(pbExpr.Sig, Equals, cases[i].sig, Commentf("function: %s, sig: %v", cases[i].name, cases[i].sig)) @@ -492,18 +509,17 @@ func (s *testEvaluatorSerialSuites) TestPushDownSwitcher(c *C) { // All disabled c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/PushDownTestSwitcher", `return("")`), IsNil) - pbExprs = ExpressionsToPBList(sc, funcs, client) - c.Assert(len(pbExprs), Equals, len(cases)) - for i, pbExpr := range pbExprs { + pc := PbConverter{client: client, sc: sc} + for i := range funcs { + pbExpr := pc.ExprToPB(funcs[i]) c.Assert(pbExpr, IsNil, Commentf("function: %s, sig: %v", cases[i].name, cases[i].sig)) } // Partial enabled fpexpr := fmt.Sprintf(`return("%s")`, strings.Join(enabled, ",")) c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/PushDownTestSwitcher", fpexpr), IsNil) - pbExprs = ExpressionsToPBList(sc, funcs, client) - c.Assert(len(pbExprs), Equals, len(cases)) - for i, pbExpr := range pbExprs { + for i := range funcs { + pbExpr := pc.ExprToPB(funcs[i]) if !cases[i].enable { c.Assert(pbExpr, IsNil, Commentf("function: %s, sig: %v", cases[i].name, cases[i].sig)) continue @@ -539,9 +555,10 @@ func (s *testEvaluatorSuite) TestControlFunc2Pb(c *C) { controlFuncs = append(controlFuncs, fc) } - pbExprs := ExpressionsToPBList(sc, controlFuncs, client) + pbExprs, err := ExpressionsToPBList(sc, controlFuncs, client) + c.Assert(err, IsNil) jsons := []string{ - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4208,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":0,\"decimal\":0,\"collate\":63,\"charset\":\"\"}}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4208,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":63,\"charset\":\"\"}}", "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4107,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}", "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4101,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}", "null", @@ -571,7 +588,8 @@ func (s *testEvaluatorSuite) TestOtherFunc2Pb(c *C) { otherFuncs = append(otherFuncs, fc) } - pbExprs := ExpressionsToPBList(sc, otherFuncs, client) + pbExprs, err := ExpressionsToPBList(sc, otherFuncs, client) + c.Assert(err, IsNil) jsons := map[string]string{ ast.Coalesce: "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4201,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":0,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}", ast.IsNull: "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":3116,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}", @@ -738,7 +756,8 @@ func (s *testEvaluatorSerialSuites) TestNewCollationsEnabled(c *C) { colExprs = append(colExprs, columnCollation(dg.genColumn(mysql.TypeVarchar, 5), "utf8_bin")) pushed, _ := PushDownExprs(sc, colExprs, client, kv.UnSpecified) c.Assert(len(pushed), Equals, len(colExprs)) - pbExprs := ExpressionsToPBList(sc, colExprs, client) + pbExprs, err := ExpressionsToPBList(sc, colExprs, client) + c.Assert(err, IsNil) jsons := []string{ "{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":15,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-46,\"charset\":\"\"}}", "{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":15,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-46,\"charset\":\"\"}}", @@ -773,7 +792,8 @@ func (s *testEvalSerialSuite) TestPushCollationDown(c *C) { tps := []*types.FieldType{types.NewFieldType(mysql.TypeVarchar), types.NewFieldType(mysql.TypeVarchar)} for _, coll := range []string{charset.CollationBin, charset.CollationLatin1, charset.CollationUTF8, charset.CollationUTF8MB4} { fc.SetCharsetAndCollation("binary", coll, types.UnspecifiedLength) // only collation matters - pbExpr := ExpressionsToPBList(sc, []Expression{fc}, client) + pbExpr, err := ExpressionsToPBList(sc, []Expression{fc}, client) + c.Assert(err, IsNil) expr, err := PBToExpr(pbExpr[0], tps, sc) c.Assert(err, IsNil) _, eColl, _ := expr.CharsetAndCollation(nil) diff --git a/expression/expression.go b/expression/expression.go index d8882f748fc4b..578a6caf8f700 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -131,9 +131,9 @@ type Expression interface { IsCorrelated() bool // ConstItem checks if this expression is constant item, regardless of query evaluation state. - // A constant item can be eval() when build a plan. // An expression is constant item if it: // refers no tables. + // refers no correlated column. // refers no subqueries that refers any tables. // refers no non-deterministic functions. // refers no statement parameters. @@ -324,7 +324,7 @@ func VecEvalBool(ctx sessionctx.Context, exprList CNFExprs, input *chunk.Chunk, return nil, nil, err } - if err := VecEval(ctx, expr, input, buf); err != nil { + if err := EvalExpr(ctx, expr, input, buf); err != nil { return nil, nil, err } @@ -389,7 +389,7 @@ func toBool(sc *stmtctx.StatementContext, eType types.EvalType, buf *chunk.Colum if buf.IsNull(i) { isZero[i] = -1 } else { - if types.RoundFloat(f64s[i]) == 0 { + if f64s[i] == 0 { isZero[i] = 0 } else { isZero[i] = 1 @@ -427,12 +427,9 @@ func toBool(sc *stmtctx.StatementContext, eType types.EvalType, buf *chunk.Colum if buf.IsNull(i) { isZero[i] = -1 } else { - iVal, err := types.StrToInt(sc, buf.GetString(i)) + iVal, err := types.StrToFloat(sc, buf.GetString(i)) if err != nil { - iVal, err = HandleOverflowOnSelection(sc, iVal, err) - if err != nil { - return err - } + return err } if iVal == 0 { isZero[i] = 0 @@ -447,11 +444,7 @@ func toBool(sc *stmtctx.StatementContext, eType types.EvalType, buf *chunk.Colum if buf.IsNull(i) { isZero[i] = -1 } else { - v, err := d64s[i].ToFloat64() - if err != nil { - return err - } - if types.RoundFloat(v) == 0 { + if d64s[i].IsZero() { isZero[i] = 0 } else { isZero[i] = 1 @@ -464,25 +457,138 @@ func toBool(sc *stmtctx.StatementContext, eType types.EvalType, buf *chunk.Colum return nil } -// VecEval evaluates this expr according to its type. -func VecEval(ctx sessionctx.Context, expr Expression, input *chunk.Chunk, result *chunk.Column) (err error) { - switch expr.GetType().EvalType() { - case types.ETInt: - err = expr.VecEvalInt(ctx, input, result) - case types.ETReal: - err = expr.VecEvalReal(ctx, input, result) - case types.ETDuration: - err = expr.VecEvalDuration(ctx, input, result) - case types.ETDatetime, types.ETTimestamp: - err = expr.VecEvalTime(ctx, input, result) - case types.ETString: - err = expr.VecEvalString(ctx, input, result) - case types.ETJson: - err = expr.VecEvalJSON(ctx, input, result) - case types.ETDecimal: - err = expr.VecEvalDecimal(ctx, input, result) - default: - err = errors.New(fmt.Sprintf("invalid eval type %v", expr.GetType().EvalType())) +// EvalExpr evaluates this expr according to its type. +// And it selects the method for evaluating expression based on +// the environment variables and whether the expression can be vectorized. +func EvalExpr(ctx sessionctx.Context, expr Expression, input *chunk.Chunk, result *chunk.Column) (err error) { + evalType := expr.GetType().EvalType() + if expr.Vectorized() && ctx.GetSessionVars().EnableVectorizedExpression { + switch evalType { + case types.ETInt: + err = expr.VecEvalInt(ctx, input, result) + case types.ETReal: + err = expr.VecEvalReal(ctx, input, result) + case types.ETDuration: + err = expr.VecEvalDuration(ctx, input, result) + case types.ETDatetime, types.ETTimestamp: + err = expr.VecEvalTime(ctx, input, result) + case types.ETString: + err = expr.VecEvalString(ctx, input, result) + case types.ETJson: + err = expr.VecEvalJSON(ctx, input, result) + case types.ETDecimal: + err = expr.VecEvalDecimal(ctx, input, result) + default: + err = errors.New(fmt.Sprintf("invalid eval type %v", expr.GetType().EvalType())) + } + } else { + ind, n := 0, input.NumRows() + iter := chunk.NewIterator4Chunk(input) + switch evalType { + case types.ETInt: + result.ResizeInt64(n, false) + i64s := result.Int64s() + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalInt(ctx, it) + if err != nil { + return err + } + if isNull { + result.SetNull(ind, isNull) + } else { + i64s[ind] = value + } + ind++ + } + case types.ETReal: + result.ResizeFloat64(n, false) + f64s := result.Float64s() + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalReal(ctx, it) + if err != nil { + return err + } + if isNull { + result.SetNull(ind, isNull) + } else { + f64s[ind] = value + } + ind++ + } + case types.ETDuration: + result.ResizeGoDuration(n, false) + d64s := result.GoDurations() + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalDuration(ctx, it) + if err != nil { + return err + } + if isNull { + result.SetNull(ind, isNull) + } else { + d64s[ind] = value.Duration + } + ind++ + } + case types.ETDatetime, types.ETTimestamp: + result.ResizeTime(n, false) + t64s := result.Times() + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalTime(ctx, it) + if err != nil { + return err + } + if isNull { + result.SetNull(ind, isNull) + } else { + t64s[ind] = value + } + ind++ + } + case types.ETString: + result.ReserveString(n) + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalString(ctx, it) + if err != nil { + return err + } + if isNull { + result.AppendNull() + } else { + result.AppendString(value) + } + } + case types.ETJson: + result.ReserveJSON(n) + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalJSON(ctx, it) + if err != nil { + return err + } + if isNull { + result.AppendNull() + } else { + result.AppendJSON(value) + } + } + case types.ETDecimal: + result.ResizeDecimal(n, false) + d64s := result.Decimals() + for it := iter.Begin(); it != iter.End(); it = iter.Next() { + value, isNull, err := expr.EvalDecimal(ctx, it) + if err != nil { + return err + } + if isNull { + result.SetNull(ind, isNull) + } else { + d64s[ind] = *value + } + ind++ + } + default: + err = errors.New(fmt.Sprintf("invalid eval type %v", expr.GetType().EvalType())) + } } return } @@ -833,6 +939,8 @@ func canFuncBePushed(sf *ScalarFunction, storeType kv.StoreType) bool { ast.PeriodAdd, ast.PeriodDiff, ast.TimestampDiff, + ast.DateAdd, + ast.FromUnixTime, // encryption functions. ast.MD5, @@ -850,7 +958,7 @@ func canFuncBePushed(sf *ScalarFunction, storeType kv.StoreType) bool { ast.IsIPv4Compat, ast.IsIPv4Mapped, ast.IsIPv6: - ret = isPushDownEnabled(sf.FuncName.L) + ret = true // A special case: Only push down Round by signature case ast.Round: @@ -859,7 +967,7 @@ func canFuncBePushed(sf *ScalarFunction, storeType kv.StoreType) bool { tipb.ScalarFuncSig_RoundReal, tipb.ScalarFuncSig_RoundInt, tipb.ScalarFuncSig_RoundDec: - ret = isPushDownEnabled(sf.FuncName.L) + ret = true } case ast.Substring, @@ -868,29 +976,46 @@ func canFuncBePushed(sf *ScalarFunction, storeType kv.StoreType) bool { case tipb.ScalarFuncSig_Substring2ArgsUTF8, tipb.ScalarFuncSig_Substring3ArgsUTF8: - ret = isPushDownEnabled(sf.FuncName.L) + ret = true } case ast.Rand: switch sf.Function.PbCode() { case tipb.ScalarFuncSig_RandWithSeedFirstGen: - ret = isPushDownEnabled(sf.FuncName.L) + ret = true } } if ret { switch storeType { case kv.TiFlash: - return scalarExprSupportedByFlash(sf) + ret = scalarExprSupportedByFlash(sf) case kv.TiKV: - return scalarExprSupportedByTiKV(sf) + ret = scalarExprSupportedByTiKV(sf) + case kv.TiDB: + ret = scalarExprSupportedByTiDB(sf) } } + if ret { + ret = IsPushDownEnabled(sf.FuncName.L, storeType) + } return ret } -func isPushDownEnabled(name string) bool { - _, disallowPushDown := DefaultExprPushDownBlacklist.Load().(map[string]struct{})[name] - return !disallowPushDown +func storeTypeMask(storeType kv.StoreType) uint32 { + if storeType == kv.UnSpecified { + return 1< 9223372036854775807-0.5 order by a`).Check(testkit.Rows(`9223372036854775807`, `18446744073709551615`)) } +func (s *testIntegrationSuite) TestExprPushdown(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int, col1 varchar(10), col2 varchar(10), col3 int, col4 int, col5 int, index key1" + + " (col1, col2, col3, col4), index key2 (col4, col3, col2, col1))") + tk.MustExec("insert into t values(1,'211111','311',4,5,6),(2,'311111','411',5,6,7),(3,'411111','511',6,7,8)," + + "(4,'511111','611',7,8,9),(5,'611111','711',8,9,10)") + + // case 1, index scan without double read, some filters can not be pushed to cop task + rows := tk.MustQuery("explain select col2, col1 from t use index(key1) where col2 like '5%' and substr(col1, 1, 1) = '4'").Rows() + c.Assert(fmt.Sprintf("%v", rows[1][2]), Equals, "root") + c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "eq(substr(test.t.col1, 1, 1), \"4\")") + c.Assert(fmt.Sprintf("%v", rows[3][2]), Equals, "cop[tikv]") + c.Assert(fmt.Sprintf("%v", rows[3][4]), Equals, "like(test.t.col2, \"5%\", 92)") + tk.MustQuery("select col2, col1 from t use index(key1) where col2 like '5%' and substr(col1, 1, 1) = '4'").Check(testkit.Rows("511 411111")) + tk.MustQuery("select count(col2) from t use index(key1) where col2 like '5%' and substr(col1, 1, 1) = '4'").Check(testkit.Rows("1")) + + // case 2, index scan without double read, none of the filters can be pushed to cop task + rows = tk.MustQuery("explain select col1, col2 from t use index(key2) where substr(col2, 1, 1) = '5' and substr(col1, 1, 1) = '4'").Rows() + c.Assert(fmt.Sprintf("%v", rows[0][2]), Equals, "root") + c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "eq(substr(test.t.col1, 1, 1), \"4\"), eq(substr(test.t.col2, 1, 1), \"5\")") + tk.MustQuery("select col1, col2 from t use index(key2) where substr(col2, 1, 1) = '5' and substr(col1, 1, 1) = '4'").Check(testkit.Rows("411111 511")) + tk.MustQuery("select count(col1) from t use index(key2) where substr(col2, 1, 1) = '5' and substr(col1, 1, 1) = '4'").Check(testkit.Rows("1")) + + // case 3, index scan with double read, some filters can not be pushed to cop task + rows = tk.MustQuery("explain select id from t use index(key1) where col2 like '5%' and substr(col1, 1, 1) = '4'").Rows() + c.Assert(fmt.Sprintf("%v", rows[1][2]), Equals, "root") + c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "eq(substr(test.t.col1, 1, 1), \"4\")") + c.Assert(fmt.Sprintf("%v", rows[3][2]), Equals, "cop[tikv]") + c.Assert(fmt.Sprintf("%v", rows[3][4]), Equals, "like(test.t.col2, \"5%\", 92)") + tk.MustQuery("select id from t use index(key1) where col2 like '5%' and substr(col1, 1, 1) = '4'").Check(testkit.Rows("3")) + tk.MustQuery("select count(id) from t use index(key1) where col2 like '5%' and substr(col1, 1, 1) = '4'").Check(testkit.Rows("1")) + + // case 4, index scan with double read, none of the filters can be pushed to cop task + rows = tk.MustQuery("explain select id from t use index(key2) where substr(col2, 1, 1) = '5' and substr(col1, 1, 1) = '4'").Rows() + c.Assert(fmt.Sprintf("%v", rows[1][2]), Equals, "root") + c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "eq(substr(test.t.col1, 1, 1), \"4\"), eq(substr(test.t.col2, 1, 1), \"5\")") + tk.MustQuery("select id from t use index(key2) where substr(col2, 1, 1) = '5' and substr(col1, 1, 1) = '4'").Check(testkit.Rows("3")) + tk.MustQuery("select count(id) from t use index(key2) where substr(col2, 1, 1) = '5' and substr(col1, 1, 1) = '4'").Check(testkit.Rows("1")) +} + func (s *testIntegrationSuite) TestExprPushdownBlacklist(c *C) { tk := testkit.NewTestKit(c, s.store) - tk.MustQuery(`select * from mysql.expr_pushdown_blacklist`).Check(testkit.Rows()) + tk.MustQuery(`select * from mysql.expr_pushdown_blacklist`).Check(testkit.Rows( + "date_add tiflash DST(daylight saving time) does not take effect in TiFlash date_add", + "cast tiflash Behavior of some corner cases(overflow, truncate etc) is different in TiFlash and TiDB")) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int , b date)") + + // 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.MustExec("insert into mysql.expr_pushdown_blacklist " + + "values('<', 'tikv,tiflash,tidb', 'for test'),('date_format', 'tikv', 'for test')") + tk.MustExec("admin reload expr_pushdown_blacklist") + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + + // < not pushed, cast only pushed to TiKV, date_format only pushed to TiFlash, + // > pushed to both TiKV and TiFlash + rows := tk.MustQuery("explain select * from test.t where b > date'1988-01-01' and b < date'1994-01-01' " + + "and cast(a as decimal(10,2)) > 10.10 and date_format(b,'%m') = '11'").Rows() + c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "lt(test.t.b, 1994-01-01)") + c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "gt(cast(test.t.a), 10.10)") + c.Assert(fmt.Sprintf("%v", rows[3][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\"), gt(test.t.b, 1988-01-01)") + + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tikv'") + rows = tk.MustQuery("explain select * from test.t where b > date'1988-01-01' and b < date'1994-01-01' " + + "and cast(a as decimal(10,2)) > 10.10 and date_format(b,'%m') = '11'").Rows() + c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "lt(test.t.b, 1994-01-01)") + c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\")") + c.Assert(fmt.Sprintf("%v", rows[3][4]), Equals, "gt(cast(test.t.a), 10.10), gt(test.t.b, 1988-01-01)") + + tk.MustExec("delete from mysql.expr_pushdown_blacklist where name = '<' and store_type = 'tikv,tiflash,tidb' and reason = 'for test'") + tk.MustExec("delete from mysql.expr_pushdown_blacklist where name = 'date_format' and store_type = 'tikv' and reason = 'for test'") + tk.MustExec("admin reload expr_pushdown_blacklist") } func (s *testIntegrationSuite) TestOptRuleBlacklist(c *C) { @@ -5349,6 +5443,55 @@ func (s *testIntegrationSuite) TestCastStrToInt(c *C) { } } +func (s *testIntegrationSerialSuite) TestIssue16205(c *C) { + tk := testkit.NewTestKit(c, s.store) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + var err error + tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + c.Assert(err, IsNil) + + tk.MustExec("use test") + tk.MustExec("prepare stmt from 'select random_bytes(3)'") + rows1 := tk.MustQuery("execute stmt").Rows() + c.Assert(len(rows1), Equals, 1) + rows2 := tk.MustQuery("execute stmt").Rows() + c.Assert(len(rows2), Equals, 1) + c.Assert(rows1[0][0].(string), Not(Equals), rows2[0][0].(string)) +} + +func (s *testIntegrationSerialSuite) TestRowCountPlanCache(c *C) { + tk := testkit.NewTestKit(c, s.store) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + var err error + tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + c.Assert(err, IsNil) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int auto_increment primary key)") + tk.MustExec("prepare stmt from 'select row_count()';") + tk.MustExec("insert into t values()") + res := tk.MustQuery("execute stmt").Rows() + c.Assert(len(res), Equals, 1) + c.Assert(res[0][0], Equals, "1") + tk.MustExec("insert into t values(),(),()") + res = tk.MustQuery("execute stmt").Rows() + c.Assert(len(res), Equals, 1) + c.Assert(res[0][0], Equals, "3") +} + func (s *testIntegrationSuite) TestValuesForBinaryLiteral(c *C) { // See issue #15310 tk := testkit.NewTestKit(c, s.store) @@ -5381,7 +5524,7 @@ func (s *testIntegrationSuite) TestIssue14146(c *C) { tk.MustQuery("select * from tt").Check(testkit.Rows("")) } -func (s *testIntegrationSuite) TestCacheRegexpr(c *C) { +func (s *testIntegrationSerialSuite) TestCacheRegexpr(c *C) { tk := testkit.NewTestKit(c, s.store) orgEnable := plannercore.PreparedPlanCacheEnabled() defer func() { @@ -5405,7 +5548,7 @@ func (s *testIntegrationSuite) TestCacheRegexpr(c *C) { tk.MustQuery("execute stmt1 using @a").Check(testkit.Rows("R1")) } -func (s *testIntegrationSuite) TestCacheRefineArgs(c *C) { +func (s *testIntegrationSerialSuite) TestCacheRefineArgs(c *C) { tk := testkit.NewTestKit(c, s.store) orgEnable := plannercore.PreparedPlanCacheEnabled() defer func() { @@ -5435,6 +5578,28 @@ func (s *testIntegrationSuite) TestCacheRefineArgs(c *C) { tk.MustQuery("execute stmt using @p0").Check(testkit.Rows("0")) } +func (s *testIntegrationSuite) TestOrderByFuncPlanCache(c *C) { + tk := testkit.NewTestKit(c, s.store) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + var err error + tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + c.Assert(err, IsNil) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("prepare stmt from 'SELECT * FROM t order by rand()'") + tk.MustQuery("execute stmt").Check(testkit.Rows()) + tk.MustExec("prepare stmt from 'SELECT * FROM t order by now()'") + tk.MustQuery("execute stmt").Check(testkit.Rows()) +} + func (s *testIntegrationSuite) TestCollation(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -5528,7 +5693,7 @@ func (s *testIntegrationSuite) TestCoercibility(c *C) { }, "from t") } -func (s *testIntegrationSuite) TestCacheConstEval(c *C) { +func (s *testIntegrationSerialSuite) TestCacheConstEval(c *C) { tk := testkit.NewTestKit(c, s.store) orgEnable := plannercore.PreparedPlanCacheEnabled() defer func() { @@ -5546,7 +5711,7 @@ func (s *testIntegrationSuite) TestCacheConstEval(c *C) { tk.MustExec("create table t(col_double double)") tk.MustExec("insert into t values (1)") tk.Se.GetSessionVars().EnableVectorizedExpression = false - tk.MustExec("insert into mysql.expr_pushdown_blacklist values('cast')") + tk.MustExec("insert into mysql.expr_pushdown_blacklist values('cast', 'tikv,tiflash,tidb', 'for test')") tk.MustExec("admin reload expr_pushdown_blacklist") tk.MustExec("prepare stmt from 'SELECT * FROM (SELECT col_double AS c0 FROM t) t WHERE (ABS((REPEAT(?, ?) OR 5617780767323292672)) < LN(EXP(c0)) + (? ^ ?))'") tk.MustExec("set @a1 = 'JuvkBX7ykVux20zQlkwDK2DFelgn7'") @@ -5557,7 +5722,7 @@ func (s *testIntegrationSuite) TestCacheConstEval(c *C) { // incompatible with MySQL actually, update the result after fixing it. tk.MustQuery("execute stmt using @a1, @a2, @a3, @a4").Check(testkit.Rows("1")) tk.Se.GetSessionVars().EnableVectorizedExpression = true - tk.MustExec("delete from mysql.expr_pushdown_blacklist") + tk.MustExec("delete from mysql.expr_pushdown_blacklist where name = 'cast' and store_type = 'tikv,tiflash,tidb' and reason = 'for test'") tk.MustExec("admin reload expr_pushdown_blacklist") } @@ -5680,6 +5845,31 @@ func (s *testIntegrationSerialSuite) TestCollateConstantPropagation(c *C) { tk.MustExec("insert into t values ('A', 'a');") tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='a' collate utf8mb4_bin;").Check(testkit.Rows("A a A a")) tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b>='a' collate utf8mb4_bin;").Check(testkit.Rows("A a A a")) + tk.MustExec("drop table t;") + tk.MustExec("set names utf8mb4") + tk.MustExec("create table t (a char(10) collate utf8mb4_general_ci, b char(10) collate utf8_general_ci);") + tk.MustExec("insert into t values ('a', 'A');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b='A'").Check(testkit.Rows("a A a A")) + tk.MustExec("drop table t;") + tk.MustExec("create table t(a char collate utf8_general_ci, b char collate utf8mb4_general_ci, c char collate utf8_bin);") + tk.MustExec("insert into t values ('b', 'B', 'B');") + tk.MustQuery("select * from t t1, t t2 where t1.a=t2.b and t2.b=t2.c;").Check(testkit.Rows("b B B b B B")) + tk.MustExec("drop table t;") + tk.MustExec("create table t(a char collate utf8_bin, b char collate utf8_general_ci);") + tk.MustExec("insert into t values ('a', 'A');") + tk.MustQuery("select * from t t1, t t2 where t1.b=t2.b and t2.b=t1.a collate utf8_general_ci;").Check(testkit.Rows("a A a A")) + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t1(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t2(a char, b varchar(10)) charset utf8mb4 collate utf8mb4_bin;") + tk.MustExec("insert into t1 values ('A', 'a');") + tk.MustExec("insert into t2 values ('a', 'a')") + tk.MustQuery("select * from t1 left join t2 on t1.a = t2.a where t1.a = 'a';").Check(testkit.Rows("A a ")) + tk.MustExec("drop table t;") + tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") + tk.MustExec("create table t(a char collate utf8mb4_bin, b char collate utf8mb4_general_ci);") + tk.MustExec("insert into t values ('a', 'a');") + tk.MustQuery("select * from t t1, t t2 where t2.b = 'A' and lower(concat(t1.a , '' )) = t2.b;").Check(testkit.Rows("a a a a")) } func (s *testIntegrationSerialSuite) prepare4Join(c *C) *testkit.TestKit { tk := testkit.NewTestKit(c, s.store) @@ -5902,6 +6092,13 @@ func (s *testIntegrationSerialSuite) TestCollateStringFunction(c *C) { tk.MustQuery("select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_bin);").Check(testkit.Rows("2")) tk.MustQuery("select FIND_IN_SET('a','b,A,c,d' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) tk.MustQuery("select FIND_IN_SET('a','b,a ,c,d' collate utf8mb4_general_ci);").Check(testkit.Rows("2")) + + tk.MustExec("select concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_bin);") + tk.MustGetErrMsg("select concat('a' collate utf8mb4_bin, 'b' collate utf8mb4_general_ci);", "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation 'concat'") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char)") + tk.MustGetErrMsg("select * from t t1 join t t2 on t1.a collate utf8mb4_bin = t2.a collate utf8mb4_general_ci;", "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation 'eq'") } func (s *testIntegrationSerialSuite) TestCollateLike(c *C) { @@ -5960,9 +6157,41 @@ func (s *testIntegrationSerialSuite) TestCollateDDL(c *C) { tk.MustExec("drop database t;") } +func (s *testIntegrationSuite) TestIssue15986(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t0") + tk.MustExec("CREATE TABLE t0(c0 int)") + tk.MustExec("INSERT INTO t0 VALUES (0)") + tk.MustQuery("SELECT t0.c0 FROM t0 WHERE CHAR(204355900);").Check(testkit.Rows("0")) + tk.MustQuery("SELECT t0.c0 FROM t0 WHERE not CHAR(204355900);").Check(testkit.Rows()) + tk.MustQuery("SELECT t0.c0 FROM t0 WHERE '.0';").Check(testkit.Rows()) + tk.MustQuery("SELECT t0.c0 FROM t0 WHERE not '.0';").Check(testkit.Rows("0")) + // If the number does not exceed the range of float64 and its value is not 0, it will be converted to true. + tk.MustQuery("select * from t0 where '.000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "0000000000000000000000000000000000000000000000000000000000000000009';").Check(testkit.Rows("0")) + tk.MustQuery("select * from t0 where not '.000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "0000000000000000000000000000000000000000000000000000000000000000009';").Check(testkit.Rows()) + + // If the number is truncated beyond the range of float64, it will be converted to true when the truncated result is 0. + tk.MustQuery("select * from t0 where '.0000000000000000000000000000000000000000000000000000000" + + "000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000009';").Check(testkit.Rows()) + tk.MustQuery("select * from t0 where not '.0000000000000000000000000000000000000000000000000000000" + + "000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000" + + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000009';").Check(testkit.Rows("0")) +} + func (s *testIntegrationSuite) TestNegativeZeroForHashJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") + tk.MustExec("drop table if exists t0, t1") tk.MustExec("CREATE TABLE t0(c0 float);") tk.MustExec("CREATE TABLE t1(c0 float);") tk.MustExec("INSERT INTO t1(c0) VALUES (0);") @@ -5972,12 +6201,112 @@ func (s *testIntegrationSuite) TestNegativeZeroForHashJoin(c *C) { tk.MustExec("drop table t1;") } +func (s *testIntegrationSuite) TestIssue15743(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t0") + tk.MustExec("CREATE TABLE t0(c0 int)") + tk.MustExec("INSERT INTO t0 VALUES (1)") + tk.MustQuery("SELECT * FROM t0 WHERE 1 AND 0.4").Check(testkit.Rows("1")) +} + +func (s *testIntegrationSuite) TestIssue15725(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)") + tk.MustExec("insert into t values(2)") + tk.MustQuery("select * from t where (not not a) = a").Check(testkit.Rows()) + tk.MustQuery("select * from t where (not not not not a) = a").Check(testkit.Rows()) +} + func (s *testIntegrationSuite) TestIssue15790(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") + tk.MustExec("drop table if exists t0") tk.MustExec("CREATE TABLE t0(c0 INT);") tk.MustExec("INSERT INTO t0(c0) VALUES (0);") tk.MustQuery("SELECT * FROM t0 WHERE -10000000000000000000 | t0.c0 UNION SELECT * FROM t0;").Check(testkit.Rows("0")) tk.MustQuery("SELECT * FROM t0 WHERE -10000000000000000000 | t0.c0 UNION all SELECT * FROM t0;").Check(testkit.Rows("0", "0")) tk.MustExec("drop table t0;") } + +func (s *testIntegrationSuite) TestIssue15990(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0;") + tk.MustExec("CREATE TABLE t0(c0 TEXT(10));") + tk.MustExec("INSERT INTO t0(c0) VALUES (1);") + tk.MustQuery("SELECT * FROM t0 WHERE ('a' != t0.c0) AND t0.c0;").Check(testkit.Rows("1")) + tk.MustExec("CREATE INDEX i0 ON t0(c0(10));") + tk.MustQuery("SELECT * FROM t0 WHERE ('a' != t0.c0) AND t0.c0;").Check(testkit.Rows("1")) + tk.MustExec("drop table t0;") +} + +func (s *testIntegrationSuite) TestIssue15992(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0") + tk.MustExec("CREATE TABLE t0(c0 INT, c1 INT AS (c0));") + tk.MustExec("CREATE INDEX i0 ON t0(c1);") + tk.MustQuery("SELECT t0.c0 FROM t0 UNION ALL SELECT 0 FROM t0;").Check(testkit.Rows()) + tk.MustExec("drop table t0;") +} + +func (s *testIntegrationSuite) TestIssue16419(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0") + tk.MustExec("drop table if exists t1") + tk.MustExec("CREATE TABLE t0(c0 INT);") + tk.MustExec("CREATE TABLE t1(c0 INT);") + tk.MustQuery("SELECT * FROM t1 NATURAL LEFT JOIN t0 WHERE NOT t1.c0;").Check(testkit.Rows()) + tk.MustExec("drop table t0, t1;") +} + +func (s *testIntegrationSuite) TestIssue16029(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t0,t1;") + tk.MustExec("CREATE TABLE t0(c0 INT);") + tk.MustExec("CREATE TABLE t1(c0 INT);") + tk.MustExec("INSERT INTO t0 VALUES (NULL), (1);") + tk.MustExec("INSERT INTO t1 VALUES (0);") + tk.MustQuery("SELECT t0.c0 FROM t0 JOIN t1 ON (t0.c0 REGEXP 1) | t1.c0 WHERE BINARY STRCMP(t1.c0, t0.c0);").Check(testkit.Rows("1")) + tk.MustExec("drop table t0;") + tk.MustExec("drop table t1;") +} + +func (s *testIntegrationSuite) TestIssue16426(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)") + tk.MustExec("insert into t values (42)") + tk.MustQuery("select a from t where a/10000").Check(testkit.Rows("42")) + tk.MustQuery("select a from t where a/100000").Check(testkit.Rows("42")) + tk.MustQuery("select a from t where a/1000000").Check(testkit.Rows("42")) + tk.MustQuery("select a from t where a/10000000").Check(testkit.Rows("42")) +} + +func (s *testIntegrationSuite) TestIssue16505(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + tk.MustExec("CREATE TABLE t(c varchar(100), index idx(c(100)));") + tk.MustExec("INSERT INTO t VALUES (NULL),('1'),('0'),(''),('aaabbb'),('0abc'),('123e456'),('0.0001deadsfeww');") + tk.MustQuery("select * from t where c;").Sort().Check(testkit.Rows("0.0001deadsfeww", "1", "123e456")) + tk.MustQuery("select /*+ USE_INDEX(t, idx) */ * from t where c;").Sort().Check(testkit.Rows("0.0001deadsfeww", "1", "123e456")) + tk.MustQuery("select /*+ IGNORE_INDEX(t, idx) */* from t where c;").Sort().Check(testkit.Rows("0.0001deadsfeww", "1", "123e456")) + tk.MustExec("drop table t;") +} + +func (s *testIntegrationSuite) TestIssue16779(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t0") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t0 (c0 int)") + tk.MustExec("create table t1 (c0 int)") + tk.MustQuery("SELECT * FROM t1 LEFT JOIN t0 ON TRUE WHERE BINARY EXPORT_SET(0, 0, 0 COLLATE 'binary', t0.c0, 0 COLLATE 'binary')") +} diff --git a/expression/scalar_function_test.go b/expression/scalar_function_test.go index 9f5b0626e8f02..f3349a87c34fb 100755 --- a/expression/scalar_function_test.go +++ b/expression/scalar_function_test.go @@ -29,7 +29,7 @@ func (s *testEvaluatorSuite) TestScalarFunction(c *C) { RetType: types.NewFieldType(mysql.TypeDouble), } sc := &stmtctx.StatementContext{TimeZone: time.Local} - sf := newFunction(ast.LT, a, One) + sf := newFunction(ast.LT, a, NewOne()) res, err := sf.MarshalJSON() c.Assert(err, IsNil) c.Assert(res, DeepEquals, []byte{0x22, 0x6c, 0x74, 0x28, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x23, 0x31, 0x2c, 0x20, 0x31, 0x29, 0x22}) @@ -52,8 +52,8 @@ func (s *testEvaluatorSuite) TestScalarFuncs2Exprs(c *C) { UniqueID: 1, RetType: types.NewFieldType(mysql.TypeDouble), } - sf0, _ := newFunction(ast.LT, a, Zero).(*ScalarFunction) - sf1, _ := newFunction(ast.LT, a, One).(*ScalarFunction) + sf0, _ := newFunction(ast.LT, a, NewZero()).(*ScalarFunction) + sf1, _ := newFunction(ast.LT, a, NewOne()).(*ScalarFunction) funcs := []*ScalarFunction{sf0, sf1} exprs := ScalarFuncs2Exprs(funcs) diff --git a/expression/simple_rewriter.go b/expression/simple_rewriter.go index f77d86ec64d9a..538c63b6d6444 100644 --- a/expression/simple_rewriter.go +++ b/expression/simple_rewriter.go @@ -420,8 +420,8 @@ func (sr *simpleRewriter) constructBinaryOpFunction(l Expression, r Expression, var expr1, expr2, expr3 Expression if op == ast.LE || op == ast.GE { expr1 = NewFunctionInternal(sr.ctx, op, types.NewFieldType(mysql.TypeTiny), larg0, rarg0) - expr1 = NewFunctionInternal(sr.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), expr1, Zero) - expr2 = Zero + expr1 = NewFunctionInternal(sr.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), expr1, NewZero()) + expr2 = NewZero() } else if op == ast.LT || op == ast.GT { expr1 = NewFunctionInternal(sr.ctx, ast.NE, types.NewFieldType(mysql.TypeTiny), larg0, rarg0) expr2 = NewFunctionInternal(sr.ctx, op, types.NewFieldType(mysql.TypeTiny), larg0, rarg0) @@ -595,7 +595,7 @@ func (sr *simpleRewriter) inToExpression(lLen int, not bool, tp *types.FieldType } leftIsNull := leftFt.Tp == mysql.TypeNull if leftIsNull { - sr.push(Null.Clone()) + sr.push(NewNull()) return } leftEt := leftFt.EvalType() diff --git a/expression/testdata/expression_suite_in.json b/expression/testdata/expression_suite_in.json index 9b94b9cc3c401..e67f1efc580fc 100644 --- a/expression/testdata/expression_suite_in.json +++ b/expression/testdata/expression_suite_in.json @@ -33,7 +33,9 @@ "explain select * from t1 left join t2 on t1.a = 1 or (t1.a = 2 and t1.a = 3)", "explain select * from t1 left join t2 on true where t1.a = 1 or (t1.a = 2 and t1.a = 3)", // Constant propagation over left outer semi join, filter with aux column should not be derived. - "explain select * from t1 where t1.b > 1 or t1.b in (select b from t2)" + "explain select * from t1 where t1.b > 1 or t1.b in (select b from t2)", + // Don't propagate for the control function. + "explain select * from t1 left join t2 on t1.a = t2.a where ifnull(t2.b, t1.a) = 1" ] } ] diff --git a/expression/testdata/expression_suite_out.json b/expression/testdata/expression_suite_out.json index 1d0e6fb636ffc..ddbd8c6678d88 100644 --- a/expression/testdata/expression_suite_out.json +++ b/expression/testdata/expression_suite_out.json @@ -274,6 +274,18 @@ " └─TableReader_11(Probe) 10000.00 root data:TableFullScan_10", " └─TableFullScan_10 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ] + }, + { + "SQL": "explain select * from t1 left join t2 on t1.a = t2.a where ifnull(t2.b, t1.a) = 1", + "Result": [ + "Selection_7 9990.00 root eq(ifnull(test.t2.b, test.t1.a), 1)", + "└─HashJoin_8 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─TableReader_14(Build) 9990.00 root data:Selection_13", + " │ └─Selection_13 9990.00 cop[tikv] not(isnull(test.t2.a))", + " │ └─TableFullScan_12 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader_11(Probe) 10000.00 root data:TableFullScan_10", + " └─TableFullScan_10 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ] } ] } diff --git a/expression/testdata/partition_pruner_out.json b/expression/testdata/partition_pruner_out.json index fb96bb79b086b..e3dcc64190589 100644 --- a/expression/testdata/partition_pruner_out.json +++ b/expression/testdata/partition_pruner_out.json @@ -94,4 +94,3 @@ ] } ] - diff --git a/expression/util.go b/expression/util.go index fff42ed3d1db1..2edf6528dc943 100644 --- a/expression/util.go +++ b/expression/util.go @@ -27,8 +27,9 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - driver "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" "golang.org/x/tools/container/intsets" @@ -215,6 +216,7 @@ func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression if v.InOperand { newExpr = setExprColumnInOperand(newExpr) } + newExpr.SetCoercibility(v.Coercibility()) return true, newExpr case *ScalarFunction: if v.FuncName.L == ast.Cast { @@ -226,8 +228,25 @@ func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression // when expr in args is changed refExprArr := cowExprRef{v.GetArgs(), nil} substituted := false + _, coll, _ := DeriveCollationFromExprs(v.GetCtx(), v.GetArgs()...) for idx, arg := range v.GetArgs() { changed, newFuncExpr := ColumnSubstituteImpl(arg, schema, newExprs) + if collate.NewCollationEnabled() { + // Make sure the collation used by the ScalarFunction isn't changed and its result collation is not weaker than the collation used by the ScalarFunction. + if changed { + changed = false + tmpArgs := make([]Expression, 0, len(v.GetArgs())) + _ = append(append(append(tmpArgs, refExprArr.Result()[0:idx]...), refExprArr.Result()[idx+1:]...), newFuncExpr) + _, newColl, _ := DeriveCollationFromExprs(v.GetCtx(), append(v.GetArgs(), newFuncExpr)...) + if coll == newColl { + collStrictness, ok1 := CollationStrictness[coll] + newResStrictness, ok2 := CollationStrictness[newFuncExpr.GetType().Collate] + if ok1 && ok2 && newResStrictness >= collStrictness { + changed = true + } + } + } + } refExprArr.Set(idx, changed, newFuncExpr) if changed { substituted = true @@ -336,6 +355,24 @@ func timeZone2Duration(tz string) time.Duration { return time.Duration(sign) * (time.Duration(h)*time.Hour + time.Duration(m)*time.Minute) } +var logicalOps = map[string]struct{}{ + ast.LT: {}, + ast.GE: {}, + ast.GT: {}, + ast.LE: {}, + ast.EQ: {}, + ast.NE: {}, + ast.UnaryNot: {}, + ast.LogicAnd: {}, + ast.LogicOr: {}, + ast.LogicXor: {}, + ast.In: {}, + ast.IsNull: {}, + ast.IsTruth: {}, + ast.IsFalsity: {}, + ast.Like: {}, +} + var oppositeOp = map[string]string{ ast.LT: ast.GE, ast.GE: ast.LT, @@ -369,12 +406,24 @@ func pushNotAcrossArgs(ctx sessionctx.Context, exprs []Expression, not bool) ([] return newExprs, flag } -// pushNotAcrossExpr try to eliminate the NOT expr in expression tree. It will records whether there's already NOT pushed. -func pushNotAcrossExpr(ctx sessionctx.Context, expr Expression, not bool) (Expression, bool) { +// pushNotAcrossExpr try to eliminate the NOT expr in expression tree. +// Input `not` indicates whether there's a `NOT` be pushed down. +// Output `changed` indicates whether the output expression differs from the +// input `expr` because of the pushed-down-not. +func pushNotAcrossExpr(ctx sessionctx.Context, expr Expression, not bool) (_ Expression, changed bool) { if f, ok := expr.(*ScalarFunction); ok { switch f.FuncName.L { case ast.UnaryNot: - return pushNotAcrossExpr(f.GetCtx(), f.GetArgs()[0], !not) + child, err := wrapWithIsTrue(ctx, true, f.GetArgs()[0], true) + if err != nil { + return expr, false + } + var childExpr Expression + childExpr, changed = pushNotAcrossExpr(f.GetCtx(), child, !not) + if !changed && !not { + return expr, false + } + return childExpr, true case ast.LT, ast.GE, ast.GT, ast.LE, ast.EQ, ast.NE: if not { return NewFunctionInternal(f.GetCtx(), oppositeOp[f.FuncName.L], f.GetType(), f.GetArgs()...), true @@ -718,6 +767,27 @@ func BuildNotNullExpr(ctx sessionctx.Context, expr Expression) Expression { return notNull } +// IsRuntimeConstExpr checks if a expr can be treated as a constant in **executor**. +func IsRuntimeConstExpr(expr Expression) bool { + switch x := expr.(type) { + case *ScalarFunction: + if _, ok := unFoldableFunctions[x.FuncName.L]; ok { + return false + } + for _, arg := range x.GetArgs() { + if !IsRuntimeConstExpr(arg) { + return false + } + } + return true + case *Column: + return false + case *Constant, *CorrelatedColumn: + return true + } + return false +} + // IsMutableEffectsExpr checks if expr contains function which is mutable or has side effects. func IsMutableEffectsExpr(expr Expression) bool { switch x := expr.(type) { diff --git a/expression/util_test.go b/expression/util_test.go index c831577b14cfe..fe0c4c96cb562 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -48,8 +48,9 @@ func (s *testUtilSuite) checkPanic(f func()) (ret bool) { func (s *testUtilSuite) TestBaseBuiltin(c *check.C) { ctx := mock.NewContext() - bf := newBaseBuiltinFuncWithTp(ctx, nil, types.ETTimestamp) - _, _, err := bf.evalInt(chunk.Row{}) + bf, err := newBaseBuiltinFuncWithTp(ctx, "", nil, types.ETTimestamp) + c.Assert(err, check.IsNil) + _, _, err = bf.evalInt(chunk.Row{}) c.Assert(err, check.NotNil) _, _, err = bf.evalReal(chunk.Row{}) c.Assert(err, check.NotNil) @@ -256,13 +257,13 @@ func (s testUtilSuite) TestGetStrIntFromConstant(c *check.C) { func (s *testUtilSuite) TestSubstituteCorCol2Constant(c *check.C) { ctx := mock.NewContext() - corCol1 := &CorrelatedColumn{Data: &One.Value} + corCol1 := &CorrelatedColumn{Data: &NewOne().Value} corCol1.RetType = types.NewFieldType(mysql.TypeLonglong) - corCol2 := &CorrelatedColumn{Data: &One.Value} + corCol2 := &CorrelatedColumn{Data: &NewOne().Value} corCol2.RetType = types.NewFieldType(mysql.TypeLonglong) cast := BuildCastFunction(ctx, corCol1, types.NewFieldType(mysql.TypeLonglong)) plus := newFunction(ast.Plus, cast, corCol2) - plus2 := newFunction(ast.Plus, plus, One) + plus2 := newFunction(ast.Plus, plus, NewOne()) ans1 := &Constant{Value: types.NewIntDatum(3), RetType: types.NewFieldType(mysql.TypeLonglong)} ret, err := SubstituteCorCol2Constant(plus2) c.Assert(err, check.IsNil) @@ -283,18 +284,47 @@ func (s *testUtilSuite) TestPushDownNot(c *check.C) { ctx := mock.NewContext() col := &Column{Index: 1, RetType: types.NewFieldType(mysql.TypeLonglong)} // !((a=1||a=1)&&a=1) - eqFunc := newFunction(ast.EQ, col, One) + eqFunc := newFunction(ast.EQ, col, NewOne()) orFunc := newFunction(ast.LogicOr, eqFunc, eqFunc) andFunc := newFunction(ast.LogicAnd, orFunc, eqFunc) notFunc := newFunction(ast.UnaryNot, andFunc) // (a!=1&&a!=1)||a=1 - neFunc := newFunction(ast.NE, col, One) + neFunc := newFunction(ast.NE, col, NewOne()) andFunc2 := newFunction(ast.LogicAnd, neFunc, neFunc) orFunc2 := newFunction(ast.LogicOr, andFunc2, neFunc) notFuncCopy := notFunc.Clone() ret := PushDownNot(ctx, notFunc) c.Assert(ret.Equal(ctx, orFunc2), check.IsTrue) c.Assert(notFunc.Equal(ctx, notFuncCopy), check.IsTrue) + + // issue 15725 + // (not not a) should be optimized to (a is true) + notFunc = newFunction(ast.UnaryNot, col) + notFunc = newFunction(ast.UnaryNot, notFunc) + ret = PushDownNot(ctx, notFunc) + c.Assert(ret.Equal(ctx, newFunction(ast.IsTruth, col)), check.IsTrue) + + // (not not (a+1)) should be optimized to (a+1 is true) + plusFunc := newFunction(ast.Plus, col, NewOne()) + notFunc = newFunction(ast.UnaryNot, plusFunc) + notFunc = newFunction(ast.UnaryNot, notFunc) + ret = PushDownNot(ctx, notFunc) + c.Assert(ret.Equal(ctx, newFunction(ast.IsTruth, plusFunc)), check.IsTrue) + + // (not not not a) should be optimized to (not (a is true)) + notFunc = newFunction(ast.UnaryNot, col) + notFunc = newFunction(ast.UnaryNot, notFunc) + notFunc = newFunction(ast.UnaryNot, notFunc) + ret = PushDownNot(ctx, notFunc) + c.Assert(ret.Equal(ctx, newFunction(ast.UnaryNot, newFunction(ast.IsTruth, col))), check.IsTrue) + + // (not not not not a) should be optimized to (a is true) + notFunc = newFunction(ast.UnaryNot, col) + notFunc = newFunction(ast.UnaryNot, notFunc) + notFunc = newFunction(ast.UnaryNot, notFunc) + notFunc = newFunction(ast.UnaryNot, notFunc) + ret = PushDownNot(ctx, notFunc) + c.Assert(ret.Equal(ctx, newFunction(ast.IsTruth, col)), check.IsTrue) } func (s *testUtilSuite) TestFilter(c *check.C) { @@ -341,7 +371,7 @@ func (s *testUtilSuite) TestHashGroupKey(c *check.C) { bufs[j] = bufs[j][:0] } var err error - err = VecEval(ctx, colExpr, input, colBuf) + err = EvalExpr(ctx, colExpr, input, colBuf) if err != nil { c.Fatal(err) } @@ -461,7 +491,7 @@ func (m *MockExpr) MarshalJSON() ([]byte, error) { return nil, nil } func (m *MockExpr) Eval(row chunk.Row) (types.Datum, error) { return types.NewDatum(m.i), m.err } func (m *MockExpr) EvalInt(ctx sessionctx.Context, row chunk.Row) (val int64, isNull bool, err error) { if x, ok := m.i.(int64); ok { - return int64(x), false, m.err + return x, false, m.err } return 0, m.i == nil, m.err } @@ -473,7 +503,7 @@ func (m *MockExpr) EvalReal(ctx sessionctx.Context, row chunk.Row) (val float64, } func (m *MockExpr) EvalString(ctx sessionctx.Context, row chunk.Row) (val string, isNull bool, err error) { if x, ok := m.i.(string); ok { - return string(x), false, m.err + return x, false, m.err } return "", m.i == nil, m.err } diff --git a/go.mod b/go.mod index 3b9eee8c60a28..81ff9621de85d 100644 --- a/go.mod +++ b/go.mod @@ -1,80 +1,66 @@ module github.com/pingcap/tidb -replace github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 => github.com/hanfei1991/tipb v0.0.0-20200421083959-59279b989667 +replace github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee => github.com/hanfei1991/tipb v0.0.0-20200506100205-b17edac4757f -replace github.com/pingcap/parser v0.0.0-20200407074807-436f1c8c4cff => github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c +replace github.com/pingcap/parser v0.0.0-20200430071110-a1bca4f6cf2a => github.com/hanfei1991/parser v0.0.0-20200506102604-dde82b00618d require ( github.com/BurntSushi/toml v0.3.1 github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d - github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd // indirect - github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142 // indirect - github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8 github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37 github.com/dgraph-io/ristretto v0.0.1 github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 - github.com/dustin/go-humanize v1.0.0 // indirect github.com/go-sql-driver/mysql v1.4.1 github.com/gogo/protobuf v1.3.1 - github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6 // indirect github.com/golang/protobuf v1.3.4 - github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db + github.com/golang/snappy v0.0.1 github.com/google/btree v1.0.0 github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3 github.com/google/uuid v1.1.1 github.com/gorilla/mux v1.7.3 - github.com/gorilla/websocket v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4 - github.com/jeremywohl/flatten v0.0.0-20190921043622-d936035e55cf - github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 - github.com/konsorten/go-windows-terminal-sequences v1.0.2 // indirect - github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808 // indirect + github.com/klauspost/cpuid v1.2.0 github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef github.com/opentracing/basictracer-go v1.0.0 - github.com/opentracing/opentracing-go v1.0.2 + github.com/opentracing/opentracing-go v1.1.0 + github.com/pingcap/br v0.0.0-20200426093517-dd11ae28b885 github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011 github.com/pingcap/failpoint v0.0.0-20200210140405-f8f9fb234798 github.com/pingcap/fn v0.0.0-20191016082858-07623b84a47d github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20200409034505-a5af800ca2ef + github.com/pingcap/kvproto v0.0.0-20200424032552-6650270c39c3 github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd - github.com/pingcap/parser v0.0.0-20200407074807-436f1c8c4cff - github.com/pingcap/pd/v4 v4.0.0-beta.1.0.20200305072537-61d9f9cc35d3 + github.com/pingcap/parser v0.0.0-20200430071110-a1bca4f6cf2a + github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 - github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incompatible - github.com/pingcap/tipb v0.0.0-20200212061130-c4d518eb1d60 + github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible + github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee github.com/prometheus/client_golang v1.0.0 github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4 github.com/prometheus/common v0.4.1 github.com/shirou/gopsutil v2.19.10+incompatible - github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371 // indirect - github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca // indirect - github.com/sirupsen/logrus v1.2.0 + github.com/sirupsen/logrus v1.4.2 github.com/soheilhy/cmux v0.1.4 github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 github.com/uber-go/atomic v1.3.2 - github.com/uber/jaeger-client-go v2.15.0+incompatible - github.com/uber/jaeger-lib v1.5.0 // indirect + github.com/uber/jaeger-client-go v2.22.1+incompatible + github.com/uber/jaeger-lib v2.2.0+incompatible // indirect go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738 go.uber.org/atomic v1.6.0 go.uber.org/automaxprocs v1.2.0 go.uber.org/zap v1.14.1 - golang.org/x/crypto v0.0.0-20191206172530-e9b2fee46413 // indirect - golang.org/x/lint v0.0.0-20200302205851-738671d3881b // indirect - golang.org/x/net v0.0.0-20200301022130-244492dfa37a - golang.org/x/sys v0.0.0-20200113162924-86b910548bc1 + golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e + golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e + golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd golang.org/x/text v0.3.2 - golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 // indirect golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1 google.golang.org/grpc v1.25.1 - gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 - honnef.co/go/tools v0.0.1-2020.1.3 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) diff --git a/go.sum b/go.sum index 6a7e10b4ab19b..8e9226c114bdd 100644 --- a/go.sum +++ b/go.sum @@ -1,6 +1,23 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= +cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= +cloud.google.com/go v0.46.3 h1:AVXDdKsrtX33oR9fbCMu/+c1o8Ofjq6Ku/MInaLVg5Y= +cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= +cloud.google.com/go/bigquery v1.0.1 h1:hL+ycaJpVE9M7nLoiXb/Pn10ENE2u+oddxbD8uu0ZVU= +cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= +cloud.google.com/go/datastore v1.0.0 h1:Kt+gOPPp2LEPWp8CSfxhsM8ik9CcyE/gYu+0r+RnZvM= +cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/pubsub v1.0.1 h1:W9tAK3E57P75u0XLLR82LZyw8VpAnhmyTOxW9qzmyj8= +cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= +cloud.google.com/go/storage v1.4.0 h1:KDdqY5VTXBTqpSbctVTt0mVvfanP6JZzNzLE0qNY100= +cloud.google.com/go/storage v1.4.0/go.mod h1:ZusYJWlOshgSBGbt6K3GnB3MT3H1xs2id9+TCl4fDBA= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/KyleBanks/depth v1.2.1/go.mod h1:jzSb9d0L43HxTQfT+oSA1EEp2q+ne2uh6XgeJcm8brE= github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= @@ -9,12 +26,17 @@ github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbt github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= +github.com/VividCortex/ewma v1.1.1 h1:MnEK4VOv6n0RSY4vtRe3h11qjxL3+t0B8yOL8iMXdcM= +github.com/VividCortex/ewma v1.1.1/go.mod h1:2Tkkvm3sRDVXaiyucHiACn4cqf7DpdyLvmxzcbUokwA= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q= github.com/appleboy/gin-jwt/v2 v2.6.3/go.mod h1:MfPYA4ogzvOcVkRwAxT7quHOtQmVKDpTwxyUrC2DNw0= github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhplt43+Wczp3rw= +github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= +github.com/aws/aws-sdk-go v1.26.1 h1:JGQggXhOiNJIqsmbYUl3cYtJZUffeOWlHtxfzGK7WPI= +github.com/aws/aws-sdk-go v1.26.1/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0 h1:HWo1m869IqiPhD389kmkxeTalrjNbbJTC8LXupb+sl0= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= @@ -24,6 +46,8 @@ github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d/go.mo github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= +github.com/cheggaaa/pb/v3 v3.0.1 h1:m0BngUk2LuSRYdx4fujDKNRXNDpbNCfptPfVT2m6OJY= +github.com/cheggaaa/pb/v3 v3.0.1/go.mod h1:SqqeMF/pMOIu3xgGoxtPYhMNQP258xE4x/XRTYua+KU= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20171208011716-f6d7a1f6fbf3/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= @@ -32,14 +56,21 @@ github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= +github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= +github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142 h1:3jFq2xL4ZajGK4aZY8jz+DAF0FHjI51BXjjSwCzS1Dk= github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f h1:JOrtw2xFKzlg+cbHpyrpLDmnN1HqhBfnX7WDiW7eG2c= +github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= +github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/cznic/golex v0.0.0-20181122101858-9c343928389c/go.mod h1:+bmmJDNmKlhWNG+gwWCkaBoTy39Fs+bzRxVBzoTQbIc= @@ -76,8 +107,14 @@ github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymF github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/erikstmartin/go-testdb v0.0.0-20160219214506-8d10e4a1bae5/go.mod h1:a2zkGnVExMxdzMo3M0Hi/3sEU+cWnZpSni0O6/Yb/P0= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= +github.com/fatih/color v1.9.0 h1:8xPHl4/q1VyqGIPif1F+1V3Y3lSmrq01EabUW3CoW5s= +github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= +github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= +github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/fsouza/fake-gcs-server v1.15.0 h1:ss/ztlt10Y64A5qslmxZKsiqW/i28t5DkRtv6qSFaLQ= +github.com/fsouza/fake-gcs-server v1.15.0/go.mod h1:HNxAJ/+FY/XSsxuwz8iIYdp2GtMmPbJ8WQjjGMxd6Qk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/ghodss/yaml v1.0.1-0.20190212211648-25d852aebe32/go.mod h1:GIjDIg/heH5DOkXY3YJ/wNhfHsQHoXGjl8G8amsYQ1I= github.com/gin-contrib/gzip v0.0.1/go.mod h1:fGBJBCdt6qCZuCAOwWuFhBB4OOq9EFqlo5dEaFhhu5w= @@ -87,10 +124,12 @@ github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= github.com/gin-gonic/gin v1.5.0/go.mod h1:Nd6IXA8m5kNZdNEHMBd93KT+mdY3+bewLgRvmCsR2Do= -github.com/go-bindata/go-bindata v3.1.2+incompatible/go.mod h1:xK8Dsgwmeed+BBsSy2XTopBn/8uK2HWuGSnA11C3Joo= +github.com/go-bindata/go-bindata/v3 v3.1.3/go.mod h1:1/zrpXsLD8YDIbhZRqXzm1Ghc7NhEvIN9+Z6R5/xH4I= github.com/go-chi/chi v4.0.2+incompatible/go.mod h1:eB3wogJHnLi3x/kFX2A+IbTBlXxmMeXJVKy9tTv1XzQ= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas= github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-openapi/jsonpointer v0.17.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M= @@ -102,9 +141,11 @@ github.com/go-openapi/jsonreference v0.19.2/go.mod h1:jMjeRr2HHw6nAVajTXJ4eiUwoh github.com/go-openapi/jsonreference v0.19.3/go.mod h1:rjx6GuL8TTa9VaixXglHmQmIL98+wF9xc8zWvFonSJ8= github.com/go-openapi/spec v0.19.0/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI= github.com/go-openapi/spec v0.19.4/go.mod h1:FpwSN1ksY1eteniUU7X0N/BgJ7a4WvBFVA8Lj9mJglo= +github.com/go-openapi/spec v0.19.7/go.mod h1:Hm2Jr4jv8G1ciIAo+frC/Ft+rR2kQDh8JHKHb3gWUSk= github.com/go-openapi/swag v0.17.0/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg= github.com/go-openapi/swag v0.19.2/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= +github.com/go-openapi/swag v0.19.8/go.mod h1:ao+8BpOPyKdpQz3AOJfbeEVpLmWAvlT1IfTe5McPyhY= github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM= github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8= github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c/go.mod h1:UqxAgEOt89sCiXlrc/ycnx00LVvUO/eS8tMUkWX4R7w= @@ -127,6 +168,8 @@ github.com/golang/groupcache v0.0.0-20181024230925-c65c006176ff/go.mod h1:cIg4er github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6 h1:ZgQEtGgCBiWRM39fZuwSd1LwSqqSW0hOdXCYYDX0R3I= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/protobuf v0.0.0-20180814211427-aa810b61a9c7/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -135,10 +178,20 @@ github.com/golang/protobuf v1.3.4 h1:87PNWwrRvUSnqS4dlcBU/ftvOIBep4sYuBLlh6rX2wk github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db h1:woRePGFeVFfLKN/pOkfl+p/TAqKOfFu+7KPlMVpok/w= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4= +github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1 h1:Xye71clBPdm5HgqGwUkwhbynsUJZhDbS20FvLhQ2izg= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3 h1:3CYI9xg87xNAD+es02gZxbX/ky4KQeoFBsNOzuoAQZg= github.com/google/pprof v0.0.0-20190930153522-6ce02741cba3/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= @@ -146,6 +199,9 @@ github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIE github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= github.com/gorilla/mux v1.7.3 h1:gnP5JzjVOuiZD07fKKToCAOjS0yOpj/qPETTXCCS6hw= github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= @@ -160,23 +216,29 @@ github.com/grpc-ecosystem/grpc-gateway v1.9.5 h1:UImYN5qQ8tuGpGE16ZmjvcTtTw24zw1 github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.12.1 h1:zCy2xE9ablevUOrUZc3Dl72Dt+ya2FNAvC2yLYMHzi4= github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqCisSPWTxCZ7sBRjU6iH9c= +github.com/grpc-ecosystem/grpc-gateway v1.14.3 h1:OCJlWkOUoTnl0neNGlf4fUm3TmbEtguw7vR+nGtnDjY= +github.com/grpc-ecosystem/grpc-gateway v1.14.3/go.mod h1:6CwZWGDSPRJidgKAtJVvND6soZe6fT7iteq8wDPdhb0= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c h1:akJYnz7Fe6AXtCTcZai3gf+HrZxuoVbnaGXxEmtxvpo= github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/hanfei1991/tipb v0.0.0-20200212061130-c4d518eb1d60 h1:i4siZGSlOhQSQYyDszniOVsGp8X72rAbl0FLZN9nxCI= -github.com/hanfei1991/tipb v0.0.0-20200212061130-c4d518eb1d60/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= -github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00 h1:wRO6S3mWrAcoS78mGn1hvjPQjqTgA+MBzbq5NkKRQRM= -github.com/hanfei1991/tipb v0.0.0-20200308103915-5f3b53798f00/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= -github.com/hanfei1991/tipb v0.0.0-20200421083959-59279b989667 h1:hrW/fUBSqAWYp0LFv5sct69QuvSjmYz89jjfWyZr7YM= -github.com/hanfei1991/tipb v0.0.0-20200421083959-59279b989667/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hanfei1991/parser v0.0.0-20200506100705-497ab5ecbc16 h1:zlhQFwi+d+tkwFvVr0Wma3mpXVizCOP1e6F04DFVHiA= +github.com/hanfei1991/parser v0.0.0-20200506100705-497ab5ecbc16/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/hanfei1991/parser v0.0.0-20200506102604-dde82b00618d h1:m2LqNBUncEXeUo44rcIlrKv0v5fbvBUihcidWDDdg2E= +github.com/hanfei1991/parser v0.0.0-20200506102604-dde82b00618d/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/hanfei1991/tipb v0.0.0-20200506100205-b17edac4757f h1:zO5fnD8Wmce+3D9ukaP/2dUL7pgGNVrfiLWwx6kdMYs= +github.com/hanfei1991/tipb v0.0.0-20200506100205-b17edac4757f/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/jeremywohl/flatten v0.0.0-20190921043622-d936035e55cf h1:Ut4tTtPNmInWiEWJRernsWm688R0RN6PFO8sZhwI0sk= -github.com/jeremywohl/flatten v0.0.0-20190921043622-d936035e55cf/go.mod h1:4AmD/VxjWcI5SRB0n6szE2A6s2fsNHDLO0nAlMHgfLQ= github.com/jinzhu/gorm v1.9.12/go.mod h1:vhTjlKSJUTWNtcbQtrMBFCxy7eXTzeCAzfL5fBZT/Qs= github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= github.com/jinzhu/now v1.0.1/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= +github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af h1:pmfjZENx5imkbgOkpRUYLnmbU7UEFbjtDA2hxJ1ichM= +github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/joho/godotenv v1.3.0/go.mod h1:7hK45KPybAkOC6peb+G5yklZfMxEjkZhHbwpqxOKXbg= github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= @@ -187,14 +249,19 @@ github.com/json-iterator/go v1.1.7 h1:KfgG9LzI+pYjr4xvmz/5H4FXjokeP+rlHLhv3iH62F github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.9 h1:9yzud/Ht36ygwatGx56VwCZtlI/2AD15T1X2sjSuGns= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024 h1:rBMNdlhTLzJjJSDIjNEXX1Pz3Hmwmz91v+zycvx9PJc= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/juju/ratelimit v1.0.1 h1:+7AIFJVQ0EQgq/K9+0Krm7m530Du7tIz0METWzN0RgY= github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSgWNm/qk= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/kevinburke/go-bindata v3.18.0+incompatible/go.mod h1:/pEEZ72flUW2p0yi30bslSp9YqD9pysLxunQDdb2CPM= github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q= github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 h1:2U0HzY8BJ8hVwDKIzp7y4voR9CX/nvcfymLmg2UiOio= github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= +github.com/klauspost/cpuid v1.2.0 h1:NMpwD2G9JSFOE1/TJjGSo5zG7Yb2bTe7eq1jH+irmeE= +github.com/klauspost/cpuid v1.2.0/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/konsorten/go-windows-terminal-sequences v1.0.2 h1:DB17ag19krx9CFsz4o3enTrPXyIXCl+2iCXH/aMAp9s= github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -208,20 +275,34 @@ github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/leodido/go-urn v1.1.0/go.mod h1:+cyI34gQWZcE1eQU7NVgKkkzdXDQHr1dBMtdAPozLkw= github.com/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII= github.com/lib/pq v1.1.1/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= +github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= +github.com/mailru/easyjson v0.7.1/go.mod h1:KAzv3t3aY1NaHWoQz1+4F1ccyAH66Jk7yos7ldAVICs= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= +github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= +github.com/mattn/go-colorable v0.1.4 h1:snbPLB8fVfU9iwbbo30TPtbLRzwWu6aJS6Xh4eaaviA= +github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= +github.com/mattn/go-isatty v0.0.11 h1:FxPOTFNqGkuDUGi3H/qkUbQO4ZiBa2brKq5r0l8TGeM= github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= +github.com/mattn/go-runewidth v0.0.4/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= +github.com/mattn/go-runewidth v0.0.7 h1:Ei8KR0497xHyKJPAv59M1dkC+rOZCMBJ+t3fZ+twI54= +github.com/mattn/go-runewidth v0.0.7/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-shellwords v1.0.3/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vqg+NOMyg4B2o= github.com/mattn/go-sqlite3 v2.0.1+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81/go.mod h1:KQ7+USdGKfpPjXk4Ga+5XxQM4Lm4e3gAogrreFAYpOg= +github.com/mgechev/revive v1.0.2/go.mod h1:rb0dQy1LVAxW9SWy5R3LPUjevzUbUS316U5MFySA2lo= +github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -231,6 +312,8 @@ github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3Rllmb github.com/montanaflynn/stats v0.0.0-20151014174947-eeaced052adb/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808 h1:pmpDGKLw4n82EtrNiLqB+xSz/JQwFOaZuMALYUHwX5s= github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= +github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= +github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 h1:7KAv7KMGTTqSmYZtNdcNTgsos+vFzULLwyElndwn+5c= github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7/go.mod h1:iWMfgwqYW+e8n5lC/jjNEhwcjbRDpl5NT7n2h+4UNcI= @@ -238,18 +321,30 @@ github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef h1:K0Fn+DoFqNqktdZtdV3 github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef/go.mod h1:7WjlapSfwQyo6LNmIvEWzsW1hbBQfpUO4JWnuQRmva8= github.com/nicksnyder/go-i18n v1.10.0/go.mod h1:HrK7VCrbOvQoUAQ7Vpy7i87N7JZZZ7R2xBGjv0j365Q= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= +github.com/olekukonko/tablewriter v0.0.4/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA= github.com/onsi/ginkgo v1.6.0 h1:Ix8l273rp3QzYgXSR+c8d1fTG7UPgYkOSELPhiY/YGw= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.11.0 h1:JAKSXpt1YjtLA7YpPiqO9ss6sNXEsPfSGdwN0UHqzrw= +github.com/onsi/ginkgo v1.11.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/gomega v1.4.2 h1:3mYCb7aPxS/RU7TI1y4rkEn1oKmPRjNJLNEXgw7MH2I= github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= +github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= +github.com/onsi/gomega v1.8.1 h1:C5Dqfs/LeauYDX0jJXIe2SWmwCbGzx9yF8C8xy3Lh34= +github.com/onsi/gomega v1.8.1/go.mod h1:Ho0h+IUsWyvy1OpqCwxlQ/21gkhVunqlU8fDGcoTdcA= github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= github.com/opentracing/opentracing-go v1.0.2 h1:3jA2P6O1F9UOrWVpwrIo17pu01KWvNWg4X946/Y5Zwg= github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU= +github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pelletier/go-toml v1.3.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= -github.com/pingcap-incubator/tidb-dashboard v0.0.0-20200302022638-35a6e979dca9/go.mod h1:YUceA4BHY/MTtp63yZLTYP22waFSwMNo9lXq2FDtzVw= +github.com/pingcap-incubator/tidb-dashboard v0.0.0-20200407064406-b2b8ad403d01/go.mod h1:77fCh8d3oKzC5ceOJWeZXAS/mLzVgdZ7rKniwmOyFuo= +github.com/pingcap/br v0.0.0-20200426093517-dd11ae28b885 h1:gI14HoGBF9UyECMgqSRZx1ONhREtrZe8JCmZ/6OFilw= +github.com/pingcap/br v0.0.0-20200426093517-dd11ae28b885/go.mod h1:4w3meMnk7HDNpNgjuRAxavruTeKJvUiXxoEWTjzXPnA= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= @@ -271,22 +366,34 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200214064158-62d31900d88e/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200221034943-a2aa1d1e20a8/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200409034505-a5af800ca2ef h1:t+bOucRUlIlzW+6S32qG8ufu4iC8F8LEld4Rdhhp1Aw= -github.com/pingcap/kvproto v0.0.0-20200409034505-a5af800ca2ef/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20200417092353-efbe03bcffbd/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20200420075417-e0c6e8842f22/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20200423020121-038e31959c2a/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20200424032552-6650270c39c3 h1:EBq71gaLG7bxgCjR5DA00mzvF5dwEZBPd8xuGSztamw= +github.com/pingcap/kvproto v0.0.0-20200424032552-6650270c39c3/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= +github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= +github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfEgRJ4T9NGgGTxdHpJerent7rM= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfEgRJ4T9NGgGTxdHpJerent7rM= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/pd/v4 v4.0.0-beta.1.0.20200305072537-61d9f9cc35d3 h1:Yrp99FnjHAEuDrSBql2l0IqCtJX7KwJbTsD5hIArkvk= -github.com/pingcap/pd/v4 v4.0.0-beta.1.0.20200305072537-61d9f9cc35d3/go.mod h1:25GfNw6+Jcr9kca5rtmTb4gKCJ4jOpow2zV2S9Dgafs= +github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= +github.com/pingcap/parser v0.0.0-20200424075042-8222d8b724a4/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/pingcap/parser v0.0.0-20200430071110-a1bca4f6cf2a h1:e/Lcw6VDQgtErYKszjeCi+nN4pB9zMzY0bTwMqfazps= +github.com/pingcap/parser v0.0.0-20200430071110-a1bca4f6cf2a/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 h1:JTzYYukREvxVSKW/ncrzNjFitd8snoQ/Xz32pw8i+s8= +github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= +github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= -github.com/pingcap/sysutil v0.0.0-20200302022240-21c8c70d0ab1 h1:YUnUZ914SHFMsOSe/xgH5DKK/thtRma8X8hcszRo3CA= -github.com/pingcap/sysutil v0.0.0-20200302022240-21c8c70d0ab1/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 h1:PI8YpTl45F8ilNkrPtT4IdbcZB1SCEa+gK/U5GJYl3E= github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= +github.com/pingcap/tidb v1.1.0-beta.0.20200424154252-5ede18f10eed/go.mod h1:m2VDlJDbUeHPCXAfKPajqLmB1uLvWpkKk3zALNqDYdw= github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incompatible h1:84F7MFMfdAYObrznvRslmVu43aoihrlL+7mMyMlOi0o= github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible h1:+K5bqDYG5HT+GqLdx4GH5VmS84+xHgpHbGg6Xt6qQec= +github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= +github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee h1:XJQ6/LGzOSc/jo33AD8t7jtc4GohxcyODsYnb+kZXJM= +github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -312,6 +419,7 @@ github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6So github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= +github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= @@ -320,20 +428,31 @@ github.com/shirou/gopsutil v2.19.10+incompatible h1:lA4Pi29JEVIQIgATSeftHSY0rMGI github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4 h1:udFKJ0aHUL60LboW/A+DfgoHVedieIzIXE8uylPue0U= github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= -github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371 h1:SWV2fHctRpRrp49VXJ6UZja7gU9QLHwRpIPBN89SKEo= -github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= +github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749 h1:bUGsEnyNbVPw06Bs80sCeARAlK8lhwqGyi6UT8ymuGk= +github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca h1:3fECS8atRjByijiI8yYiuwLwQ2ZxXobW7ua/8GRB3pI= github.com/shurcooL/vfsgen v0.0.0-20181020040650-a97a25d856ca/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/sirupsen/logrus v1.2.0 h1:juTguoYk5qI21pwyTXY3B3Y5cOTH3ZUyZCg1v/mihuo= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.4.2 h1:SPIRibHv4MatM3XXNO2BJeFLZwZ2LvZgfQ5+UNI2im4= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= +github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= +github.com/spf13/cobra v0.0.5 h1:f0B+LkLX6DtmRH1isoNA9VTtNUK9K8xYd28JNNfOv/s= +github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= +github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= github.com/spf13/pflag v1.0.1 h1:aCvUg6QPl3ibpQUxyLkrEkCHtPqYJL4x9AuhqVqFis4= github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= +github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= +github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= +github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= +github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= @@ -344,11 +463,15 @@ github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81P github.com/swaggo/files v0.0.0-20190704085106-630677cd5c14/go.mod h1:gxQT6pBGRuIGunNf/+tSOB5OHvguWi8Tbt82WOkf35E= github.com/swaggo/gin-swagger v1.2.0/go.mod h1:qlH2+W7zXGZkczuL+r2nEBR2JTT+/lX05Nn6vPhc7OI= github.com/swaggo/http-swagger v0.0.0-20200103000832-0e9263c4b516/go.mod h1:O1lAbCgAAX/KZ80LM/OXwtWFI/5TvZlwxSg8Cq08PV0= +github.com/swaggo/http-swagger v0.0.0-20200308142732-58ac5e232fba/go.mod h1:O1lAbCgAAX/KZ80LM/OXwtWFI/5TvZlwxSg8Cq08PV0= github.com/swaggo/swag v1.5.1/go.mod h1:1Bl9F/ZBpVWh22nY0zmYyASPO1lI/zIwRDrpZU+tv8Y= github.com/swaggo/swag v1.6.3/go.mod h1:wcc83tB4Mb2aNiL/HP4MFeQdpHUrca+Rp/DRNgWAUio= github.com/swaggo/swag v1.6.5/go.mod h1:Y7ZLSS0d0DdxhWGVhQdu+Bu1QhaF5k0RD7FKdiAykeY= +github.com/swaggo/swag v1.6.6-0.20200323071853-8e21f4cefeea/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d h1:4J9HCZVpvDmj2tiKGSTUnb3Ok/9CEQb9oqu9LHKQQpc= github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d/go.mod h1:Z4AUp2Km+PwemOoO/VB5AOx9XSsIItzFjoJlOSiYmn0= +github.com/syndtr/goleveldb v1.0.1-0.20190625010220-02440ea7a285 h1:uSDYjYejelKyceA6DiCsngFof9jAyeaSyX9XC5a1a7Q= +github.com/syndtr/goleveldb v1.0.1-0.20190625010220-02440ea7a285/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= @@ -357,25 +480,34 @@ github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhV github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20171017195756-830351dc03c6 h1:lYIiVDtZnyTWlNwiAxLj0bbpTcx1BWCFhXjfsvmPdNc= github.com/tmc/grpc-websocket-proxy v0.0.0-20171017195756-830351dc03c6/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5 h1:LnC5Kc/wtumK+WB441p7ynQJzVuNRJiqddSIE3IlSEQ= +github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/uber-go/atomic v1.3.2 h1:Azu9lPBWRNKzYXSIwRfgRuDuS0YKsK4NFhiQv98gkxo= github.com/uber-go/atomic v1.3.2/go.mod h1:/Ct5t2lcmbJ4OSe/waGBoaVvVqtO0bmtfVNex1PFV8g= github.com/uber/jaeger-client-go v2.15.0+incompatible h1:NP3qsSqNxh8VYr956ur1N/1C1PjvOJnJykCzcD5QHbk= github.com/uber/jaeger-client-go v2.15.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= +github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM= +github.com/uber/jaeger-client-go v2.22.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-lib v1.5.0 h1:OHbgr8l656Ub3Fw5k9SWnBfIEwvoHQ+W2y+Aa9D1Uyo= github.com/uber/jaeger-lib v1.5.0/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= +github.com/uber/jaeger-lib v2.2.0+incompatible h1:MxZXOiR2JuoANZ3J6DE/U0kSFv/eJ/GfSYVCjK7dyaw= +github.com/uber/jaeger-lib v2.2.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= github.com/ugorji/go v1.1.5-pre/go.mod h1:FwP/aQVg39TXzItUBMwnWp9T9gPQnXw4Poh4/oBQZ/0= github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= github.com/ugorji/go/codec v0.0.0-20181022190402-e5e69e061d4f/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= +github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/ugorji/go/codec v1.1.5-pre/go.mod h1:tULtS6Gy1AE1yCENaw4Vb//HLH5njI2tfCQDUqRd8fI= github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= github.com/unrolled/render v0.0.0-20171102162132-65450fb6b2d3/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= +github.com/urfave/cli/v2 v2.1.1/go.mod h1:SE9GqnLQmjVa0iPEY0f1w3ygNIYcIJ0OKPMoW2caLfQ= github.com/urfave/negroni v0.3.0 h1:PaXOb61mWeZJxc1Ji2xJjpVg9QfPo0rrB+lHyBxGNSU= github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= +github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= @@ -383,12 +515,18 @@ go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738 h1:lWF4f9Nypl1ZqSb4gLeh/DGvBYVaUYHuiB93teOmwgc= go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= +go.opencensus.io v0.22.2 h1:75k/FF0Q2YM8QYo07VPddOLBslDt1MZOdEslOHvmzAs= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/automaxprocs v1.2.0 h1:+RUihKM+nmYUoB9w0D0Ov5TJ2PpFO2FgenTxMJiZBZA= go.uber.org/automaxprocs v1.2.0/go.mod h1:YfO3fm683kQpzETxlTGZhGIVmXAhaw3gxeBADbpZtnU= +go.uber.org/dig v1.8.0/go.mod h1:X34SnWGr8Fyla9zQNO2GSO2D+TIuqB14OS8JhYocIyw= +go.uber.org/fx v1.10.0/go.mod h1:vLRicqpG/qQEzno4SYU86iCwfT95EZza+Eba0ItuxqY= go.uber.org/goleak v0.10.0 h1:G3eWbSNIskeRqtsN/1uI5B+eP73y3JUuBsv9AZjehb4= go.uber.org/goleak v0.10.0/go.mod h1:VCZuO8V8mFPlL0F5J5GK1rtHV3DrFcQ1R8ryq7FK0aI= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= @@ -406,22 +544,38 @@ go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= go.uber.org/zap v1.14.1 h1:nYDKopTbvAPq/NrUVZwT15y2lpROBiLLyoRTbXOYWOo= go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190325154230-a5d413f7728c/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191205180655-e7c4368fe9dd/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20191206172530-e9b2fee46413 h1:ULYEB3JvPRE/IfO+9uO7vKV/xzVTO7XPAwm8xbf4w2g= golang.org/x/crypto v0.0.0-20191206172530-e9b2fee46413/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= +golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136 h1:A1gGSx58LAGVHUUsOf7IiR0u8Xb6W51gRwfDBhkdcaw= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/lint v0.0.0-20200302205851-738671d3881b h1:Wh+f8QHJXR411sJR8/vRBTZ7YapZaRvUcLFFJhusH0k= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee h1:WG0RUwxtNT4qqaXX3DPA8zHFNm/D9xaBpxzHt1WcA/E= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.2.0 h1:KU7oHjnv3XNWfa5COkzUifxZmxp1TyI7ImMXqFxLwvQ= @@ -432,10 +586,13 @@ golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= golang.org/x/net v0.0.0-20190611141213-3f473d35a33a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -444,36 +601,48 @@ golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20191002035440-2ec189313ef0 h1:2mqDk8w/o6UmeUCu5Qiq2y7iMf6anbx+YA8d1JFoFrs= golang.org/x/net v0.0.0-20191002035440-2ec189313ef0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20200301022130-244492dfa37a h1:GuSPYbZzB5/dcLNCwLQLsg3obCJtX9IJhpXkvY7kzk0= -golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e h1:3G+cUijn7XD+S4eJFddp53Pv7+slrESplyjG25HgL+k= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45 h1:SVwTIAaPC2U/AvvLNZ2a7OVsmBpC8L5BlwK1whH3hm0= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e h1:vcxGaoTs7kV8m5Np9uUNQin4BrLOthgV7252N8V+FwY= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181228144115-9a3f9b0469bb/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190610200419-93c9922d18ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190616124812-15dcb6c0061f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20200113162924-86b910548bc1 h1:gZpLHxUX5BdYLA08Lj4YCJNN/jk7KtquiArPoeX0WvA= -golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd h1:xhmwyvizuTgC2qz7ZlMluP20uW+C3Rm0FD/WLDX8884= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4 h1:SvFZT6jyqRaOeXpc5h/JSfZenJ2O330aBsf7JfSUXmQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -482,18 +651,32 @@ golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGm golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190606050223-4d9ae51c2468/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190611222205-d73e1c7e250b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190614205625-5aca471b1d59/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191030062658-86caa796c7ab/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191107010934-f79515f33823/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191114200427-caa0b0f7d508/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200301222351-066e0c02454c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200325010219-a49f79bcc224/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1 h1:odiryKYJy7CjdrZxhrcE1Z8L9+kGyGZOnfpuauvdCeU= golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -501,16 +684,35 @@ golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898 h1:/atklqdjdhuosWIl6AIbO golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= +google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.14.0 h1:uMf5uLi4eQMRrMKhCplNik4U4H8Z6C1br3zOtAa/aDE= +google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0 h1:/wp5JvzpHIxhs/dumFmF7BXTf3Z+dd4uXta4kVyO508= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.1 h1:QzqyMA1tlu6CgqCDUtU9V+ZKhLFT2dkJuANu5QaxI3I= +google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20181004005441-af9cb2a35e7f/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c h1:hrpEMCZ2O7DR5gC1n2AJGVhrwiEjOi35+jxtIuZpTMo= google.golang.org/genproto v0.0.0-20190927181202-20e1ac93f88c/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9 h1:6XzpBoANz1NqMNfDXzc2QmHmbb1vyMsvRfoP5rM+K1I= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRnRtcA= @@ -542,15 +744,20 @@ gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.3 h1:fvjTMHxHEw/mxHbtzPi3JCcKXQRAnQTBRo6YCJSVHKI= gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8 h1:obN1ZagJSUGI0Ek/LBmuj4SNLPfIny3KsKFopxRdj10= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3 h1:sXmLre5bzIR6ypkjXCDI3jHPssRhc8KD/Ome589sc3U= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= +rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 h1:VO9oZbbkvTwqLimlQt15QNdOOBArT2dw/bvzsMZBiqQ= diff --git a/infoschema/builder.go b/infoschema/builder.go index 896e8642debc0..c58c8499c907f 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -74,8 +74,9 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro // We try to reuse the old allocator, so the cached auto ID can be reused. var allocs autoid.Allocators if tableIDIsValid(oldTableID) { - if oldTableID == newTableID && diff.Type != model.ActionRenameTable && diff.Type != model.ActionRebaseAutoID { - allocs, _ = b.is.AllocByID(oldTableID) + if oldTableID == newTableID && diff.Type != model.ActionRenameTable { + oldAllocs, _ := b.is.AllocByID(oldTableID) + allocs = filterAllocators(diff, oldAllocs) } tmpIDs := tblIDs @@ -108,6 +109,32 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro return tblIDs, nil } +func filterAllocators(diff *model.SchemaDiff, oldAllocs autoid.Allocators) autoid.Allocators { + var newAllocs autoid.Allocators + switch diff.Type { + case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: + // Only drop auto-increment allocator. + for _, alloc := range oldAllocs { + if alloc.GetType() == autoid.RowIDAllocType || alloc.GetType() == autoid.AutoIncrementType { + continue + } + newAllocs = append(newAllocs, alloc) + } + case model.ActionRebaseAutoRandomBase: + // Only drop auto-random allocator. + for _, alloc := range oldAllocs { + if alloc.GetType() == autoid.AutoRandomType { + continue + } + newAllocs = append(newAllocs, alloc) + } + default: + // Keep all allocators. + newAllocs = oldAllocs + } + return newAllocs +} + func appendAffectedIDs(affected []int64, tblInfo *model.TableInfo) []int64 { affected = append(affected, tblInfo.ID) if pi := tblInfo.GetPartitionInfo(); pi != nil { @@ -224,6 +251,15 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i if len(allocs) == 0 { allocs = autoid.NewAllocatorsFromTblInfo(b.handle.store, dbInfo.ID, tblInfo) + } else { + switch tp { + case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType) + allocs = append(allocs, newAlloc) + case model.ActionRebaseAutoRandomBase: + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) + allocs = append(allocs, newAlloc) + } } tbl, err := tables.TableFromMeta(allocs, tblInfo) if err != nil { diff --git a/infoschema/cluster.go b/infoschema/cluster.go index 097cfb0c12447..2bb3998934815 100644 --- a/infoschema/cluster.go +++ b/infoschema/cluster.go @@ -30,12 +30,18 @@ const ( // ClusterTableSlowLog is the string constant of cluster slow query memory table. ClusterTableSlowLog = "CLUSTER_SLOW_QUERY" ClusterTableProcesslist = "CLUSTER_PROCESSLIST" + // ClusterTableStatementsSummary is the string constant of cluster statement summary table. + ClusterTableStatementsSummary = "CLUSTER_STATEMENTS_SUMMARY" + // ClusterTableStatementsSummaryHistory is the string constant of cluster statement summary history table. + ClusterTableStatementsSummaryHistory = "CLUSTER_STATEMENTS_SUMMARY_HISTORY" ) // memTableToClusterTables means add memory table to cluster table. var memTableToClusterTables = map[string]string{ - TableSlowQuery: ClusterTableSlowLog, - TableProcesslist: ClusterTableProcesslist, + TableSlowQuery: ClusterTableSlowLog, + TableProcesslist: ClusterTableProcesslist, + TableStatementsSummary: ClusterTableStatementsSummary, + TableStatementsSummaryHistory: ClusterTableStatementsSummaryHistory, } func init() { diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 604d1e16ea624..72a2fe48d88db 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -50,6 +50,8 @@ type InfoSchema interface { SchemaMetaVersion() int64 // TableIsView indicates whether the schema.table is a view. TableIsView(schema, table model.CIStr) bool + // TableIsSequence indicates whether the schema.table is a sequence. + TableIsSequence(schema, table model.CIStr) bool FindTableByPartitionID(partitionID int64) (table.Table, *model.DBInfo) } @@ -178,6 +180,15 @@ func (is *infoSchema) TableIsView(schema, table model.CIStr) bool { return false } +func (is *infoSchema) TableIsSequence(schema, table model.CIStr) bool { + if tbNames, ok := is.schemaMap[schema.L]; ok { + if t, ok := tbNames.tables[table.L]; ok { + return t.Meta().IsSequence() + } + } + return false +} + func (is *infoSchema) TableExists(schema, table model.CIStr) bool { if tbNames, ok := is.schemaMap[schema.L]; ok { if _, ok = tbNames.tables[table.L]; ok { diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index fc4e651ef856f..fb84d6835fa0a 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -162,6 +162,7 @@ func (*testSuite) TestT(c *C) { c.Assert(is.TableExists(dbName, tbName), IsTrue) c.Assert(is.TableExists(dbName, noexist), IsFalse) c.Assert(is.TableIsView(dbName, tbName), IsFalse) + c.Assert(is.TableIsSequence(dbName, tbName), IsFalse) tb, ok := is.TableByID(tbID) c.Assert(ok, IsTrue) diff --git a/infoschema/metric_table_def.go b/infoschema/metric_table_def.go index f33f004d197c8..c2453e770259a 100644 --- a/infoschema/metric_table_def.go +++ b/infoschema/metric_table_def.go @@ -387,13 +387,13 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance", "type"}, Comment: "The total count of pd client command fail", }, - "pd_handle_request_ops": { + "pd_request_rpc_ops": { PromQL: "sum(rate(pd_client_request_handle_requests_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION]))", Labels: []string{"instance", "type"}, - Comment: "pd handle request operation per second", + Comment: "pd client handle request operation per second", }, - "pd_handle_request_duration": { - Comment: "The quantile of pd handle request duration(second)", + "pd_request_rpc_duration": { + Comment: "The quantile of pd client handle request duration(second)", PromQL: "histogram_quantile($QUANTILE, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (le,type,instance))", Labels: []string{"instance", "type"}, Quantile: 0.999, @@ -839,7 +839,7 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance", "type"}, Comment: "The current term of Raft", }, - "pd_handle_request_duration_avg": { + "pd_request_rpc_duration_avg": { PromQL: `avg(rate(pd_client_request_handle_requests_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type) / avg(rate(pd_client_request_handle_requests_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (type)`, Labels: []string{"type"}, }, @@ -2263,15 +2263,15 @@ var MetricTableMap = map[string]MetricTableDef{ Labels: []string{"instance", "grpc_method"}, Comment: "The total time of completing each kind of gRPC commands", }, - "pd_handle_request_total_count": { + "pd_request_rpc_total_count": { PromQL: "sum(increase(pd_client_request_handle_requests_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,type)", Labels: []string{"instance", "type"}, - Comment: "The total count of pd handle request duration(second)", + Comment: "The total count of pd client handle request duration(second)", }, - "pd_handle_request_total_time": { + "pd_request_rpc_total_time": { PromQL: "sum(increase(pd_client_request_handle_requests_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,type)", Labels: []string{"instance", "type"}, - Comment: "The total time of pd handle request duration(second)", + Comment: "The total time of pd client handle request duration(second)", }, "pd_handle_transactions_total_count": { PromQL: "sum(increase(pd_txn_handle_txns_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance,result)", @@ -2334,22 +2334,22 @@ var MetricTableMap = map[string]MetricTableDef{ Comment: "The total time of duration of the waiting time for getting the start timestamp oracle", }, "pd_tso_rpc_total_count": { - PromQL: "sum(increase(pd_client_request_handle_requests_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", + PromQL: "sum(increase(pd_client_request_handle_requests_duration_seconds_count{type=\"tso\"}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total count of a client sending TSO request until received the response.", }, "pd_tso_rpc_total_time": { - PromQL: "sum(increase(pd_client_request_handle_requests_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", + PromQL: "sum(increase(pd_client_request_handle_requests_duration_seconds_sum{type=\"tso\"}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total time of a client sending TSO request until received the response.", }, "pd_tso_wait_total_count": { - PromQL: "sum(increase(pd_client_cmd_handle_cmds_duration_seconds_count{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", + PromQL: "sum(increase(pd_client_cmd_handle_cmds_duration_seconds_count{type=\"wait\"}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total count of a client starting to wait for the TS until received the TS result.", }, "pd_tso_wait_total_time": { - PromQL: "sum(increase(pd_client_cmd_handle_cmds_duration_seconds_sum{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (instance)", + PromQL: "sum(increase(pd_client_cmd_handle_cmds_duration_seconds_sum{type=\"wait\"}[$RANGE_DURATION])) by (instance)", Labels: []string{"instance"}, Comment: "The total time of a client starting to wait for the TS until received the TS result.", }, diff --git a/infoschema/perfschema/const.go b/infoschema/perfschema/const.go index a01a1136da1f5..5ce6ab82c8278 100644 --- a/infoschema/perfschema/const.go +++ b/infoschema/perfschema/const.go @@ -32,9 +32,6 @@ var perfSchemaTables = []string{ tableStagesHistory, tableStagesHistoryLong, tableEventsStatementsSummaryByDigest, - tableEventsStatementsSummaryByDigestHistory, - tableClusterEventsStatementsSummaryByDigest, - tableClusterEventsStatementsSummaryByDigestHistory, tableTiDBProfileCPU, tableTiDBProfileMemory, tableTiDBProfileMutex, @@ -384,93 +381,45 @@ const tableStagesHistoryLong = "CREATE TABLE if not exists performance_schema." "NESTING_EVENT_ID BIGINT(20) UNSIGNED," + "NESTING_EVENT_TYPE ENUM('TRANSACTION','STATEMENT','STAGE'));" -// Fields in `events_statements_summary_by_digest` and `events_statements_summary_by_digest_history` are the same. -const fieldsInEventsStatementsSummary = "SUMMARY_BEGIN_TIME TIMESTAMP(6) NOT NULL COMMENT 'Begin time of this summary'," + - "SUMMARY_END_TIME TIMESTAMP(6) NOT NULL COMMENT 'End time of this summary'," + - "STMT_TYPE VARCHAR(64) NOT NULL COMMENT 'Statement type'," + - "SCHEMA_NAME VARCHAR(64) DEFAULT NULL COMMENT 'Current schema'," + - "DIGEST VARCHAR(64) NOT NULL," + - "DIGEST_TEXT LONGTEXT NOT NULL COMMENT 'Normalized statement'," + - "TABLE_NAMES TEXT DEFAULT NULL COMMENT 'Involved tables'," + - "INDEX_NAMES TEXT DEFAULT NULL COMMENT 'Used indices'," + - "SAMPLE_USER VARCHAR(64) DEFAULT NULL COMMENT 'Sampled user who executed these statements'," + - "EXEC_COUNT BIGINT(20) UNSIGNED NOT NULL COMMENT 'Count of execution'," + - "SUM_LATENCY BIGINT(20) UNSIGNED NOT NULL COMMENT 'Sum latency of these statements'," + - "MAX_LATENCY BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max latency of these statements'," + - "MIN_LATENCY BIGINT(20) UNSIGNED NOT NULL COMMENT 'Min latency of these statements'," + - "AVG_LATENCY BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average latency of these statements'," + - "AVG_PARSE_LATENCY BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average latency of parsing'," + - "MAX_PARSE_LATENCY BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max latency of parsing'," + - "AVG_COMPILE_LATENCY BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average latency of compiling'," + - "MAX_COMPILE_LATENCY BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max latency of compiling'," + - "COP_TASK_NUM BIGINT(20) UNSIGNED NOT NULL COMMENT 'Total number of CopTasks'," + - "AVG_COP_PROCESS_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average processing time of CopTasks'," + - "MAX_COP_PROCESS_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max processing time of CopTasks'," + - "MAX_COP_PROCESS_ADDRESS VARCHAR(256) DEFAULT NULL COMMENT 'Address of the CopTask with max processing time'," + - "AVG_COP_WAIT_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average waiting time of CopTasks'," + - "MAX_COP_WAIT_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max waiting time of CopTasks'," + - "MAX_COP_WAIT_ADDRESS VARCHAR(256) DEFAULT NULL COMMENT 'Address of the CopTask with max waiting time'," + - "AVG_PROCESS_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average processing time in TiKV'," + - "MAX_PROCESS_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max processing time in TiKV'," + - "AVG_WAIT_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average waiting time in TiKV'," + - "MAX_WAIT_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max waiting time in TiKV'," + - "AVG_BACKOFF_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average waiting time before retry'," + - "MAX_BACKOFF_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max waiting time before retry'," + - "AVG_TOTAL_KEYS BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average number of scanned keys'," + - "MAX_TOTAL_KEYS BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max number of scanned keys'," + - "AVG_PROCESSED_KEYS BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average number of processed keys'," + - "MAX_PROCESSED_KEYS BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max number of processed keys'," + - "AVG_PREWRITE_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average time of prewrite phase'," + - "MAX_PREWRITE_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max time of prewrite phase'," + - "AVG_COMMIT_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average time of commit phase'," + - "MAX_COMMIT_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max time of commit phase'," + - "AVG_GET_COMMIT_TS_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average time of getting commit_ts'," + - "MAX_GET_COMMIT_TS_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max time of getting commit_ts'," + - "AVG_COMMIT_BACKOFF_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average time before retry during commit phase'," + - "MAX_COMMIT_BACKOFF_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max time before retry during commit phase'," + - "AVG_RESOLVE_LOCK_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average time for resolving locks'," + - "MAX_RESOLVE_LOCK_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max time for resolving locks'," + - "AVG_LOCAL_LATCH_WAIT_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average waiting time of local transaction'," + - "MAX_LOCAL_LATCH_WAIT_TIME BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max waiting time of local transaction'," + - "AVG_WRITE_KEYS DOUBLE UNSIGNED NOT NULL COMMENT 'Average count of written keys'," + - "MAX_WRITE_KEYS BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max count of written keys'," + - "AVG_WRITE_SIZE DOUBLE NOT NULL COMMENT 'Average amount of written bytes'," + - "MAX_WRITE_SIZE BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max amount of written bytes'," + - "AVG_PREWRITE_REGIONS DOUBLE NOT NULL COMMENT 'Average number of involved regions in prewrite phase'," + - "MAX_PREWRITE_REGIONS INT(11) UNSIGNED NOT NULL COMMENT 'Max number of involved regions in prewrite phase'," + - "AVG_TXN_RETRY DOUBLE NOT NULL COMMENT 'Average number of transaction retries'," + - "MAX_TXN_RETRY INT(11) UNSIGNED NOT NULL COMMENT 'Max number of transaction retries'," + - "SUM_BACKOFF_TIMES BIGINT(20) UNSIGNED NOT NULL COMMENT 'Sum of retries'," + - "BACKOFF_TYPES VARCHAR(1024) DEFAULT NULL COMMENT 'Types of errors and the number of retries for each type'," + - "AVG_MEM BIGINT(20) UNSIGNED NOT NULL COMMENT 'Average memory(byte) used'," + - "MAX_MEM BIGINT(20) UNSIGNED NOT NULL COMMENT 'Max memory(byte) used'," + - "AVG_AFFECTED_ROWS DOUBLE UNSIGNED NOT NULL COMMENT 'Average number of rows affected'," + - "FIRST_SEEN TIMESTAMP(6) NOT NULL COMMENT 'The time these statements are seen for the first time'," + - "LAST_SEEN TIMESTAMP(6) NOT NULL COMMENT 'The time these statements are seen for the last time'," + - "QUERY_SAMPLE_TEXT LONGTEXT DEFAULT NULL COMMENT 'Sampled original statement'," + - "PREV_SAMPLE_TEXT LONGTEXT DEFAULT NULL COMMENT 'The previous statement before commit'," + - "PLAN_DIGEST VARCHAR(64) DEFAULT NULL COMMENT 'Digest of its execution plan'," + - "PLAN LONGTEXT DEFAULT NULL COMMENT 'Sampled execution plan');" - // tableEventsStatementsSummaryByDigest contains the column name definitions for table // events_statements_summary_by_digest, same as MySQL. -const tableEventsStatementsSummaryByDigest = "CREATE TABLE if not exists " + tableNameEventsStatementsSummaryByDigest + - "(" + fieldsInEventsStatementsSummary - -// tableEventsStatementsSummaryByDigestHistory contains the column name definitions for table -// events_statements_summary_by_digest_history. -const tableEventsStatementsSummaryByDigestHistory = "CREATE TABLE if not exists " + tableNameEventsStatementsSummaryByDigestHistory + - "(" + fieldsInEventsStatementsSummary - -// tableClusterEventsStatementsSummaryByDigest contains the column name definitions for table -// cluster_events_statements_summary_by_digest, same as MySQL. -const tableClusterEventsStatementsSummaryByDigest = "CREATE TABLE if not exists " + tableNameClusterEventsStatementsSummaryByDigest + - "(ADDRESS VARCHAR(64) DEFAULT NULL COMMENT 'Address of TiDB server'," + fieldsInEventsStatementsSummary - -// tableClusterEventsStatementsSummaryByDigestHistory contains the column name definitions for table -// cluster_events_statements_summary_by_digest_history. -const tableClusterEventsStatementsSummaryByDigestHistory = "CREATE TABLE if not exists " + tableNameClusterEventsStatementsSummaryByDigestHistory + - "(ADDRESS VARCHAR(64) DEFAULT NULL COMMENT 'Address of TiDB server'," + fieldsInEventsStatementsSummary +const tableEventsStatementsSummaryByDigest = "CREATE TABLE if not exists performance_schema." + tableNameEventsStatementsSummaryByDigest + " (" + + "SCHEMA_NAME varchar(64) DEFAULT NULL," + + "DIGEST varchar(64) DEFAULT NULL," + + "DIGEST_TEXT longtext," + + "COUNT_STAR bigint unsigned NOT NULL," + + "SUM_TIMER_WAIT bigint unsigned NOT NULL," + + "MIN_TIMER_WAIT bigint unsigned NOT NULL," + + "AVG_TIMER_WAIT bigint unsigned NOT NULL," + + "MAX_TIMER_WAIT bigint unsigned NOT NULL," + + "SUM_LOCK_TIME bigint unsigned NOT NULL," + + "SUM_ERRORS bigint unsigned NOT NULL," + + "SUM_WARNINGS bigint unsigned NOT NULL," + + "SUM_ROWS_AFFECTED bigint unsigned NOT NULL," + + "SUM_ROWS_SENT bigint unsigned NOT NULL," + + "SUM_ROWS_EXAMINED bigint unsigned NOT NULL," + + "SUM_CREATED_TMP_DISK_TABLES bigint unsigned NOT NULL," + + "SUM_CREATED_TMP_TABLES bigint unsigned NOT NULL," + + "SUM_SELECT_FULL_JOIN bigint unsigned NOT NULL," + + "SUM_SELECT_FULL_RANGE_JOIN bigint unsigned NOT NULL," + + "SUM_SELECT_RANGE bigint unsigned NOT NULL," + + "SUM_SELECT_RANGE_CHECK bigint unsigned NOT NULL," + + "SUM_SELECT_SCAN bigint unsigned NOT NULL," + + "SUM_SORT_MERGE_PASSES bigint unsigned NOT NULL," + + "SUM_SORT_RANGE bigint unsigned NOT NULL," + + "SUM_SORT_ROWS bigint unsigned NOT NULL," + + "SUM_SORT_SCAN bigint unsigned NOT NULL," + + "SUM_NO_INDEX_USED bigint unsigned NOT NULL," + + "SUM_NO_GOOD_INDEX_USED bigint unsigned NOT NULL," + + "FIRST_SEEN timestamp(6) NOT NULL DEFAULT '0000-00-00 00:00:00.000000'," + + "LAST_SEEN timestamp(6) NOT NULL DEFAULT '0000-00-00 00:00:00.000000'," + + "QUANTILE_95 bigint unsigned NOT NULL," + + "QUANTILE_99 bigint unsigned NOT NULL," + + "QUANTILE_999 bigint unsigned NOT NULL," + + "QUERY_SAMPLE_TEXT longtext," + + "QUERY_SAMPLE_SEEN timestamp(6) NOT NULL DEFAULT '0000-00-00 00:00:00.000000'," + + "QUERY_SAMPLE_TIMER_WAIT bigint unsigned NOT NULL," + + "UNIQUE KEY `SCHEMA_NAME` (`SCHEMA_NAME`,`DIGEST`));" // tableTiDBProfileCPU contains the columns name definitions for table tidb_profile_cpu const tableTiDBProfileCPU = "CREATE TABLE IF NOT EXISTS " + tableNameTiDBProfileCPU + " (" + diff --git a/infoschema/perfschema/tables.go b/infoschema/perfschema/tables.go index b29cb1b424e14..ff12ecb02a0df 100644 --- a/infoschema/perfschema/tables.go +++ b/infoschema/perfschema/tables.go @@ -24,90 +24,81 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" - "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/profile" - "github.com/pingcap/tidb/util/stmtsummary" ) const ( - tableNameGlobalStatus = "global_status" - tableNameSessionStatus = "session_status" - tableNameSetupActors = "setup_actors" - tableNameSetupObjects = "setup_objects" - tableNameSetupInstruments = "setup_instruments" - tableNameSetupConsumers = "setup_consumers" - tableNameEventsStatementsCurrent = "events_statements_current" - tableNameEventsStatementsHistory = "events_statements_history" - tableNameEventsStatementsHistoryLong = "events_statements_history_long" - tableNamePreparedStatementsInstances = "prepared_statements_instances" - tableNameEventsTransactionsCurrent = "events_transactions_current" - tableNameEventsTransactionsHistory = "events_transactions_history" - tableNameEventsTransactionsHistoryLong = "events_transactions_history_long" - tableNameEventsStagesCurrent = "events_stages_current" - tableNameEventsStagesHistory = "events_stages_history" - tableNameEventsStagesHistoryLong = "events_stages_history_long" - tableNameEventsStatementsSummaryByDigest = "events_statements_summary_by_digest" - tableNameEventsStatementsSummaryByDigestHistory = "events_statements_summary_by_digest_history" - tableNameClusterEventsStatementsSummaryByDigest = "cluster_events_statements_summary_by_digest" - tableNameClusterEventsStatementsSummaryByDigestHistory = "cluster_events_statements_summary_by_digest_history" - tableNameTiDBProfileCPU = "tidb_profile_cpu" - tableNameTiDBProfileMemory = "tidb_profile_memory" - tableNameTiDBProfileMutex = "tidb_profile_mutex" - tableNameTiDBProfileAllocs = "tidb_profile_allocs" - tableNameTiDBProfileBlock = "tidb_profile_block" - tableNameTiDBProfileGoroutines = "tidb_profile_goroutines" - tableNameTiKVProfileCPU = "tikv_profile_cpu" - tableNamePDProfileCPU = "pd_profile_cpu" - tableNamePDProfileMemory = "pd_profile_memory" - tableNamePDProfileMutex = "pd_profile_mutex" - tableNamePDProfileAllocs = "pd_profile_allocs" - tableNamePDProfileBlock = "pd_profile_block" - tableNamePDProfileGoroutines = "pd_profile_goroutines" + tableNameGlobalStatus = "global_status" + tableNameSessionStatus = "session_status" + tableNameSetupActors = "setup_actors" + tableNameSetupObjects = "setup_objects" + tableNameSetupInstruments = "setup_instruments" + tableNameSetupConsumers = "setup_consumers" + tableNameEventsStatementsCurrent = "events_statements_current" + tableNameEventsStatementsHistory = "events_statements_history" + tableNameEventsStatementsHistoryLong = "events_statements_history_long" + tableNamePreparedStatementsInstances = "prepared_statements_instances" + tableNameEventsTransactionsCurrent = "events_transactions_current" + tableNameEventsTransactionsHistory = "events_transactions_history" + tableNameEventsTransactionsHistoryLong = "events_transactions_history_long" + tableNameEventsStagesCurrent = "events_stages_current" + tableNameEventsStagesHistory = "events_stages_history" + tableNameEventsStagesHistoryLong = "events_stages_history_long" + tableNameEventsStatementsSummaryByDigest = "events_statements_summary_by_digest" + tableNameTiDBProfileCPU = "tidb_profile_cpu" + tableNameTiDBProfileMemory = "tidb_profile_memory" + tableNameTiDBProfileMutex = "tidb_profile_mutex" + tableNameTiDBProfileAllocs = "tidb_profile_allocs" + tableNameTiDBProfileBlock = "tidb_profile_block" + tableNameTiDBProfileGoroutines = "tidb_profile_goroutines" + tableNameTiKVProfileCPU = "tikv_profile_cpu" + tableNamePDProfileCPU = "pd_profile_cpu" + tableNamePDProfileMemory = "pd_profile_memory" + tableNamePDProfileMutex = "pd_profile_mutex" + tableNamePDProfileAllocs = "pd_profile_allocs" + tableNamePDProfileBlock = "pd_profile_block" + tableNamePDProfileGoroutines = "pd_profile_goroutines" ) var tableIDMap = map[string]int64{ - tableNameGlobalStatus: autoid.PerformanceSchemaDBID + 1, - tableNameSessionStatus: autoid.PerformanceSchemaDBID + 2, - tableNameSetupActors: autoid.PerformanceSchemaDBID + 3, - tableNameSetupObjects: autoid.PerformanceSchemaDBID + 4, - tableNameSetupInstruments: autoid.PerformanceSchemaDBID + 5, - tableNameSetupConsumers: autoid.PerformanceSchemaDBID + 6, - tableNameEventsStatementsCurrent: autoid.PerformanceSchemaDBID + 7, - tableNameEventsStatementsHistory: autoid.PerformanceSchemaDBID + 8, - tableNameEventsStatementsHistoryLong: autoid.PerformanceSchemaDBID + 9, - tableNamePreparedStatementsInstances: autoid.PerformanceSchemaDBID + 10, - tableNameEventsTransactionsCurrent: autoid.PerformanceSchemaDBID + 11, - tableNameEventsTransactionsHistory: autoid.PerformanceSchemaDBID + 12, - tableNameEventsTransactionsHistoryLong: autoid.PerformanceSchemaDBID + 13, - tableNameEventsStagesCurrent: autoid.PerformanceSchemaDBID + 14, - tableNameEventsStagesHistory: autoid.PerformanceSchemaDBID + 15, - tableNameEventsStagesHistoryLong: autoid.PerformanceSchemaDBID + 16, - tableNameEventsStatementsSummaryByDigest: autoid.PerformanceSchemaDBID + 17, - tableNameTiDBProfileCPU: autoid.PerformanceSchemaDBID + 18, - tableNameTiDBProfileMemory: autoid.PerformanceSchemaDBID + 19, - tableNameTiDBProfileMutex: autoid.PerformanceSchemaDBID + 20, - tableNameTiDBProfileAllocs: autoid.PerformanceSchemaDBID + 21, - tableNameTiDBProfileBlock: autoid.PerformanceSchemaDBID + 22, - tableNameTiDBProfileGoroutines: autoid.PerformanceSchemaDBID + 23, - tableNameTiKVProfileCPU: autoid.PerformanceSchemaDBID + 24, - tableNamePDProfileCPU: autoid.PerformanceSchemaDBID + 25, - tableNamePDProfileMemory: autoid.PerformanceSchemaDBID + 26, - tableNamePDProfileMutex: autoid.PerformanceSchemaDBID + 27, - tableNamePDProfileAllocs: autoid.PerformanceSchemaDBID + 28, - tableNamePDProfileBlock: autoid.PerformanceSchemaDBID + 29, - tableNamePDProfileGoroutines: autoid.PerformanceSchemaDBID + 30, - tableNameEventsStatementsSummaryByDigestHistory: autoid.PerformanceSchemaDBID + 31, - tableNameClusterEventsStatementsSummaryByDigest: autoid.PerformanceSchemaDBID + 32, - tableNameClusterEventsStatementsSummaryByDigestHistory: autoid.PerformanceSchemaDBID + 33, + tableNameGlobalStatus: autoid.PerformanceSchemaDBID + 1, + tableNameSessionStatus: autoid.PerformanceSchemaDBID + 2, + tableNameSetupActors: autoid.PerformanceSchemaDBID + 3, + tableNameSetupObjects: autoid.PerformanceSchemaDBID + 4, + tableNameSetupInstruments: autoid.PerformanceSchemaDBID + 5, + tableNameSetupConsumers: autoid.PerformanceSchemaDBID + 6, + tableNameEventsStatementsCurrent: autoid.PerformanceSchemaDBID + 7, + tableNameEventsStatementsHistory: autoid.PerformanceSchemaDBID + 8, + tableNameEventsStatementsHistoryLong: autoid.PerformanceSchemaDBID + 9, + tableNamePreparedStatementsInstances: autoid.PerformanceSchemaDBID + 10, + tableNameEventsTransactionsCurrent: autoid.PerformanceSchemaDBID + 11, + tableNameEventsTransactionsHistory: autoid.PerformanceSchemaDBID + 12, + tableNameEventsTransactionsHistoryLong: autoid.PerformanceSchemaDBID + 13, + tableNameEventsStagesCurrent: autoid.PerformanceSchemaDBID + 14, + tableNameEventsStagesHistory: autoid.PerformanceSchemaDBID + 15, + tableNameEventsStagesHistoryLong: autoid.PerformanceSchemaDBID + 16, + tableNameEventsStatementsSummaryByDigest: autoid.PerformanceSchemaDBID + 17, + tableNameTiDBProfileCPU: autoid.PerformanceSchemaDBID + 18, + tableNameTiDBProfileMemory: autoid.PerformanceSchemaDBID + 19, + tableNameTiDBProfileMutex: autoid.PerformanceSchemaDBID + 20, + tableNameTiDBProfileAllocs: autoid.PerformanceSchemaDBID + 21, + tableNameTiDBProfileBlock: autoid.PerformanceSchemaDBID + 22, + tableNameTiDBProfileGoroutines: autoid.PerformanceSchemaDBID + 23, + tableNameTiKVProfileCPU: autoid.PerformanceSchemaDBID + 24, + tableNamePDProfileCPU: autoid.PerformanceSchemaDBID + 25, + tableNamePDProfileMemory: autoid.PerformanceSchemaDBID + 26, + tableNamePDProfileMutex: autoid.PerformanceSchemaDBID + 27, + tableNamePDProfileAllocs: autoid.PerformanceSchemaDBID + 28, + tableNamePDProfileBlock: autoid.PerformanceSchemaDBID + 29, + tableNamePDProfileGoroutines: autoid.PerformanceSchemaDBID + 30, } // perfSchemaTable stands for the fake table all its data is in the memory. @@ -149,10 +140,6 @@ func createPerfSchemaTable(meta *model.TableInfo) *perfSchemaTable { columns = append(columns, col) } tp := table.VirtualTable - switch meta.Name.L { - case tableNameClusterEventsStatementsSummaryByDigest, tableNameClusterEventsStatementsSummaryByDigestHistory: - tp = table.ClusterTable - } t := &perfSchemaTable{ meta: meta, cols: columns, @@ -202,19 +189,7 @@ func (vt *perfSchemaTable) Type() table.Type { } func (vt *perfSchemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) (fullRows [][]types.Datum, err error) { - // Extract user and privilege info (is super user?) here - // for statement summary tables' access privilege check - user := ctx.GetSessionVars().User - isSuper := false - if pm := privilege.GetPrivilegeManager(ctx); pm != nil { - isSuper = pm.RequestVerificationWithUser("", "", "", mysql.SuperPriv, user) - } - switch vt.meta.Name.O { - case tableNameEventsStatementsSummaryByDigest: - fullRows = stmtsummary.StmtSummaryByDigestMap.ToCurrentDatum(user, isSuper) - case tableNameEventsStatementsSummaryByDigestHistory: - fullRows = stmtsummary.StmtSummaryByDigestMap.ToHistoryDatum(user, isSuper) case tableNameTiDBProfileCPU: fullRows, err = (&profile.Collector{}).ProfileGraph("cpu") case tableNameTiDBProfileMemory: @@ -243,10 +218,6 @@ func (vt *perfSchemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) fullRows, err = dataForRemoteProfile(ctx, "pd", "/pd/api/v1/debug/pprof/block", false) case tableNamePDProfileGoroutines: fullRows, err = dataForRemoteProfile(ctx, "pd", "/pd/api/v1/debug/pprof/goroutine?debug=2", true) - // Data for cluster memory table. - case tableNameClusterEventsStatementsSummaryByDigest, tableNameClusterEventsStatementsSummaryByDigestHistory: - fullRows, err = getClusterMemTableRows(ctx, vt.meta.Name.L) - } if err != nil { return @@ -265,29 +236,6 @@ func (vt *perfSchemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) return rows, nil } -func getClusterMemTableRows(ctx sessionctx.Context, tableName string) (rows [][]types.Datum, err error) { - // Extract user and privilege info (is super user?) here - // for statement summary tables' access privilege check - user := ctx.GetSessionVars().User - isSuper := false - if pm := privilege.GetPrivilegeManager(ctx); pm != nil { - isSuper = pm.RequestVerificationWithUser("", "", "", mysql.SuperPriv, user) - } - - switch tableName { - case tableNameClusterEventsStatementsSummaryByDigest: - rows = stmtsummary.StmtSummaryByDigestMap.ToCurrentDatum(user, isSuper) - case tableNameClusterEventsStatementsSummaryByDigestHistory: - rows = stmtsummary.StmtSummaryByDigestMap.ToHistoryDatum(user, isSuper) - default: - err = errors.Errorf("unknown cluster table: %v", tableName) - } - if err != nil { - return nil, err - } - return infoschema.AppendHostInfoToRows(rows) -} - // IterRecords implements table.Table IterRecords interface. func (vt *perfSchemaTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols []*table.Column, fn table.RecordIterFunc) error { @@ -317,7 +265,7 @@ func dataForRemoteProfile(ctx sessionctx.Context, nodeType, uri string, isGorout ) switch nodeType { case "tikv": - servers, err = infoschema.GetTiKVServerInfo(ctx) + servers, err = infoschema.GetStoreServerInfo(ctx) case "pd": servers, err = infoschema.GetPDServerInfo(ctx) default: diff --git a/infoschema/perfschema/tables_test.go b/infoschema/perfschema/tables_test.go index 0914bb4647e49..c960290072e62 100644 --- a/infoschema/perfschema/tables_test.go +++ b/infoschema/perfschema/tables_test.go @@ -27,7 +27,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/parser/auth" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema/perfschema" @@ -82,338 +81,6 @@ func (s *testTableSuite) TestPerfSchemaTables(c *C) { tk.MustQuery("select * from events_stages_history_long").Check(testkit.Rows()) } -// Test events_statements_summary_by_digest. -func (s *testTableSuite) TestStmtSummaryTable(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - - tk.MustQuery("select column_comment from information_schema.columns " + - "where table_name='events_statements_summary_by_digest' and column_name='STMT_TYPE'", - ).Check(testkit.Rows("Statement type")) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b varchar(10), key k(a))") - - // Clear all statements. - tk.MustExec("set session tidb_enable_stmt_summary = 0") - tk.MustExec("set session tidb_enable_stmt_summary = ''") - - tk.MustExec("set global tidb_enable_stmt_summary = 1") - defer tk.MustExec("set global tidb_enable_stmt_summary = ''") - tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) - - // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. - s.dom.GetGlobalVarsCache().Disable() - // Disable refreshing summary. - tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") - tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) - - // Create a new session to test. - tk = testkit.NewTestKitWithInit(c, s.store) - - // Test INSERT - tk.MustExec("insert into t values(1, 'a')") - tk.MustExec("insert into t values(2, 'b')") - tk.MustExec("insert into t VALUES(3, 'c')") - tk.MustExec("/**/insert into t values(4, 'd')") - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, plan - from performance_schema.events_statements_summary_by_digest - where digest_text like 'insert into t%'`, - ).Check(testkit.Rows("insert test test.t 4 0 0 0 0 0 2 2 1 1 1 insert into t values(1, 'a') ")) - - // Test point get. - tk.MustExec("drop table if exists p") - tk.MustExec("create table p(a int primary key, b int)") - for i := 1; i < 3; i++ { - tk.MustQuery("select b from p where a=1") - expectedResult := fmt.Sprintf("%d \tPoint_Get_1\troot\t1\ttable:p, handle:1 %s", i, "test.p") - // Also make sure that the plan digest is not empty - tk.MustQuery(`select exec_count, plan, table_names - from performance_schema.events_statements_summary_by_digest - where digest_text like 'select b from p%' and plan_digest != ''`, - ).Check(testkit.Rows(expectedResult)) - } - - // Point get another database. - tk.MustQuery("select variable_value from mysql.tidb where variable_name = 'system_tz'") - tk.MustQuery(`select table_names - from performance_schema.events_statements_summary_by_digest - where digest_text like 'select variable_value%' and schema_name='test'`, - ).Check(testkit.Rows("mysql.tidb")) - - // Test `create database`. - tk.MustExec("create database if not exists test") - tk.MustQuery(`select table_names - from performance_schema.events_statements_summary_by_digest - where digest_text like 'create database%' and schema_name='test'`, - ).Check(testkit.Rows("")) - - // Test SELECT. - const failpointName = "github.com/pingcap/tidb/planner/core/mockPlanRowCount" - c.Assert(failpoint.Enable(failpointName, "return(100)"), IsNil) - defer func() { c.Assert(failpoint.Disable(failpointName), IsNil) }() - tk.MustQuery("select * from t where a=2") - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, plan - from performance_schema.events_statements_summary_by_digest - where digest_text like 'select * from t%'`, - ).Check(testkit.Rows("select test test.t t:k 1 2 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tIndexLookUp_10\troot\t100\t\n" + - "\t├─IndexScan_8 \tcop \t100\ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + - "\t└─TableScan_9 \tcop \t100\ttable:t, keep order:false, stats:pseudo")) - - // select ... order by - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, plan - from performance_schema.events_statements_summary_by_digest - order by exec_count desc limit 1`, - ).Check(testkit.Rows("insert test test.t 4 0 0 0 0 0 2 2 1 1 1 insert into t values(1, 'a') ")) - - // Test different plans with same digest. - c.Assert(failpoint.Enable(failpointName, "return(1000)"), IsNil) - tk.MustQuery("select * from t where a=3") - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, plan - from performance_schema.events_statements_summary_by_digest - where digest_text like 'select * from t%'`, - ).Check(testkit.Rows("select test test.t t:k 2 4 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tIndexLookUp_10\troot\t100\t\n" + - "\t├─IndexScan_8 \tcop \t100\ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + - "\t└─TableScan_9 \tcop \t100\ttable:t, keep order:false, stats:pseudo")) - - // Disable it again. - tk.MustExec("set global tidb_enable_stmt_summary = false") - tk.MustExec("set session tidb_enable_stmt_summary = false") - defer tk.MustExec("set global tidb_enable_stmt_summary = ''") - defer tk.MustExec("set session tidb_enable_stmt_summary = ''") - tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("0")) - - // Create a new session to test - tk = testkit.NewTestKitWithInit(c, s.store) - - // This statement shouldn't be summarized. - tk.MustQuery("select * from t where a=2") - - // The table should be cleared. - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, plan - from performance_schema.events_statements_summary_by_digest`, - ).Check(testkit.Rows()) - - // Enable it in session scope. - tk.MustExec("set session tidb_enable_stmt_summary = on") - // It should work immediately. - tk.MustExec("begin") - tk.MustExec("insert into t values(1, 'a')") - tk.MustExec("commit") - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, prev_sample_text, plan - from performance_schema.events_statements_summary_by_digest - where digest_text like 'insert into t%'`, - ).Check(testkit.Rows("insert test test.t 1 0 0 0 0 0 0 0 0 0 1 insert into t values(1, 'a') ")) - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, prev_sample_text, plan - from performance_schema.events_statements_summary_by_digest - where digest_text='commit'`, - ).Check(testkit.Rows("commit test 1 0 0 0 0 0 2 2 1 1 0 commit insert into t values(1, 'a') ")) - - tk.MustQuery("select * from t where a=2") - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, plan - from performance_schema.events_statements_summary_by_digest - where digest_text like 'select * from t%'`, - ).Check(testkit.Rows("select test test.t t:k 1 2 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tIndexLookUp_10\troot\t1000\t\n" + - "\t├─IndexScan_8 \tcop \t1000\ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + - "\t└─TableScan_9 \tcop \t1000\ttable:t, keep order:false, stats:pseudo")) - - // Disable it in global scope. - tk.MustExec("set global tidb_enable_stmt_summary = false") - - // Create a new session to test. - tk = testkit.NewTestKitWithInit(c, s.store) - - tk.MustQuery("select * from t where a=2") - - // Statement summary is still enabled. - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, plan - from performance_schema.events_statements_summary_by_digest - where digest_text like 'select * from t%'`, - ).Check(testkit.Rows("select test test.t t:k 2 4 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tIndexLookUp_10\troot\t1000\t\n" + - "\t├─IndexScan_8 \tcop \t1000\ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + - "\t└─TableScan_9 \tcop \t1000\ttable:t, keep order:false, stats:pseudo")) - - // Unset session variable. - tk.MustExec("set session tidb_enable_stmt_summary = ''") - tk.MustQuery("select * from t where a=2") - - // Statement summary is disabled. - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, plan - from performance_schema.events_statements_summary_by_digest`, - ).Check(testkit.Rows()) - - // Create a new session to test - tk = testkit.NewTestKitWithInit(c, s.store) - - tk.MustExec("set global tidb_enable_stmt_summary = on") - tk.MustExec("set global tidb_stmt_summary_history_size = 24") - - // Create a new user to test statements summary table privilege - tk.MustExec("create user 'test_user'@'localhost'") - tk.MustExec("grant select on *.* to 'test_user'@'localhost'") - tk.Se.Auth(&auth.UserIdentity{ - Username: "root", - Hostname: "%", - AuthUsername: "root", - AuthHostname: "%", - }, nil, nil) - tk.MustExec("select * from t where a=1") - result := tk.MustQuery(`select * - from performance_schema.events_statements_summary_by_digest - where digest_text like 'select * from t%'`, - ) - // Super user can query all reocrds - c.Assert(len(result.Rows()), Equals, 1) - result = tk.MustQuery(`select * - from performance_schema.events_statements_summary_by_digest_history - where digest_text like 'select * from t%'`, - ) - c.Assert(len(result.Rows()), Equals, 1) - tk.Se.Auth(&auth.UserIdentity{ - Username: "test_user", - Hostname: "localhost", - AuthUsername: "test_user", - AuthHostname: "localhost", - }, nil, nil) - result = tk.MustQuery(`select * - from performance_schema.events_statements_summary_by_digest - where digest_text like 'select * from t%'`, - ) - // Ordinary users can not see others' records - c.Assert(len(result.Rows()), Equals, 0) - result = tk.MustQuery(`select * - from performance_schema.events_statements_summary_by_digest_history - where digest_text like 'select * from t%'`, - ) - c.Assert(len(result.Rows()), Equals, 0) - tk.MustExec("select * from t where a=1") - result = tk.MustQuery(`select * - from performance_schema.events_statements_summary_by_digest - where digest_text like 'select * from t%'`, - ) - c.Assert(len(result.Rows()), Equals, 1) - tk.MustExec("select * from t where a=1") - result = tk.MustQuery(`select * - from performance_schema.events_statements_summary_by_digest_history - where digest_text like 'select * from t%'`, - ) - c.Assert(len(result.Rows()), Equals, 1) - // use root user to set variables back - tk.Se.Auth(&auth.UserIdentity{ - Username: "root", - Hostname: "%", - AuthUsername: "root", - AuthHostname: "%", - }, nil, nil) - tk.MustExec("set global tidb_enable_stmt_summary = off") -} - -// Test events_statements_summary_by_digest_history. -func (s *testTableSuite) TestStmtSummaryHistoryTable(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b varchar(10), key k(a))") - - tk.MustExec("set global tidb_enable_stmt_summary = 1") - tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) - defer tk.MustExec("set global tidb_enable_stmt_summary = ''") - - // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. - s.dom.GetGlobalVarsCache().Disable() - // Disable refreshing summary. - tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") - tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) - - // Create a new session to test. - tk = testkit.NewTestKitWithInit(c, s.store) - - // Test INSERT - tk.MustExec("insert into t values(1, 'a')") - tk.MustExec("insert into t values(2, 'b')") - tk.MustExec("insert into t VALUES(3, 'c')") - tk.MustExec("/**/insert into t values(4, 'd')") - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, plan - from performance_schema.events_statements_summary_by_digest_history - where digest_text like 'insert into t%'`, - ).Check(testkit.Rows("insert test test.t 4 0 0 0 0 0 2 2 1 1 1 insert into t values(1, 'a') ")) - - tk.MustExec("set global tidb_stmt_summary_history_size = 0") - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, cop_task_num, avg_total_keys, - max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, - max_prewrite_regions, avg_affected_rows, query_sample_text, plan - from performance_schema.events_statements_summary_by_digest_history`, - ).Check(testkit.Rows()) -} - -// Test events_statements_summary_by_digest_history. -func (s *testTableSuite) TestStmtSummaryInternalQuery(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b varchar(10), key k(a))") - - // We use the sql binding evolve to check the internal query summary. - tk.MustExec("set @@tidb_use_plan_baselines = 1") - tk.MustExec("set @@tidb_evolve_plan_baselines = 1") - tk.MustExec("create global binding for select * from t where t.a = 1 using select * from t ignore index(k) where t.a = 1") - tk.MustExec("set global tidb_enable_stmt_summary = 1") - tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) - defer tk.MustExec("set global tidb_enable_stmt_summary = ''") - // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. - s.dom.GetGlobalVarsCache().Disable() - // Disable refreshing summary. - tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") - tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) - - // Test Internal - - // Create a new session to test. - tk = testkit.NewTestKitWithInit(c, s.store) - - tk.MustExec("select * from t where t.a = 1") - tk.MustQuery(`select exec_count, digest_text - from performance_schema.events_statements_summary_by_digest - where digest_text like "select original_sql , bind_sql , default_db , status%"`).Check(testkit.Rows()) - - // Enable internal query and evolve baseline. - tk.MustExec("set global tidb_stmt_summary_internal_query = 1") - defer tk.MustExec("set global tidb_stmt_summary_internal_query = false") - - // Create a new session to test. - tk = testkit.NewTestKitWithInit(c, s.store) - - tk.MustExec("admin flush bindings") - tk.MustExec("admin evolve bindings") - - tk.MustQuery(`select exec_count, digest_text - from performance_schema.events_statements_summary_by_digest - where digest_text like "select original_sql , bind_sql , default_db , status%"`).Check(testkit.Rows( - "1 select original_sql , bind_sql , default_db , status , create_time , update_time , charset , collation from mysql . bind_info" + - " where update_time > ? order by update_time")) -} - func currentSourceDir() string { _, file, _, _ := runtime.Caller(0) return filepath.Dir(file) diff --git a/infoschema/tables.go b/infoschema/tables.go index 51e544d6d5300..2419feb8a6937 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -140,6 +141,10 @@ const ( TableDDLJobs = "DDL_JOBS" // TableSequences is the string constant of all sequences created by user. TableSequences = "SEQUENCES" + // TableStatementsSummary is the string constant of statement summary table. + TableStatementsSummary = "STATEMENTS_SUMMARY" + // TableStatementsSummaryHistory is the string constant of statements summary history table. + TableStatementsSummaryHistory = "STATEMENTS_SUMMARY_HISTORY" ) var tableIDMap = map[string]int64{ @@ -201,6 +206,10 @@ var tableIDMap = map[string]int64{ TableInspectionRules: autoid.InformationSchemaDBID + 56, TableDDLJobs: autoid.InformationSchemaDBID + 57, TableSequences: autoid.InformationSchemaDBID + 58, + TableStatementsSummary: autoid.InformationSchemaDBID + 59, + TableStatementsSummaryHistory: autoid.InformationSchemaDBID + 60, + ClusterTableStatementsSummary: autoid.InformationSchemaDBID + 61, + ClusterTableStatementsSummaryHistory: autoid.InformationSchemaDBID + 62, } type columnInfo struct { @@ -210,6 +219,7 @@ type columnInfo struct { decimal int flag uint deflt interface{} + comment string } func buildColumnInfo(col columnInfo) *model.ColumnInfo { @@ -232,6 +242,7 @@ func buildColumnInfo(col columnInfo) *model.ColumnInfo { FieldType: fieldType, State: model.StatePublic, DefaultValue: col.deflt, + Comment: col.comment, } } @@ -692,7 +703,7 @@ var tableTiDBIndexesCols = []columnInfo{ } var slowQueryCols = []columnInfo{ - {name: variable.SlowLogTimeStr, tp: mysql.TypeTimestamp, size: 26}, + {name: variable.SlowLogTimeStr, tp: mysql.TypeTimestamp, size: 26, decimal: 6}, {name: variable.SlowLogTxnStartTSStr, tp: mysql.TypeLonglong, size: 20, flag: mysql.UnsignedFlag}, {name: variable.SlowLogUserStr, tp: mysql.TypeVarchar, size: 64}, {name: variable.SlowLogHostStr, tp: mysql.TypeVarchar, size: 64}, @@ -712,6 +723,7 @@ var slowQueryCols = []columnInfo{ {name: execdetails.WriteSizeStr, tp: mysql.TypeLonglong, size: 22}, {name: execdetails.PrewriteRegionStr, tp: mysql.TypeLonglong, size: 22}, {name: execdetails.TxnRetryStr, tp: mysql.TypeLonglong, size: 22}, + {name: execdetails.CopTimeStr, tp: mysql.TypeDouble, size: 22}, {name: execdetails.ProcessTimeStr, tp: mysql.TypeDouble, size: 22}, {name: execdetails.WaitTimeStr, tp: mysql.TypeDouble, size: 22}, {name: execdetails.BackoffTimeStr, tp: mysql.TypeDouble, size: 22}, @@ -804,6 +816,8 @@ var TableTiKVRegionStatusCols = []columnInfo{ {name: "READ_BYTES", tp: mysql.TypeLonglong, size: 21}, {name: "APPROXIMATE_SIZE", tp: mysql.TypeLonglong, size: 21}, {name: "APPROXIMATE_KEYS", tp: mysql.TypeLonglong, size: 21}, + {name: "REPLICATIONSTATUS_STATE", tp: mysql.TypeVarchar, size: 64}, + {name: "REPLICATIONSTATUS_STATEID", tp: mysql.TypeLonglong, size: 21}, } // TableTiKVRegionPeersCols is TiKV region peers mem table columns. @@ -952,6 +966,7 @@ var tableInspectionSummaryCols = []columnInfo{ {name: "AVG_VALUE", tp: mysql.TypeDouble, size: 22, decimal: 6}, {name: "MIN_VALUE", tp: mysql.TypeDouble, size: 22, decimal: 6}, {name: "MAX_VALUE", tp: mysql.TypeDouble, size: 22, decimal: 6}, + {name: "COMMENT", tp: mysql.TypeVarchar, size: 256}, } var tableInspectionRulesCols = []columnInfo{ @@ -1019,6 +1034,75 @@ var tableSequencesCols = []columnInfo{ {name: "COMMENT", tp: mysql.TypeVarchar, size: 64}, } +var tableStatementsSummaryCols = []columnInfo{ + {name: "SUMMARY_BEGIN_TIME", tp: mysql.TypeTimestamp, size: 26, flag: mysql.NotNullFlag, comment: "Begin time of this summary"}, + {name: "SUMMARY_END_TIME", tp: mysql.TypeTimestamp, size: 26, flag: mysql.NotNullFlag, comment: "End time of this summary"}, + {name: "STMT_TYPE", tp: mysql.TypeVarchar, size: 64, flag: mysql.NotNullFlag, comment: "Statement type"}, + {name: "SCHEMA_NAME", tp: mysql.TypeVarchar, size: 64, flag: mysql.NotNullFlag, comment: "Current schema"}, + {name: "DIGEST", tp: mysql.TypeVarchar, size: 64, flag: mysql.NotNullFlag}, + {name: "DIGEST_TEXT", tp: mysql.TypeBlob, size: types.UnspecifiedLength, flag: mysql.NotNullFlag, comment: "Normalized statement"}, + {name: "TABLE_NAMES", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Involved tables"}, + {name: "INDEX_NAMES", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Used indices"}, + {name: "SAMPLE_USER", tp: mysql.TypeVarchar, size: 64, comment: "Sampled user who executed these statements"}, + {name: "EXEC_COUNT", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Count of executions"}, + {name: "SUM_ERRORS", tp: mysql.TypeLong, size: 11, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Sum of errors"}, + {name: "SUM_WARNINGS", tp: mysql.TypeLong, size: 11, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Sum of warnings"}, + {name: "SUM_LATENCY", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Sum latency of these statements"}, + {name: "MAX_LATENCY", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max latency of these statements"}, + {name: "MIN_LATENCY", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Min latency of these statements"}, + {name: "AVG_LATENCY", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average latency of these statements"}, + {name: "AVG_PARSE_LATENCY", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average latency of parsing"}, + {name: "MAX_PARSE_LATENCY", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max latency of parsing"}, + {name: "AVG_COMPILE_LATENCY", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average latency of compiling"}, + {name: "MAX_COMPILE_LATENCY", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max latency of compiling"}, + {name: "SUM_COP_TASK_NUM", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Total number of CopTasks"}, + {name: "MAX_COP_PROCESS_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max processing time of CopTasks"}, + {name: "MAX_COP_PROCESS_ADDRESS", tp: mysql.TypeVarchar, size: 256, comment: "Address of the CopTask with max processing time"}, + {name: "MAX_COP_WAIT_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max waiting time of CopTasks"}, + {name: "MAX_COP_WAIT_ADDRESS", tp: mysql.TypeVarchar, size: 256, comment: "Address of the CopTask with max waiting time"}, + {name: "AVG_PROCESS_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average processing time in TiKV"}, + {name: "MAX_PROCESS_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max processing time in TiKV"}, + {name: "AVG_WAIT_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average waiting time in TiKV"}, + {name: "MAX_WAIT_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max waiting time in TiKV"}, + {name: "AVG_BACKOFF_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average waiting time before retry"}, + {name: "MAX_BACKOFF_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max waiting time before retry"}, + {name: "AVG_TOTAL_KEYS", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average number of scanned keys"}, + {name: "MAX_TOTAL_KEYS", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max number of scanned keys"}, + {name: "AVG_PROCESSED_KEYS", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average number of processed keys"}, + {name: "MAX_PROCESSED_KEYS", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max number of processed keys"}, + {name: "AVG_PREWRITE_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average time of prewrite phase"}, + {name: "MAX_PREWRITE_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max time of prewrite phase"}, + {name: "AVG_COMMIT_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average time of commit phase"}, + {name: "MAX_COMMIT_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max time of commit phase"}, + {name: "AVG_GET_COMMIT_TS_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average time of getting commit_ts"}, + {name: "MAX_GET_COMMIT_TS_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max time of getting commit_ts"}, + {name: "AVG_COMMIT_BACKOFF_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average time before retry during commit phase"}, + {name: "MAX_COMMIT_BACKOFF_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max time before retry during commit phase"}, + {name: "AVG_RESOLVE_LOCK_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average time for resolving locks"}, + {name: "MAX_RESOLVE_LOCK_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max time for resolving locks"}, + {name: "AVG_LOCAL_LATCH_WAIT_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average waiting time of local transaction"}, + {name: "MAX_LOCAL_LATCH_WAIT_TIME", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max waiting time of local transaction"}, + {name: "AVG_WRITE_KEYS", tp: mysql.TypeDouble, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average count of written keys"}, + {name: "MAX_WRITE_KEYS", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max count of written keys"}, + {name: "AVG_WRITE_SIZE", tp: mysql.TypeDouble, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average amount of written bytes"}, + {name: "MAX_WRITE_SIZE", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max amount of written bytes"}, + {name: "AVG_PREWRITE_REGIONS", tp: mysql.TypeDouble, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average number of involved regions in prewrite phase"}, + {name: "MAX_PREWRITE_REGIONS", tp: mysql.TypeLong, size: 11, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max number of involved regions in prewrite phase"}, + {name: "AVG_TXN_RETRY", tp: mysql.TypeDouble, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average number of transaction retries"}, + {name: "MAX_TXN_RETRY", tp: mysql.TypeLong, size: 11, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max number of transaction retries"}, + {name: "SUM_BACKOFF_TIMES", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Sum of retries"}, + {name: "BACKOFF_TYPES", tp: mysql.TypeVarchar, size: 1024, comment: "Types of errors and the number of retries for each type"}, + {name: "AVG_MEM", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average memory(byte) used"}, + {name: "MAX_MEM", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Max memory(byte) used"}, + {name: "AVG_AFFECTED_ROWS", tp: mysql.TypeDouble, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average number of rows affected"}, + {name: "FIRST_SEEN", tp: mysql.TypeTimestamp, size: 26, flag: mysql.NotNullFlag, comment: "The time these statements are seen for the first time"}, + {name: "LAST_SEEN", tp: mysql.TypeTimestamp, size: 26, flag: mysql.NotNullFlag, comment: "The time these statements are seen for the last time"}, + {name: "QUERY_SAMPLE_TEXT", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Sampled original statement"}, + {name: "PREV_SAMPLE_TEXT", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "The previous statement before commit"}, + {name: "PLAN_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of its execution plan"}, + {name: "PLAN", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Sampled execution plan"}, +} + // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. // The returned description string may be: // - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. @@ -1086,7 +1170,7 @@ func GetClusterServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { type retriever func(ctx sessionctx.Context) ([]ServerInfo, error) var servers []ServerInfo - for _, r := range []retriever{GetTiDBServerInfo, GetPDServerInfo, GetTiKVServerInfo} { + for _, r := range []retriever{GetTiDBServerInfo, GetPDServerInfo, GetStoreServerInfo} { nodes, err := r(ctx) if err != nil { return nil, err @@ -1169,10 +1253,10 @@ func GetPDServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { return nil, errors.Trace(err) } pdVersion, err := ioutil.ReadAll(resp.Body) + terror.Log(resp.Body.Close()) if err != nil { return nil, errors.Trace(err) } - terror.Log(resp.Body.Close()) version := strings.Trim(strings.Trim(string(pdVersion), "\n"), "\"") // Get PD git_hash @@ -1190,10 +1274,11 @@ func GetPDServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { GitHash string `json:"git_hash"` StartTimestamp int64 `json:"start_timestamp"` }{} - if err := json.NewDecoder(resp.Body).Decode(&content); err != nil { + err = json.NewDecoder(resp.Body).Decode(&content) + terror.Log(resp.Body.Close()) + if err != nil { return nil, errors.Trace(err) } - terror.Log(resp.Body.Close()) servers = append(servers, ServerInfo{ ServerType: "pd", @@ -1207,13 +1292,23 @@ func GetPDServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { return servers, nil } -// GetTiKVServerInfo returns all TiKV nodes information of cluster -func GetTiKVServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { +// GetStoreServerInfo returns all store nodes(TiKV or TiFlash) cluster information +func GetStoreServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { + isTiFlashStore := func(store *metapb.Store) bool { + isTiFlash := false + for _, label := range store.Labels { + if label.GetKey() == "engine" && label.GetValue() == "tiflash" { + isTiFlash = true + } + } + return isTiFlash + } + store := ctx.GetStore() // Get TiKV servers info. tikvStore, ok := store.(tikv.Storage) if !ok { - return nil, errors.Errorf("%T is not an TiKV store instance", store) + return nil, errors.Errorf("%T is not an TiKV or TiFlash store instance", store) } pdClient := tikvStore.GetRegionCache().PDClient() if pdClient == nil { @@ -1225,7 +1320,12 @@ func GetTiKVServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { } var servers []ServerInfo for _, store := range stores { - tp := tikv.GetStoreTypeByMeta(store).Name() + var tp string + if isTiFlashStore(store) { + tp = "tiflash" + } else { + tp = tikv.GetStoreTypeByMeta(store).Name() + } servers = append(servers, ServerInfo{ ServerType: tp, Address: store.Address, @@ -1294,6 +1394,8 @@ var tableNameToColumns = map[string][]columnInfo{ TableInspectionRules: tableInspectionRulesCols, TableDDLJobs: tableDDLJobsCols, TableSequences: tableSequencesCols, + TableStatementsSummary: tableStatementsSummaryCols, + TableStatementsSummaryHistory: tableStatementsSummaryCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { @@ -1390,12 +1492,12 @@ func (it *infoschemaTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, } // RowWithCols implements table.Table RowWithCols interface. -func (it *infoschemaTable) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Column) ([]types.Datum, error) { +func (it *infoschemaTable) RowWithCols(ctx sessionctx.Context, h kv.Handle, cols []*table.Column) ([]types.Datum, error) { return nil, table.ErrUnsupportedOp } // Row implements table.Table Row interface. -func (it *infoschemaTable) Row(ctx sessionctx.Context, h int64) ([]types.Datum, error) { +func (it *infoschemaTable) Row(ctx sessionctx.Context, h kv.Handle) ([]types.Datum, error) { return nil, table.ErrUnsupportedOp } @@ -1455,22 +1557,22 @@ func (it *infoschemaTable) FirstKey() kv.Key { } // RecordKey implements table.Table RecordKey interface. -func (it *infoschemaTable) RecordKey(h int64) kv.Key { +func (it *infoschemaTable) RecordKey(h kv.Handle) kv.Key { return nil } // AddRecord implements table.Table AddRecord interface. -func (it *infoschemaTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID int64, err error) { - return 0, table.ErrUnsupportedOp +func (it *infoschemaTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) { + return nil, table.ErrUnsupportedOp } // RemoveRecord implements table.Table RemoveRecord interface. -func (it *infoschemaTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { +func (it *infoschemaTable) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []types.Datum) error { return table.ErrUnsupportedOp } // UpdateRecord implements table.Table UpdateRecord interface. -func (it *infoschemaTable) UpdateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, touched []bool) error { +func (it *infoschemaTable) UpdateRecord(ctx sessionctx.Context, h kv.Handle, oldData, newData []types.Datum, touched []bool) error { return table.ErrUnsupportedOp } @@ -1480,7 +1582,7 @@ func (it *infoschemaTable) Allocators(_ sessionctx.Context) autoid.Allocators { } // RebaseAutoID implements table.Table RebaseAutoID interface. -func (it *infoschemaTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { +func (it *infoschemaTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool, tp autoid.AllocatorType) error { return table.ErrUnsupportedOp } @@ -1495,8 +1597,8 @@ func (it *infoschemaTable) GetPhysicalID() int64 { } // Seek implements table.Table Seek interface. -func (it *infoschemaTable) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) { - return 0, false, table.ErrUnsupportedOp +func (it *infoschemaTable) Seek(ctx sessionctx.Context, h kv.Handle) (kv.Handle, bool, error) { + return nil, false, table.ErrUnsupportedOp } // Type implements table.Table Type interface. @@ -1517,12 +1619,12 @@ func (vt *VirtualTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, col } // RowWithCols implements table.Table RowWithCols interface. -func (vt *VirtualTable) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Column) ([]types.Datum, error) { +func (vt *VirtualTable) RowWithCols(ctx sessionctx.Context, h kv.Handle, cols []*table.Column) ([]types.Datum, error) { return nil, table.ErrUnsupportedOp } // Row implements table.Table Row interface. -func (vt *VirtualTable) Row(ctx sessionctx.Context, h int64) ([]types.Datum, error) { +func (vt *VirtualTable) Row(ctx sessionctx.Context, h kv.Handle) ([]types.Datum, error) { return nil, table.ErrUnsupportedOp } @@ -1582,22 +1684,22 @@ func (vt *VirtualTable) FirstKey() kv.Key { } // RecordKey implements table.Table RecordKey interface. -func (vt *VirtualTable) RecordKey(h int64) kv.Key { +func (vt *VirtualTable) RecordKey(h kv.Handle) kv.Key { return nil } // AddRecord implements table.Table AddRecord interface. -func (vt *VirtualTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID int64, err error) { - return 0, table.ErrUnsupportedOp +func (vt *VirtualTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) { + return nil, table.ErrUnsupportedOp } // RemoveRecord implements table.Table RemoveRecord interface. -func (vt *VirtualTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { +func (vt *VirtualTable) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []types.Datum) error { return table.ErrUnsupportedOp } // UpdateRecord implements table.Table UpdateRecord interface. -func (vt *VirtualTable) UpdateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, touched []bool) error { +func (vt *VirtualTable) UpdateRecord(ctx sessionctx.Context, h kv.Handle, oldData, newData []types.Datum, touched []bool) error { return table.ErrUnsupportedOp } @@ -1607,7 +1709,7 @@ func (vt *VirtualTable) Allocators(_ sessionctx.Context) autoid.Allocators { } // RebaseAutoID implements table.Table RebaseAutoID interface. -func (vt *VirtualTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { +func (vt *VirtualTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool, tp autoid.AllocatorType) error { return table.ErrUnsupportedOp } @@ -1622,8 +1724,8 @@ func (vt *VirtualTable) GetPhysicalID() int64 { } // Seek implements table.Table Seek interface. -func (vt *VirtualTable) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) { - return 0, false, table.ErrUnsupportedOp +func (vt *VirtualTable) Seek(ctx sessionctx.Context, h kv.Handle) (kv.Handle, bool, error) { + return nil, false, table.ErrUnsupportedOp } // Type implements table.Table Type interface. diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index f5f77c4878808..38b7671bae595 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -273,12 +273,12 @@ func (s *testTableSuite) TestInfoschemaFieldValue(c *C) { tk.MustQuery("show create table information_schema.PROCESSLIST").Check( testkit.Rows("" + "PROCESSLIST CREATE TABLE `PROCESSLIST` (\n" + - " `ID` bigint(21) unsigned NOT NULL DEFAULT '0',\n" + + " `ID` bigint(21) unsigned NOT NULL DEFAULT 0,\n" + " `USER` varchar(16) NOT NULL DEFAULT '',\n" + " `HOST` varchar(64) NOT NULL DEFAULT '',\n" + " `DB` varchar(64) DEFAULT NULL,\n" + " `COMMAND` varchar(16) NOT NULL DEFAULT '',\n" + - " `TIME` int(7) NOT NULL DEFAULT '0',\n" + + " `TIME` int(7) NOT NULL DEFAULT 0,\n" + " `STATE` varchar(7) DEFAULT NULL,\n" + " `INFO` binary(512) DEFAULT NULL,\n" + " `MEM` bigint(21) unsigned DEFAULT NULL,\n" + @@ -516,7 +516,7 @@ func prepareSlowLogfile(c *C, slowLogFileName string) { # Parse_time: 0.4 # Compile_time: 0.2 # LockKeys_time: 1.71 Request_count: 1 Prewrite_time: 0.19 Wait_prewrite_binlog_time: 0.21 Commit_time: 0.01 Commit_backoff_time: 0.18 Backoff_types: [txnLock] Resolve_lock_time: 0.03 Write_keys: 15 Write_size: 480 Prewrite_region: 1 Txn_retry: 8 -# Process_time: 0.161 Request_count: 1 Total_keys: 100001 Process_keys: 100000 +# Cop_time: 0.3824278 Process_time: 0.161 Request_count: 1 Total_keys: 100001 Process_keys: 100000 # Wait_time: 0.101 # Backoff_time: 0.092 # DB: test @@ -597,10 +597,10 @@ func (s *testTableSuite) TestSlowQuery(c *C) { tk.MustExec("set time_zone = '+08:00';") re := tk.MustQuery("select * from information_schema.slow_query") re.Check(testutil.RowsWithSep("|", - "2019-02-12 19:33:56.571953|406315658548871171|root|127.0.0.1|6|4.895492|0.4|0.2|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;")) + "2019-02-12 19:33:56.571953|406315658548871171|root|127.0.0.1|6|4.895492|0.4|0.2|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;")) tk.MustExec("set time_zone = '+00:00';") re = tk.MustQuery("select * from information_schema.slow_query") - re.Check(testutil.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root|127.0.0.1|6|4.895492|0.4|0.2|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;")) + re.Check(testutil.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root|127.0.0.1|6|4.895492|0.4|0.2|0.19|0.21|0.01|0|0.18|[txnLock]|0.03|0|15|480|1|8|0.3824278|0.161|0.101|0.092|1.71|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|1|abcd|60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4|update t set i = 2;|select * from t_slim;")) // Test for long query. f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644) @@ -754,28 +754,28 @@ func (s *testClusterTableSuite) TestSelectClusterTable(c *C) { tk.MustExec(fmt.Sprintf("set @@tidb_enable_streaming=%d", i)) tk.MustExec("set @@global.tidb_enable_stmt_summary=1") tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("1")) + tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY` where time='2019-02-12 19:33:56.571953'").Check(testkit.Rows("1")) tk.MustQuery("select count(*) from `CLUSTER_PROCESSLIST`").Check(testkit.Rows("1")) tk.MustQuery("select * from `CLUSTER_PROCESSLIST`").Check(testkit.Rows(fmt.Sprintf(":10080 1 root 127.0.0.1 Query 9223372036 %s 0 ", ""))) tk.MustQuery("select query_time, conn_id from `CLUSTER_SLOW_QUERY` order by time limit 1").Check(testkit.Rows("4.895492 6")) tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY` group by digest").Check(testkit.Rows("1")) tk.MustQuery("select digest, count(*) from `CLUSTER_SLOW_QUERY` group by digest").Check(testkit.Rows("42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 1")) tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY` where time > now() group by digest").Check(testkit.Rows()) - tk.MustExec("use performance_schema") - re := tk.MustQuery("select * from `CLUSTER_events_statements_summary_by_digest`") + re := tk.MustQuery("select * from `CLUSTER_statements_summary`") c.Assert(re, NotNil) c.Assert(len(re.Rows()) > 0, IsTrue) // Test for TiDB issue 14915. - re = tk.MustQuery("select sum(exec_count*avg_mem) from cluster_events_statements_summary_by_digest_history group by schema_name,digest,digest_text;") + re = tk.MustQuery("select sum(exec_count*avg_mem) from cluster_statements_summary_history group by schema_name,digest,digest_text;") c.Assert(re, NotNil) c.Assert(len(re.Rows()) > 0, IsTrue) - tk.MustQuery("select * from `CLUSTER_events_statements_summary_by_digest_history`") + tk.MustQuery("select * from `CLUSTER_statements_summary_history`") c.Assert(re, NotNil) c.Assert(len(re.Rows()) > 0, IsTrue) tk.MustExec("set @@global.tidb_enable_stmt_summary=0") - re = tk.MustQuery("select * from `CLUSTER_events_statements_summary_by_digest`") + re = tk.MustQuery("select * from `CLUSTER_statements_summary`") c.Assert(re, NotNil) c.Assert(len(re.Rows()) == 0, IsTrue) - tk.MustQuery("select * from `CLUSTER_events_statements_summary_by_digest_history`") + tk.MustQuery("select * from `CLUSTER_statements_summary_history`") c.Assert(re, NotNil) c.Assert(len(re.Rows()) == 0, IsTrue) } @@ -870,3 +870,356 @@ func (s *testTableSuite) TestFormatVersion(c *C) { c.Assert(version, Equals, res[i]) } } + +// Test statements_summary. +func (s *testTableSuite) TestStmtSummaryTable(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + + tk.MustQuery("select column_comment from information_schema.columns " + + "where table_name='STATEMENTS_SUMMARY' and column_name='STMT_TYPE'", + ).Check(testkit.Rows("Statement type")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10), key k(a))") + + // Clear all statements. + tk.MustExec("set session tidb_enable_stmt_summary = 0") + tk.MustExec("set session tidb_enable_stmt_summary = ''") + + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) + + // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. + s.dom.GetGlobalVarsCache().Disable() + // Disable refreshing summary. + tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") + tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) + + // Create a new session to test. + tk = testkit.NewTestKitWithInit(c, s.store) + + // Test INSERT + tk.MustExec("insert into t values(1, 'a')") + tk.MustExec("insert into t values(2, 'b')") + tk.MustExec("insert into t VALUES(3, 'c')") + tk.MustExec("/**/insert into t values(4, 'd')") + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary + where digest_text like 'insert into t%'`, + ).Check(testkit.Rows("insert test test.t 4 0 0 0 0 0 2 2 1 1 1 insert into t values(1, 'a') ")) + + // Test point get. + tk.MustExec("drop table if exists p") + tk.MustExec("create table p(a int primary key, b int)") + for i := 1; i < 3; i++ { + tk.MustQuery("select b from p where a=1") + expectedResult := fmt.Sprintf("%d \tPoint_Get_1\troot\t1\ttable:p, handle:1 %s", i, "test.p") + // Also make sure that the plan digest is not empty + tk.MustQuery(`select exec_count, plan, table_names + from information_schema.statements_summary + where digest_text like 'select b from p%' and plan_digest != ''`, + ).Check(testkit.Rows(expectedResult)) + } + + // Point get another database. + tk.MustQuery("select variable_value from mysql.tidb where variable_name = 'system_tz'") + tk.MustQuery(`select table_names + from information_schema.statements_summary + where digest_text like 'select variable_value%' and schema_name='test'`, + ).Check(testkit.Rows("mysql.tidb")) + + // Test `create database`. + tk.MustExec("create database if not exists test") + tk.MustQuery(`select table_names + from information_schema.statements_summary + where digest_text like 'create database%' and schema_name='test'`, + ).Check(testkit.Rows("")) + + // Test SELECT. + const failpointName = "github.com/pingcap/tidb/planner/core/mockPlanRowCount" + c.Assert(failpoint.Enable(failpointName, "return(100)"), IsNil) + defer func() { c.Assert(failpoint.Disable(failpointName), IsNil) }() + tk.MustQuery("select * from t where a=2") + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary + where digest_text like 'select * from t%'`, + ).Check(testkit.Rows("select test test.t t:k 1 2 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tIndexLookUp_10\troot\t100\t\n" + + "\t├─IndexScan_8 \tcop \t100\ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableScan_9 \tcop \t100\ttable:t, keep order:false, stats:pseudo")) + + // select ... order by + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary + order by exec_count desc limit 1`, + ).Check(testkit.Rows("insert test test.t 4 0 0 0 0 0 2 2 1 1 1 insert into t values(1, 'a') ")) + + // Test different plans with same digest. + c.Assert(failpoint.Enable(failpointName, "return(1000)"), IsNil) + tk.MustQuery("select * from t where a=3") + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary + where digest_text like 'select * from t%'`, + ).Check(testkit.Rows("select test test.t t:k 2 4 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tIndexLookUp_10\troot\t100\t\n" + + "\t├─IndexScan_8 \tcop \t100\ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableScan_9 \tcop \t100\ttable:t, keep order:false, stats:pseudo")) + + // Disable it again. + tk.MustExec("set global tidb_enable_stmt_summary = false") + tk.MustExec("set session tidb_enable_stmt_summary = false") + defer tk.MustExec("set global tidb_enable_stmt_summary = 1") + defer tk.MustExec("set session tidb_enable_stmt_summary = ''") + tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("0")) + + // Create a new session to test + tk = testkit.NewTestKitWithInit(c, s.store) + + // This statement shouldn't be summarized. + tk.MustQuery("select * from t where a=2") + + // The table should be cleared. + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary`, + ).Check(testkit.Rows()) + + // Enable it in session scope. + tk.MustExec("set session tidb_enable_stmt_summary = on") + // It should work immediately. + tk.MustExec("begin") + tk.MustExec("insert into t values(1, 'a')") + tk.MustExec("commit") + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, prev_sample_text, plan + from information_schema.statements_summary + where digest_text like 'insert into t%'`, + ).Check(testkit.Rows("insert test test.t 1 0 0 0 0 0 0 0 0 0 1 insert into t values(1, 'a') ")) + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, prev_sample_text, plan + from information_schema.statements_summary + where digest_text='commit'`, + ).Check(testkit.Rows("commit test 1 0 0 0 0 0 2 2 1 1 0 commit insert into t values(1, 'a') ")) + + tk.MustQuery("select * from t where a=2") + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary + where digest_text like 'select * from t%'`, + ).Check(testkit.Rows("select test test.t t:k 1 2 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tIndexLookUp_10\troot\t1000\t\n" + + "\t├─IndexScan_8 \tcop \t1000\ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableScan_9 \tcop \t1000\ttable:t, keep order:false, stats:pseudo")) + + // Disable it in global scope. + tk.MustExec("set global tidb_enable_stmt_summary = false") + + // Create a new session to test. + tk = testkit.NewTestKitWithInit(c, s.store) + + tk.MustQuery("select * from t where a=2") + + // Statement summary is still enabled. + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary + where digest_text like 'select * from t%'`, + ).Check(testkit.Rows("select test test.t t:k 2 4 0 0 0 0 0 0 0 0 0 select * from t where a=2 \tIndexLookUp_10\troot\t1000\t\n" + + "\t├─IndexScan_8 \tcop \t1000\ttable:t, index:k(a), range:[2,2], keep order:false, stats:pseudo\n" + + "\t└─TableScan_9 \tcop \t1000\ttable:t, keep order:false, stats:pseudo")) + + // Unset session variable. + tk.MustExec("set session tidb_enable_stmt_summary = ''") + tk.MustQuery("select * from t where a=2") + + // Statement summary is disabled. + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary`, + ).Check(testkit.Rows()) + + // Create a new session to test + tk = testkit.NewTestKitWithInit(c, s.store) + + tk.MustExec("set global tidb_enable_stmt_summary = on") + tk.MustExec("set global tidb_stmt_summary_history_size = 24") + + // Create a new user to test statements summary table privilege + tk.MustExec("create user 'test_user'@'localhost'") + tk.MustExec("grant select on *.* to 'test_user'@'localhost'") + tk.Se.Auth(&auth.UserIdentity{ + Username: "root", + Hostname: "%", + AuthUsername: "root", + AuthHostname: "%", + }, nil, nil) + tk.MustExec("select * from t where a=1") + result := tk.MustQuery(`select * + from information_schema.statements_summary + where digest_text like 'select * from t%'`, + ) + // Super user can query all records. + c.Assert(len(result.Rows()), Equals, 1) + result = tk.MustQuery(`select * + from information_schema.statements_summary_history + where digest_text like 'select * from t%'`, + ) + c.Assert(len(result.Rows()), Equals, 1) + tk.Se.Auth(&auth.UserIdentity{ + Username: "test_user", + Hostname: "localhost", + AuthUsername: "test_user", + AuthHostname: "localhost", + }, nil, nil) + result = tk.MustQuery(`select * + from information_schema.statements_summary + where digest_text like 'select * from t%'`, + ) + // Ordinary users can not see others' records + c.Assert(len(result.Rows()), Equals, 0) + result = tk.MustQuery(`select * + from information_schema.statements_summary_history + where digest_text like 'select * from t%'`, + ) + c.Assert(len(result.Rows()), Equals, 0) + tk.MustExec("select * from t where a=1") + result = tk.MustQuery(`select * + from information_schema.statements_summary + where digest_text like 'select * from t%'`, + ) + c.Assert(len(result.Rows()), Equals, 1) + tk.MustExec("select * from t where a=1") + result = tk.MustQuery(`select * + from information_schema.statements_summary_history + where digest_text like 'select * from t%'`, + ) + c.Assert(len(result.Rows()), Equals, 1) + // use root user to set variables back + tk.Se.Auth(&auth.UserIdentity{ + Username: "root", + Hostname: "%", + AuthUsername: "root", + AuthHostname: "%", + }, nil, nil) +} + +// Test statements_summary_history. +func (s *testTableSuite) TestStmtSummaryHistoryTable(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + + tk.MustExec("drop table if exists test_summary") + tk.MustExec("create table test_summary(a int, b varchar(10), key k(a))") + + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) + + // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. + s.dom.GetGlobalVarsCache().Disable() + // Disable refreshing summary. + tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") + tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) + + // Create a new session to test. + tk = testkit.NewTestKitWithInit(c, s.store) + + // Test INSERT + tk.MustExec("insert into test_summary values(1, 'a')") + tk.MustExec("insert into test_summary values(2, 'b')") + tk.MustExec("insert into TEST_SUMMARY VALUES(3, 'c')") + tk.MustExec("/**/insert into test_summary values(4, 'd')") + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary_history + where digest_text like 'insert into test_summary%'`, + ).Check(testkit.Rows("insert test test.test_summary 4 0 0 0 0 0 2 2 1 1 1 insert into test_summary values(1, 'a') ")) + + tk.MustExec("set global tidb_stmt_summary_history_size = 0") + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, + max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, + max_prewrite_regions, avg_affected_rows, query_sample_text, plan + from information_schema.statements_summary_history`, + ).Check(testkit.Rows()) +} + +// Test statements_summary_history. +func (s *testTableSuite) TestStmtSummaryInternalQuery(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10), key k(a))") + + // We use the sql binding evolve to check the internal query summary. + tk.MustExec("set @@tidb_use_plan_baselines = 1") + tk.MustExec("set @@tidb_evolve_plan_baselines = 1") + tk.MustExec("create global binding for select * from t where t.a = 1 using select * from t ignore index(k) where t.a = 1") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) + // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. + s.dom.GetGlobalVarsCache().Disable() + // Disable refreshing summary. + tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") + tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) + + // Test Internal + + // Create a new session to test. + tk = testkit.NewTestKitWithInit(c, s.store) + + tk.MustExec("select * from t where t.a = 1") + tk.MustQuery(`select exec_count, digest_text + from information_schema.statements_summary + where digest_text like "select original_sql , bind_sql , default_db , status%"`).Check(testkit.Rows()) + + // Enable internal query and evolve baseline. + tk.MustExec("set global tidb_stmt_summary_internal_query = 1") + defer tk.MustExec("set global tidb_stmt_summary_internal_query = false") + + // Create a new session to test. + tk = testkit.NewTestKitWithInit(c, s.store) + + tk.MustExec("admin flush bindings") + tk.MustExec("admin evolve bindings") + + tk.MustQuery(`select exec_count, digest_text + from information_schema.statements_summary + where digest_text like "select original_sql , bind_sql , default_db , status%"`).Check(testkit.Rows( + "1 select original_sql , bind_sql , default_db , status , create_time , update_time , charset , collation from mysql . bind_info" + + " where update_time > ? order by update_time")) +} + +// Test error count and warning count. +func (s *testTableSuite) TestStmtSummaryErrorCount(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + + // Clear summaries. + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + + tk.MustExec("use test") + tk.MustExec("drop table if exists stmt_summary_test") + tk.MustExec("create table stmt_summary_test(id int primary key)") + tk.MustExec("insert into stmt_summary_test values(1)") + _, err := tk.Exec("insert into stmt_summary_test values(1)") + c.Assert(err, NotNil) + + tk.MustQuery(`select exec_count, sum_errors, sum_warnings + from information_schema.statements_summary + where digest_text like "insert into stmt_summary_test%"`).Check(testkit.Rows("2 1 0")) + + tk.MustExec("insert ignore into stmt_summary_test values(1)") + tk.MustQuery(`select exec_count, sum_errors, sum_warnings + from information_schema.statements_summary + where digest_text like "insert ignore into stmt_summary_test%"`).Check(testkit.Rows("1 0 1")) +} diff --git a/kv/buffer_store.go b/kv/buffer_store.go index f81ffdc47a5b3..d246c9c13526f 100644 --- a/kv/buffer_store.go +++ b/kv/buffer_store.go @@ -37,6 +37,14 @@ func NewBufferStore(r Retriever) *BufferStore { } } +// NewBufferStoreFrom creates a BufferStore from retriever and mem-buffer. +func NewBufferStoreFrom(r Retriever, buf MemBuffer) *BufferStore { + return &BufferStore{ + r: r, + MemBuffer: buf, + } +} + // NewStagingBufferStore returns a BufferStore with buffer derived from the buffer. func NewStagingBufferStore(buf MemBuffer) *BufferStore { return &BufferStore{ diff --git a/kv/key.go b/kv/key.go index ff015053b96cc..386059a24968f 100644 --- a/kv/key.go +++ b/kv/key.go @@ -16,6 +16,11 @@ package kv import ( "bytes" "encoding/hex" + "fmt" + "strconv" + "strings" + + "github.com/pingcap/tidb/util/codec" ) // Key represents high-level Key type. @@ -25,7 +30,7 @@ type Key []byte func (k Key) Next() Key { // add 0x0 to the end of key buf := make([]byte, len(k)+1) - copy(buf, []byte(k)) + copy(buf, k) return buf } @@ -42,7 +47,7 @@ func (k Key) Next() Key { // If we seek 'rowkey1' PrefixNext, we will get 'rowkey2'. func (k Key) PrefixNext() Key { buf := make([]byte, len(k)) - copy(buf, []byte(k)) + copy(buf, k) var i int for i = len(k) - 1; i >= 0; i-- { buf[i]++ @@ -71,7 +76,7 @@ func (k Key) HasPrefix(prefix Key) bool { // Clone returns a deep copy of the Key. func (k Key) Clone() Key { ck := make([]byte, len(k)) - copy(ck, []byte(k)) + copy(ck, k) return ck } @@ -119,3 +124,260 @@ func (r *KeyRange) IsPoint() bool { return r.StartKey[diffOneIdx]+1 == r.EndKey[diffOneIdx] && bytes.Equal(r.StartKey[:diffOneIdx], r.EndKey[:diffOneIdx]) } + +// Handle is the ID of a row. +type Handle interface { + // IsInt returns if the handle type is int64. + IsInt() bool + // IntValue returns the int64 value if IsInt is true, it panics if IsInt returns false. + IntValue() int64 + // Next returns the minimum handle that is greater than this handle. + Next() Handle + // Equal returns if the handle equals to another handle, it panics if the types are different. + Equal(h Handle) bool + // Compare returns the comparison result of the two handles, it panics if the types are different. + Compare(h Handle) int + // Encoded returns the encoded bytes. + Encoded() []byte + // Len returns the length of the encoded bytes. + Len() int + // NumCols returns the number of columns of the handle, + NumCols() int + // EncodedCol returns the encoded column value at the given column index. + EncodedCol(idx int) []byte + // String implements the fmt.Stringer interface. + String() string +} + +// IntHandle implement the Handle interface for int64 type handle. +type IntHandle int64 + +// IsInt implements the Handle interface. +func (ih IntHandle) IsInt() bool { + return true +} + +// IntValue implements the Handle interface. +func (ih IntHandle) IntValue() int64 { + return int64(ih) +} + +// Next implements the Handle interface. +func (ih IntHandle) Next() Handle { + return IntHandle(int64(ih) + 1) +} + +// Equal implements the Handle interface. +func (ih IntHandle) Equal(h Handle) bool { + return h.IsInt() && int64(ih) == h.IntValue() +} + +// Compare implements the Handle interface. +func (ih IntHandle) Compare(h Handle) int { + if !h.IsInt() { + panic("IntHandle compares to CommonHandle") + } + ihVal := ih.IntValue() + hVal := h.IntValue() + if ihVal > hVal { + return 1 + } + if ihVal < hVal { + return -1 + } + return 0 +} + +// Encoded implements the Handle interface. +func (ih IntHandle) Encoded() []byte { + return codec.EncodeInt(nil, int64(ih)) +} + +// Len implements the Handle interface. +func (ih IntHandle) Len() int { + return 8 +} + +// NumCols implements the Handle interface, not supported for IntHandle type. +func (ih IntHandle) NumCols() int { + panic("not supported in IntHandle") +} + +// EncodedCol implements the Handle interface., not supported for IntHandle type. +func (ih IntHandle) EncodedCol(idx int) []byte { + panic("not supported in IntHandle") +} + +// String implements the Handle interface. +func (ih IntHandle) String() string { + return strconv.FormatInt(int64(ih), 10) +} + +// CommonHandle implements the Handle interface for non-int64 type handle. +type CommonHandle struct { + encoded []byte + colEndOffsets []uint16 +} + +// NewCommonHandle creates a CommonHandle from a encoded bytes which is encoded by code.EncodeKey. +func NewCommonHandle(encoded []byte) (*CommonHandle, error) { + ch := &CommonHandle{encoded: encoded} + remain := encoded + endOff := uint16(0) + for len(remain) > 0 { + var err error + var col []byte + col, remain, err = codec.CutOne(remain) + if err != nil { + return nil, err + } + endOff += uint16(len(col)) + ch.colEndOffsets = append(ch.colEndOffsets, endOff) + } + return ch, nil +} + +// IsInt implements the Handle interface. +func (ch *CommonHandle) IsInt() bool { + return false +} + +// IntValue implements the Handle interface, not supported for CommonHandle type. +func (ch *CommonHandle) IntValue() int64 { + panic("not supported in CommonHandle") +} + +// Next implements the Handle interface. +func (ch *CommonHandle) Next() Handle { + return &CommonHandle{ + encoded: Key(ch.encoded).PrefixNext(), + colEndOffsets: ch.colEndOffsets, + } +} + +// Equal implements the Handle interface. +func (ch *CommonHandle) Equal(h Handle) bool { + return !h.IsInt() && bytes.Equal(ch.encoded, h.Encoded()) +} + +// Compare implements the Handle interface. +func (ch *CommonHandle) Compare(h Handle) int { + if h.IsInt() { + panic("CommonHandle compares to IntHandle") + } + return bytes.Compare(ch.encoded, h.Encoded()) +} + +// Encoded implements the Handle interface. +func (ch *CommonHandle) Encoded() []byte { + return ch.encoded +} + +// Len implements the Handle interface. +func (ch *CommonHandle) Len() int { + return len(ch.encoded) +} + +// NumCols implements the Handle interface. +func (ch *CommonHandle) NumCols() int { + return len(ch.colEndOffsets) +} + +// EncodedCol implements the Handle interface. +func (ch *CommonHandle) EncodedCol(idx int) []byte { + colStartOffset := uint16(0) + if idx > 0 { + colStartOffset = ch.colEndOffsets[idx-1] + } + return ch.encoded[colStartOffset:ch.colEndOffsets[idx]] +} + +// String implements the Handle interface. +func (ch *CommonHandle) String() string { + strs := make([]string, 0, ch.NumCols()) + for i := 0; i < ch.NumCols(); i++ { + encodedCol := ch.EncodedCol(i) + _, d, err := codec.DecodeOne(encodedCol) + if err != nil { + return err.Error() + } + str, err := d.ToString() + if err != nil { + return err.Error() + } + strs = append(strs, str) + } + return fmt.Sprintf("{%s}", strings.Join(strs, ", ")) +} + +// HandleMap is the map for Handle. +type HandleMap struct { + ints map[int64]interface{} + strs map[string]strHandleVal +} + +type strHandleVal struct { + h Handle + val interface{} +} + +// NewHandleMap creates a new map for handle. +func NewHandleMap() *HandleMap { + // Initialize the two maps to avoid checking nil. + return &HandleMap{ + ints: map[int64]interface{}{}, + strs: map[string]strHandleVal{}, + } +} + +// Get gets a value by a Handle. +func (m *HandleMap) Get(h Handle) (v interface{}, ok bool) { + if h.IsInt() { + v, ok = m.ints[h.IntValue()] + } else { + var strVal strHandleVal + strVal, ok = m.strs[string(h.Encoded())] + v = strVal.val + } + return +} + +// Set sets a value with a Handle. +func (m *HandleMap) Set(h Handle, val interface{}) { + if h.IsInt() { + m.ints[h.IntValue()] = val + } else { + m.strs[string(h.Encoded())] = strHandleVal{ + h: h, + val: val, + } + } +} + +// Delete deletes a entry from the map. +func (m *HandleMap) Delete(h Handle) { + if h.IsInt() { + delete(m.ints, h.IntValue()) + } else { + delete(m.strs, string(h.Encoded())) + } +} + +// Len returns the length of the map. +func (m *HandleMap) Len() int { + return len(m.ints) + len(m.strs) +} + +// Range iterates the HandleMap with fn, the fn returns true to continue, returns false to stop. +func (m *HandleMap) Range(fn func(h Handle, val interface{}) bool) { + for h, val := range m.ints { + if !fn(IntHandle(h), val) { + return + } + } + for _, strVal := range m.strs { + if !fn(strVal.h, strVal.val) { + return + } + } +} diff --git a/kv/key_test.go b/kv/key_test.go index 411743e6d2907..515bf0dc4ac72 100644 --- a/kv/key_test.go +++ b/kv/key_test.go @@ -114,6 +114,84 @@ func (s *testKeySuite) TestBasicFunc(c *C) { c.Assert(IsTxnRetryableError(errors.New("test")), IsFalse) } +func (s *testKeySuite) TestHandle(c *C) { + ih := IntHandle(100) + c.Assert(ih.IsInt(), IsTrue) + _, iv, _ := codec.DecodeInt(ih.Encoded()) + c.Assert(iv, Equals, ih.IntValue()) + ih2 := ih.Next() + c.Assert(ih2.IntValue(), Equals, int64(101)) + c.Assert(ih.Equal(ih2), IsFalse) + c.Assert(ih.Compare(ih2), Equals, -1) + c.Assert(ih.String(), Equals, "100") + ch := mustNewCommonHandle(c, 100, "abc") + c.Assert(ch.IsInt(), IsFalse) + ch2 := ch.Next() + c.Assert(ch.Equal(ch2), IsFalse) + c.Assert(ch.Compare(ch2), Equals, -1) + c.Assert(ch2.Encoded(), HasLen, len(ch.Encoded())) + c.Assert(ch.NumCols(), Equals, 2) + _, d, err := codec.DecodeOne(ch.EncodedCol(0)) + c.Assert(err, IsNil) + c.Assert(d.GetInt64(), Equals, int64(100)) + _, d, err = codec.DecodeOne(ch.EncodedCol(1)) + c.Assert(err, IsNil) + c.Assert(d.GetString(), Equals, "abc") + c.Assert(ch.String(), Equals, "{100, abc}") +} + +func (s *testKeySuite) TestHandleMap(c *C) { + m := NewHandleMap() + h := IntHandle(1) + m.Set(h, 1) + v, ok := m.Get(h) + c.Assert(ok, IsTrue) + c.Assert(v, Equals, 1) + m.Delete(h) + v, ok = m.Get(h) + c.Assert(ok, IsFalse) + c.Assert(v, IsNil) + ch := mustNewCommonHandle(c, 100, "abc") + m.Set(ch, "a") + v, ok = m.Get(ch) + c.Assert(ok, IsTrue) + c.Assert(v, Equals, "a") + m.Delete(ch) + v, ok = m.Get(ch) + c.Assert(ok, IsFalse) + c.Assert(v, IsNil) + m.Set(ch, "a") + ch2 := mustNewCommonHandle(c, 101, "abc") + m.Set(ch2, "b") + ch3 := mustNewCommonHandle(c, 99, "def") + m.Set(ch3, "c") + c.Assert(m.Len(), Equals, 3) + cnt := 0 + m.Range(func(h Handle, val interface{}) bool { + cnt++ + if h.Equal(ch) { + c.Assert(val, Equals, "a") + } else if h.Equal(ch2) { + c.Assert(val, Equals, "b") + } else { + c.Assert(val, Equals, "c") + } + if cnt == 2 { + return false + } + return true + }) + c.Assert(cnt, Equals, 2) +} + +func mustNewCommonHandle(c *C, values ...interface{}) *CommonHandle { + encoded, err := codec.EncodeKey(new(stmtctx.StatementContext), nil, types.MakeDatums(values...)...) + c.Assert(err, IsNil) + ch, err := NewCommonHandle(encoded) + c.Assert(err, IsNil) + return ch +} + func BenchmarkIsPoint(b *testing.B) { b.ReportAllocs() kr := KeyRange{ diff --git a/kv/memdb/memdb.go b/kv/memdb/memdb.go index e5984a2d06c9b..644381c44e709 100644 --- a/kv/memdb/memdb.go +++ b/kv/memdb/memdb.go @@ -456,6 +456,7 @@ func (ms *mergeState) calculateRecomputeHeight(key []byte, sb *Sandbox) int { !prev.addr.isNull() && bytes.Compare(key, prev.getKey(arena.getFrom(prev.addr))) <= 0 { // Key is before splice. + // For merge operation this branch unused actually, because keys are added in ascending order. for prev.addr == ms.prev[recomputeHeight].addr { recomputeHeight++ } diff --git a/kv/memdb/memdb_norace_test.go b/kv/memdb/memdb_norace_test.go new file mode 100644 index 0000000000000..e443d535250db --- /dev/null +++ b/kv/memdb/memdb_norace_test.go @@ -0,0 +1,109 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// +build !race + +package memdb + +import ( + "encoding/binary" + "math/rand" + + . "github.com/pingcap/check" + "github.com/pingcap/goleveldb/leveldb/comparer" + "github.com/pingcap/goleveldb/leveldb/memdb" +) + +// The test takes too long under the race detector. +func (s testMemDBSuite) TestRandom(c *C) { + c.Parallel() + const cnt = 500000 + keys := make([][]byte, cnt) + for i := range keys { + keys[i] = make([]byte, rand.Intn(19)+1) + rand.Read(keys[i]) + } + + p1 := NewSandbox() + p2 := memdb.New(comparer.DefaultComparer, 4*1024) + for _, k := range keys { + p1.Put(k, k) + _ = p2.Put(k, k) + } + + c.Check(p1.Len(), Equals, p2.Len()) + c.Check(p1.Size(), Equals, p2.Size()) + + rand.Shuffle(cnt, func(i, j int) { keys[i], keys[j] = keys[j], keys[i] }) + + for _, k := range keys { + newValue := make([]byte, rand.Intn(19)+1) + rand.Read(newValue) + p1.Put(k, newValue) + _ = p2.Put(k, newValue) + } + s.checkConsist(c, p1, p2) +} + +// The test takes too long under the race detector. +func (s testMemDBSuite) TestRandomDerive(c *C) { + c.Parallel() + s.testRandomDeriveRecur(c, NewSandbox(), memdb.New(comparer.DefaultComparer, 4*1024), 0) +} + +func (s testMemDBSuite) testRandomDeriveRecur(c *C, sb *Sandbox, db *memdb.DB, depth int) { + var keys [][]byte + if rand.Float64() < 0.5 { + start, end := rand.Intn(512), rand.Intn(512)+512 + cnt := end - start + keys = make([][]byte, cnt) + for i := range keys { + keys[i] = make([]byte, 8) + binary.BigEndian.PutUint64(keys[i], uint64(start+i)) + } + } else { + keys = make([][]byte, rand.Intn(512)+512) + for i := range keys { + keys[i] = make([]byte, rand.Intn(19)+1) + rand.Read(keys[i]) + } + } + + vals := make([][]byte, len(keys)) + for i := range vals { + vals[i] = make([]byte, rand.Intn(255)+1) + rand.Read(vals[i]) + } + + sbBuf := sb.Derive() + dbBuf := memdb.New(comparer.DefaultComparer, 4*1024) + for i := range keys { + sbBuf.Put(keys[i], vals[i]) + _ = dbBuf.Put(keys[i], vals[i]) + } + + if depth < 1000 { + s.testRandomDeriveRecur(c, sbBuf, dbBuf, depth+1) + } + + if rand.Float64() < 0.3 && depth > 0 { + sbBuf.Discard() + } else { + sbBuf.Flush() + it := dbBuf.NewIterator(nil) + for it.First(); it.Valid(); it.Next() { + _ = db.Put(it.Key(), it.Value()) + } + } + s.checkConsist(c, sb, db) +} diff --git a/kv/memdb/memdb_test.go b/kv/memdb/memdb_test.go index 3a73f90614e69..51ea4b96f0f51 100644 --- a/kv/memdb/memdb_test.go +++ b/kv/memdb/memdb_test.go @@ -19,7 +19,6 @@ import ( "testing" . "github.com/pingcap/check" - "github.com/pingcap/goleveldb/leveldb/comparer" "github.com/pingcap/goleveldb/leveldb/memdb" ) @@ -48,6 +47,19 @@ func (s testMemDBSuite) TestGetSet(c *C) { } } +func (s testMemDBSuite) TestBigKV(c *C) { + p := NewSandbox() + p.Put([]byte{1}, make([]byte, 80<<20)) + c.Check(p.arena.blockSize, Equals, maxBlockSize) + c.Check(len(p.arena.blocks), Equals, 1) + p1 := p.Derive() + p1.Put([]byte{2}, make([]byte, 127<<20)) + p1.Flush() + c.Check(p.arena.blockSize, Equals, maxBlockSize) + c.Check(len(p.arena.blocks), Equals, 2) + c.Check(func() { p.Put([]byte{3}, make([]byte, maxBlockSize+1)) }, Panics, "alloc size is larger than max block size") +} + func (s testMemDBSuite) TestIterator(c *C) { const cnt = 10000 p := s.fillDB(cnt) @@ -74,6 +86,15 @@ func (s testMemDBSuite) TestIterator(c *C) { c.Check(i, Equals, -1) } +func (s testMemDBSuite) TestRuntimeAssertion(c *C) { + p := NewSandbox() + p1 := p.Derive() + c.Check(func() { p.Put([]byte{0}, []byte{}) }, Panics, "cannot write to a sandbox when it has forked a new sanbox") + c.Check(func() { p.Derive() }, Panics, "cannot start second sandbox") + c.Check(func() { p.Discard() }, Panics, "root sandbox is freezed") + p1.Discard() +} + func (s testMemDBSuite) TestDiscard(c *C) { const cnt = 10000 p := NewSandbox() @@ -316,86 +337,6 @@ func (s testMemDBSuite) TestEmptyDB(c *C) { c.Check(it.Valid(), IsFalse) } -func (s testMemDBSuite) TestRandom(c *C) { - const cnt = 500000 - keys := make([][]byte, cnt) - for i := range keys { - keys[i] = make([]byte, rand.Intn(19)+1) - rand.Read(keys[i]) - } - - p1 := NewSandbox() - p2 := memdb.New(comparer.DefaultComparer, 4*1024) - for _, k := range keys { - p1.Put(k, k) - _ = p2.Put(k, k) - } - - c.Check(p1.Len(), Equals, p2.Len()) - c.Check(p1.Size(), Equals, p2.Size()) - - rand.Shuffle(cnt, func(i, j int) { keys[i], keys[j] = keys[j], keys[i] }) - - for _, k := range keys { - newValue := make([]byte, rand.Intn(19)+1) - rand.Read(newValue) - p1.Put(k, newValue) - _ = p2.Put(k, newValue) - } - s.checkConsist(c, p1, p2) -} - -func (s testMemDBSuite) TestRandomDerive(c *C) { - s.testRandomDeriveRecur(c, NewSandbox(), memdb.New(comparer.DefaultComparer, 4*1024), 0) -} - -func (s testMemDBSuite) testRandomDeriveRecur(c *C, sb *Sandbox, db *memdb.DB, depth int) { - var keys [][]byte - if rand.Float64() < 0.5 { - start, end := rand.Intn(512), rand.Intn(512)+512 - cnt := end - start - keys = make([][]byte, cnt) - for i := range keys { - keys[i] = make([]byte, 8) - binary.BigEndian.PutUint64(keys[i], uint64(start+i)) - } - } else { - keys = make([][]byte, rand.Intn(512)+512) - for i := range keys { - keys[i] = make([]byte, rand.Intn(19)+1) - rand.Read(keys[i]) - } - } - - vals := make([][]byte, len(keys)) - for i := range vals { - vals[i] = make([]byte, rand.Intn(255)+1) - rand.Read(vals[i]) - } - - sbBuf := sb.Derive() - dbBuf := memdb.New(comparer.DefaultComparer, 4*1024) - for i := range keys { - sbBuf.Put(keys[i], vals[i]) - _ = dbBuf.Put(keys[i], vals[i]) - } - - if depth < 1000 { - s.testRandomDeriveRecur(c, sbBuf, dbBuf, depth+1) - } - - if rand.Float64() < 0.3 && depth > 0 { - sbBuf.Discard() - } else { - sbBuf.Flush() - it := dbBuf.NewIterator(nil) - for it.First(); it.Valid(); it.Next() { - _ = db.Put(it.Key(), it.Value()) - } - } - s.checkConsist(c, sb, db) -} - func (s testMemDBSuite) checkConsist(c *C, p1 *Sandbox, p2 *memdb.DB) { c.Check(p1.Len(), Equals, p2.Len()) c.Check(p1.Size(), Equals, p2.Size()) diff --git a/kv/mock_test.go b/kv/mock_test.go index 99dbd5430260c..c682bd67b1c16 100644 --- a/kv/mock_test.go +++ b/kv/mock_test.go @@ -100,7 +100,7 @@ func (s testMockSuite) TestIsPoint(c *C) { kr = KeyRange{ StartKey: Key(""), - EndKey: Key([]byte{0}), + EndKey: []byte{0}, } c.Check(kr.IsPoint(), IsTrue) } diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 9a2acc1d0c242..3977fa213745a 100755 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -76,6 +76,20 @@ const ( SequenceType ) +// CustomAutoIncCacheOption is one kind of AllocOption to customize the allocator step length. +type CustomAutoIncCacheOption int64 + +// ApplyOn is implement the AllocOption interface. +func (step CustomAutoIncCacheOption) ApplyOn(alloc *allocator) { + alloc.step = int64(step) + alloc.customStep = true +} + +// AllocOption is a interface to define allocator custom options coming in future. +type AllocOption interface { + ApplyOn(*allocator) +} + // Allocator is an auto increment id generator. // Just keep id unique actually. type Allocator interface { @@ -138,6 +152,7 @@ type allocator struct { isUnsigned bool lastAllocTime time.Time step int64 + customStep bool allocType AllocatorType sequence *model.SequenceInfo } @@ -370,8 +385,8 @@ func NextStep(curStep int64, consumeDur time.Duration) int64 { } // NewAllocator returns a new auto increment id generator on the store. -func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool, allocType AllocatorType) Allocator { - return &allocator{ +func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool, allocType AllocatorType, opts ...AllocOption) Allocator { + alloc := &allocator{ store: store, dbID: dbID, isUnsigned: isUnsigned, @@ -379,6 +394,10 @@ func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool, allocType Alloc lastAllocTime: time.Now(), allocType: allocType, } + for _, fn := range opts { + fn.ApplyOn(alloc) + } + return alloc } // NewSequenceAllocator returns a new sequence value generator on the store. @@ -398,7 +417,11 @@ func NewSequenceAllocator(store kv.Storage, dbID int64, info *model.SequenceInfo func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.TableInfo) Allocators { var allocs []Allocator dbID := tblInfo.GetDBID(schemaID) - allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType)) + if tblInfo.AutoIdCache > 0 { + allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, CustomAutoIncCacheOption(tblInfo.AutoIdCache))) + } else { + allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType)) + } if tblInfo.ContainsAutoRandomBits() { allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType)) } @@ -606,13 +629,18 @@ func (alloc *allocator) alloc4Signed(tableID int64, n uint64, increment, offset if alloc.base+n1 > alloc.end { var newBase, newEnd int64 startTime := time.Now() - // Although it may skip a segment here, we still think it is consumed. - consumeDur := startTime.Sub(alloc.lastAllocTime) - nextStep := NextStep(alloc.step, consumeDur) - // Make sure nextStep is big enough. + nextStep := alloc.step + if !alloc.customStep { + // Although it may skip a segment here, we still think it is consumed. + consumeDur := startTime.Sub(alloc.lastAllocTime) + nextStep = NextStep(alloc.step, consumeDur) + } + // Although the step is customized by user, we still need to make sure nextStep is big enough for insert batch. if nextStep <= n1 { - alloc.step = mathutil.MinInt64(n1*2, maxStep) - } else { + nextStep = mathutil.MinInt64(n1*2, maxStep) + } + // Store the step for non-customized-step allocator to calculate next dynamic step. + if !alloc.customStep { alloc.step = nextStep } err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { @@ -622,7 +650,7 @@ func (alloc *allocator) alloc4Signed(tableID int64, n uint64, increment, offset if err1 != nil { return err1 } - tmpStep := mathutil.MinInt64(math.MaxInt64-newBase, alloc.step) + tmpStep := mathutil.MinInt64(math.MaxInt64-newBase, nextStep) // The global rest is not enough for alloc. if tmpStep < n1 { return ErrAutoincReadFailed @@ -668,13 +696,18 @@ func (alloc *allocator) alloc4Unsigned(tableID int64, n uint64, increment, offse if uint64(alloc.base)+uint64(n1) > uint64(alloc.end) { var newBase, newEnd int64 startTime := time.Now() - // Although it may skip a segment here, we still treat it as consumed. - consumeDur := startTime.Sub(alloc.lastAllocTime) - nextStep := NextStep(alloc.step, consumeDur) - // Make sure nextStep is big enough. + nextStep := alloc.step + if !alloc.customStep { + // Although it may skip a segment here, we still treat it as consumed. + consumeDur := startTime.Sub(alloc.lastAllocTime) + nextStep = NextStep(alloc.step, consumeDur) + } + // Although the step is customized by user, we still need to make sure nextStep is big enough for insert batch. if nextStep <= n1 { - alloc.step = mathutil.MinInt64(n1*2, maxStep) - } else { + nextStep = mathutil.MinInt64(n1*2, maxStep) + } + // Store the step for non-customized-step allocator to calculate next dynamic step. + if !alloc.customStep { alloc.step = nextStep } err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { @@ -684,7 +717,7 @@ func (alloc *allocator) alloc4Unsigned(tableID int64, n uint64, increment, offse if err1 != nil { return err1 } - tmpStep := int64(mathutil.MinUint64(math.MaxUint64-uint64(newBase), uint64(alloc.step))) + tmpStep := int64(mathutil.MinUint64(math.MaxUint64-uint64(newBase), uint64(nextStep))) // The global rest is not enough for alloc. if tmpStep < n1 { return ErrAutoincReadFailed @@ -829,7 +862,7 @@ func getAutoIDByAllocType(m *meta.Meta, dbID, tableID int64, allocType Allocator case SequenceType: return m.GetSequenceValue(dbID, tableID) default: - return 0, errInvalidAllocatorType.GenWithStackByArgs() + return 0, ErrInvalidAllocatorType.GenWithStackByArgs() } } @@ -842,7 +875,7 @@ func generateAutoIDByAllocType(m *meta.Meta, dbID, tableID, step int64, allocTyp case SequenceType: return m.GenSequenceValue(dbID, tableID, step) default: - return 0, errInvalidAllocatorType.GenWithStackByArgs() + return 0, ErrInvalidAllocatorType.GenWithStackByArgs() } } diff --git a/meta/autoid/autoid_test.go b/meta/autoid/autoid_test.go index 6cd060da18fe8..357dbe59efa32 100644 --- a/meta/autoid/autoid_test.go +++ b/meta/autoid/autoid_test.go @@ -86,7 +86,7 @@ func (*testSuite) TestT(c *C) { c.Assert(err, NotNil) globalAutoID, err = alloc.NextGlobalAutoID(1) c.Assert(err, IsNil) - c.Assert(globalAutoID, Equals, int64(autoid.GetStep()+1)) + c.Assert(globalAutoID, Equals, autoid.GetStep()+1) // rebase err = alloc.Rebase(1, int64(1), true) @@ -114,7 +114,7 @@ func (*testSuite) TestT(c *C) { c.Assert(alloc, NotNil) _, id, err = alloc.Alloc(1, 1, 1, 1) c.Assert(err, IsNil) - c.Assert(id, Equals, int64(autoid.GetStep()+1)) + c.Assert(id, Equals, autoid.GetStep()+1) alloc = autoid.NewAllocator(store, 1, false, autoid.RowIDAllocType) c.Assert(alloc, NotNil) @@ -288,7 +288,7 @@ func (*testSuite) TestUnsignedAutoid(c *C) { c.Assert(err, NotNil) globalAutoID, err = alloc.NextGlobalAutoID(1) c.Assert(err, IsNil) - c.Assert(globalAutoID, Equals, int64(autoid.GetStep()+1)) + c.Assert(globalAutoID, Equals, autoid.GetStep()+1) // rebase err = alloc.Rebase(1, int64(1), true) @@ -316,7 +316,7 @@ func (*testSuite) TestUnsignedAutoid(c *C) { c.Assert(alloc, NotNil) _, id, err = alloc.Alloc(1, 1, 1, 1) c.Assert(err, IsNil) - c.Assert(id, Equals, int64(autoid.GetStep()+1)) + c.Assert(id, Equals, autoid.GetStep()+1) alloc = autoid.NewAllocator(store, 1, true, autoid.RowIDAllocType) c.Assert(alloc, NotNil) @@ -400,7 +400,7 @@ func (*testSuite) TestUnsignedAutoid(c *C) { c.Assert(max-min, Equals, autoid.CalcNeededBatchSize(int64(uint64(offset)-1), 2, increment, offset, true)) firstID := autoid.SeekToFirstAutoIDUnSigned(uint64(min), uint64(increment), uint64(offset)) - c.Assert(uint64(firstID), Equals, uint64(math.MaxUint64-100)) + c.Assert(firstID, Equals, uint64(math.MaxUint64-100)) } diff --git a/meta/autoid/errors.go b/meta/autoid/errors.go index 68d7b4850417c..11a218d350c66 100644 --- a/meta/autoid/errors.go +++ b/meta/autoid/errors.go @@ -24,7 +24,7 @@ var ( errInvalidIncrementAndOffset = terror.ClassAutoid.New(mysql.ErrInvalidIncrementAndOffset, mysql.MySQLErrName[mysql.ErrInvalidIncrementAndOffset]) ErrAutoincReadFailed = terror.ClassAutoid.New(mysql.ErrAutoincReadFailed, mysql.MySQLErrName[mysql.ErrAutoincReadFailed]) ErrWrongAutoKey = terror.ClassAutoid.New(mysql.ErrWrongAutoKey, mysql.MySQLErrName[mysql.ErrWrongAutoKey]) - errInvalidAllocatorType = terror.ClassAutoid.New(mysql.ErrUnknownAllocatorType, mysql.MySQLErrName[mysql.ErrUnknownAllocatorType]) + ErrInvalidAllocatorType = terror.ClassAutoid.New(mysql.ErrUnknownAllocatorType, mysql.MySQLErrName[mysql.ErrUnknownAllocatorType]) ErrAutoRandReadFailed = terror.ClassAutoid.New(mysql.ErrAutoRandReadFailed, mysql.MySQLErrName[mysql.ErrAutoRandReadFailed]) ) diff --git a/metrics/bindinfo.go b/metrics/bindinfo.go index 958bd110c2b23..2b5da872811d7 100644 --- a/metrics/bindinfo.go +++ b/metrics/bindinfo.go @@ -23,7 +23,7 @@ var ( Subsystem: "bindinfo", Name: "bind_usage_counter", Help: "Counter of query using sql bind", - }, []string{LableScope}) + }, []string{LabelScope}) BindTotalGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ @@ -31,7 +31,7 @@ var ( Subsystem: "bindinfo", Name: "bind_total_gauge", Help: "Total number of sql bind", - }, []string{LableScope, LblType}) + }, []string{LabelScope, LblType}) BindMemoryUsage = prometheus.NewGaugeVec( prometheus.GaugeOpts{ @@ -39,5 +39,5 @@ var ( Subsystem: "bindinfo", Name: "bind_memory_usage", Help: "Memory usage of sql bind", - }, []string{LableScope, LblType}) + }, []string{LabelScope, LblType}) ) diff --git a/metrics/ddl.go b/metrics/ddl.go index b5b97800afeb6..8b1a650cfc710 100644 --- a/metrics/ddl.go +++ b/metrics/ddl.go @@ -31,7 +31,7 @@ var ( Subsystem: "ddl", Name: "handle_job_duration_seconds", Help: "Bucketed histogram of processing time (s) of handle jobs", - Buckets: prometheus.ExponentialBuckets(0.01, 2, 22), // 10ms ~ 12hours + Buckets: prometheus.ExponentialBuckets(0.01, 2, 22), // 10ms ~ 6hours }, []string{LblType, LblResult}) BatchAddIdxHistogram = prometheus.NewHistogramVec( @@ -40,7 +40,7 @@ var ( Subsystem: "ddl", Name: "batch_add_idx_duration_seconds", Help: "Bucketed histogram of processing time (s) of batch handle data", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 1hours + Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 0.5hours }, []string{LblType}) SyncerInit = "init" @@ -53,7 +53,7 @@ var ( Subsystem: "ddl", Name: "deploy_syncer_duration_seconds", Help: "Bucketed histogram of processing time (s) of deploy syncer", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 1024s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, []string{LblType, LblResult}) UpdateSelfVersionHistogram = prometheus.NewHistogramVec( @@ -62,7 +62,7 @@ var ( Subsystem: "ddl", Name: "update_self_ver_duration_seconds", Help: "Bucketed histogram of processing time (s) of update self version", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 1024s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, []string{LblResult}) OwnerUpdateGlobalVersion = "update_global_version" @@ -77,7 +77,7 @@ var ( Subsystem: "ddl", Name: "owner_handle_syncer_duration_seconds", Help: "Bucketed histogram of processing time (s) of handle syncer", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 1024s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, []string{LblType, LblResult}) // Metrics for ddl_worker.go. @@ -91,7 +91,7 @@ var ( Subsystem: "ddl", Name: "worker_operation_duration_seconds", Help: "Bucketed histogram of processing time (s) of ddl worker operations", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 4096s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 2048s }, []string{LblType, LblAction, LblResult}) CreateDDLInstance = "create_ddl_instance" diff --git a/metrics/distsql.go b/metrics/distsql.go index 23f8744a5ca57..a0164a45e897b 100644 --- a/metrics/distsql.go +++ b/metrics/distsql.go @@ -19,13 +19,13 @@ import ( // distsql metrics. var ( - DistSQLQueryHistgram = prometheus.NewHistogramVec( + DistSQLQueryHistogram = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "distsql", Name: "handle_query_duration_seconds", Help: "Bucketed histogram of processing time (s) of handled queries.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 18), // 0.5ms ~ 128s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 18), // 0.5ms ~ 64s }, []string{LblType, LblSQLType}) DistSQLScanKeysPartialHistogram = prometheus.NewHistogram( diff --git a/metrics/domain.go b/metrics/domain.go index a8ea4e3d4cbbf..dd3912555d59c 100644 --- a/metrics/domain.go +++ b/metrics/domain.go @@ -35,7 +35,7 @@ var ( Subsystem: "domain", Name: "load_schema_duration_seconds", Help: "Bucketed histogram of processing time (s) in load schema.", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 4000s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }) // LoadPrivilegeCounter records the counter of load privilege. diff --git a/metrics/gc_worker.go b/metrics/gc_worker.go index 5e6ef07511357..6ca39f6901eb2 100644 --- a/metrics/gc_worker.go +++ b/metrics/gc_worker.go @@ -33,7 +33,7 @@ var ( Subsystem: "tikvclient", Name: "gc_seconds", Help: "Bucketed histogram of gc duration.", - Buckets: prometheus.ExponentialBuckets(1, 2, 20), // 1s ~ 12days + Buckets: prometheus.ExponentialBuckets(1, 2, 20), // 1s ~ 6days }, []string{"stage"}) GCConfigGauge = prometheus.NewGaugeVec( diff --git a/metrics/grafana/overview.json b/metrics/grafana/overview.json index c2ba48561d801..8e86f21377a0b 100644 --- a/metrics/grafana/overview.json +++ b/metrics/grafana/overview.json @@ -173,68 +173,75 @@ "legendFormat": "TiKV", "refId": "C" }, + { + "expr": "\ncount(probe_success{group=\"tiflash\"} == 1)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "TiFlash", + "refId": "D" + }, { "expr": "\ncount(probe_success{group=\"pump\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Pump", - "refId": "D" + "refId": "E" }, { "expr": "\ncount(probe_success{group=\"drainer\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Drainer", - "refId": "E" + "refId": "F" }, { "expr": "\ncount(probe_success{group=\"kafka\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Kafka", - "refId": "F" + "refId": "G" }, { "expr": "\ncount(probe_success{group=\"zookeeper\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Zookeeper", - "refId": "G" + "refId": "H" }, { "expr": "\ncount(probe_success{group=\"node_exporter\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Node_exporter", - "refId": "H" + "refId": "I" }, { "expr": "\ncount(probe_success{group=\"blackbox_exporter\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Blackbox_exporter", - "refId": "I" + "refId": "J" }, { "expr": "\ncount(probe_success{group=\"grafana\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Grafana", - "refId": "J" + "refId": "K" }, { "expr": "\ncount(probe_success{job=\"blackbox_exporter_http\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Pushgateway", - "refId": "K" + "refId": "L" }, { "expr": "\ncount(probe_success{group=\"kafka_exporter\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Kafka_exporter", - "refId": "L" + "refId": "M" } ], "timeFrom": "1s", @@ -344,68 +351,75 @@ "legendFormat": "TiKV", "refId": "C" }, + { + "expr": "\ncount(probe_success{group=\"tiflash\"} == 0)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "TiFlash", + "refId": "D" + }, { "expr": "\ncount(probe_success{group=\"pump\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Pump", - "refId": "D" + "refId": "E" }, { "expr": "\ncount(probe_success{group=\"drainer\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Drainer", - "refId": "E" + "refId": "F" }, { "expr": "\ncount(probe_success{group=\"kafka\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Kafka", - "refId": "F" + "refId": "G" }, { "expr": "\ncount(probe_success{group=\"zookeeper\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Zookeeper", - "refId": "G" + "refId": "H" }, { "expr": "\ncount(probe_success{group=\"node_exporter\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Node_exporter", - "refId": "H" + "refId": "I" }, { "expr": "\ncount(probe_success{group=\"blackbox_exporter\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Blackbox_exporter", - "refId": "I" + "refId": "J" }, { "expr": "\ncount(probe_success{group=\"grafana\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Grafana", - "refId": "J" + "refId": "K" }, { "expr": "\ncount(probe_success{job=\"blackbox_exporter_http\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Pushgateway", - "refId": "K" + "refId": "L" }, { "expr": "\ncount(probe_success{group=\"kafka_exporter\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Kafka_exporter", - "refId": "L" + "refId": "M" } ], "timeFrom": "1s", @@ -2153,19 +2167,27 @@ "steppedLine": false, "targets": [ { - "expr": "go_memstats_heap_inuse_bytes{job=~\"tidb.*\"}", + "expr": "process_resident_memory_bytes{job=\"tidb\"}", + "format": "time_series", "intervalFactor": 2, - "legendFormat": "{{instance}}-{{job}}", - "metric": "go_memstats_heap_inuse_bytes", + "legendFormat": "process-{{instance}}", "refId": "A", "step": 10 + }, + { + "expr": "go_memstats_heap_inuse_bytes{job=\"tidb\"}", + "legendFormat": "HeapInuse-{{instance}}", + "format": "time_series", + "intervalFactor": 2, + "refId": "B", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Heap Memory Usage", + "title": "Memory Usage", "tooltip": { "shared": true, "sort": 0, @@ -5198,4 +5220,4 @@ "title": "Test-Cluster-Overview", "uid": "eDbRZpnWk", "version": 1 -} \ No newline at end of file +} diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 2131a3058de00..c20709a433dd8 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -1768,17 +1768,47 @@ { "expr": "process_resident_memory_bytes{job=\"tidb\"}", "format": "time_series", - "hide": false, "intervalFactor": 1, "legendFormat": "process-{{instance}}", "refId": "A" }, + { + "expr": "go_memstats_heap_sys_bytes{job=\"tidb\"}", + "legendFormat": "HeapSys-{{instance}}", + "format": "time_series", + "intervalFactor": 1, + "refId": "B", + "hide": true + }, + { + "expr": "go_memstats_heap_inuse_bytes{job=\"tidb\"}", + "legendFormat": "HeapInuse-{{instance}}", + "format": "time_series", + "intervalFactor": 1, + "refId": "C" + }, { "expr": "go_memstats_heap_alloc_bytes{job=\"tidb\"}", + "legendFormat": "HeapAlloc-{{instance}}", "format": "time_series", "intervalFactor": 1, - "legendFormat": "heap-{{instance}}", - "refId": "B" + "refId": "D", + "hide": true + }, + { + "expr": "go_memstats_heap_idle_bytes{job=\"tidb\"}", + "legendFormat": "HeapIdle-{{instance}}", + "format": "time_series", + "intervalFactor": 1, + "refId": "E", + "hide": true + }, + { + "expr": "go_memstats_heap_released_bytes{job=\"tidb\"}", + "legendFormat": "HeapReleased-{{instance}}", + "interval": "", + "refId": "F", + "hide": true } ], "thresholds": [], @@ -10334,7 +10364,7 @@ "h": 7, "w": 12, "x": 0, - "y": 151 + "y": 14 }, "id": 50, "legend": { @@ -10422,7 +10452,7 @@ "h": 7, "w": 12, "x": 12, - "y": 151 + "y": 14 }, "id": 51, "legend": { @@ -10517,7 +10547,7 @@ "h": 7, "w": 12, "x": 0, - "y": 158 + "y": 21 }, "id": 164, "legend": { @@ -10604,7 +10634,7 @@ "h": 7, "w": 12, "x": 12, - "y": 158 + "y": 21 }, "id": 52, "legend": { @@ -10707,7 +10737,7 @@ "h": 7, "w": 8, "x": 0, - "y": 152 + "y": 15 }, "id": 85, "legend": { @@ -10796,7 +10826,7 @@ "h": 7, "w": 8, "x": 8, - "y": 152 + "y": 15 }, "id": 86, "legend": { @@ -10885,7 +10915,7 @@ "h": 7, "w": 8, "x": 16, - "y": 152 + "y": 15 }, "id": 87, "legend": { @@ -10972,7 +11002,7 @@ "h": 7, "w": 8, "x": 0, - "y": 159 + "y": 22 }, "id": 88, "legend": { @@ -11058,7 +11088,7 @@ "h": 7, "w": 8, "x": 8, - "y": 159 + "y": 22 }, "id": 158, "legend": { @@ -11146,7 +11176,7 @@ "h": 7, "w": 8, "x": 16, - "y": 159 + "y": 22 }, "id": 90, "legend": { @@ -11232,7 +11262,7 @@ "h": 7, "w": 8, "x": 0, - "y": 166 + "y": 29 }, "id": 89, "legend": { @@ -11321,7 +11351,7 @@ "h": 7, "w": 8, "x": 8, - "y": 166 + "y": 29 }, "id": 181, "legend": { @@ -11420,7 +11450,7 @@ "h": 7, "w": 8, "x": 16, - "y": 166 + "y": 29 }, "id": 182, "legend": { @@ -11528,7 +11558,7 @@ "h": 7, "w": 8, "x": 0, - "y": 153 + "y": 16 }, "id": 176, "legend": { @@ -11627,7 +11657,7 @@ "h": 7, "w": 8, "x": 8, - "y": 153 + "y": 16 }, "id": 179, "legend": { @@ -11705,6 +11735,185 @@ "alignLevel": null } }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "kv storage batch client wait new connection establish duration", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 16 + }, + "id": 204, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "null as zero", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.9999, sum(rate(tidb_tikvclient_batch_client_wait_connection_establish_bucket[1m])) by (le, instance))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Wait Connection Establish Duration", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Metrics for 'no available connection'.\nThere should be no data here if the connection between TiDB and TiKV is healthy.", + "fill": 1, + "gridPos": { + "h": 7, + "w": 16, + "x": 0, + "y": 23 + }, + "id": 203, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "tidb_tikvclient_batch_client_no_available_connection_total", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "No Available Connection Counter", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { "aliasColors": {}, "bars": false, @@ -11720,7 +11929,7 @@ "h": 7, "w": 8, "x": 16, - "y": 153 + "y": 23 }, "id": 180, "legend": { @@ -11750,6 +11959,7 @@ { "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_batch_client_unavailable_seconds_bucket[1m])) by (le, instance))", "format": "time_series", + "hide": false, "intervalFactor": 2, "legendFormat": "{{instance}}", "refId": "A", @@ -11804,7 +12014,7 @@ } ], "refresh": "30s", - "schemaVersion": 18, + "schemaVersion": 20, "style": "dark", "tags": [], "templating": { @@ -11842,5 +12052,5 @@ "timezone": "browser", "title": "Test-Cluster-TiDB", "uid": "000000011", - "version": 5 -} \ No newline at end of file + "version": 3 +} diff --git a/metrics/grafana/tidb_summary.json b/metrics/grafana/tidb_summary.json index d16191884cb75..4862094826d0c 100644 --- a/metrics/grafana/tidb_summary.json +++ b/metrics/grafana/tidb_summary.json @@ -417,17 +417,47 @@ { "expr": "process_resident_memory_bytes{job=\"tidb\"}", "format": "time_series", - "hide": false, "intervalFactor": 1, "legendFormat": "process-{{instance}}", "refId": "A" }, + { + "expr": "go_memstats_heap_sys_bytes{job=\"tidb\"}", + "legendFormat": "HeapSys-{{instance}}", + "format": "time_series", + "intervalFactor": 1, + "refId": "B", + "hide": true + }, + { + "expr": "go_memstats_heap_inuse_bytes{job=\"tidb\"}", + "legendFormat": "HeapInuse-{{instance}}", + "format": "time_series", + "intervalFactor": 1, + "refId": "C" + }, { "expr": "go_memstats_heap_alloc_bytes{job=\"tidb\"}", + "legendFormat": "HeapAlloc-{{instance}}", "format": "time_series", "intervalFactor": 1, - "legendFormat": "heap-{{instance}}", - "refId": "B" + "refId": "D", + "hide": true + }, + { + "expr": "go_memstats_heap_idle_bytes{job=\"tidb\"}", + "legendFormat": "HeapIdle-{{instance}}", + "format": "time_series", + "intervalFactor": 1, + "refId": "E", + "hide": true + }, + { + "expr": "go_memstats_heap_released_bytes{job=\"tidb\"}", + "legendFormat": "HeapReleased-{{instance}}", + "interval": "", + "refId": "F", + "hide": true } ], "thresholds": [], @@ -3048,4 +3078,4 @@ "title": "Test-Cluster-TiDB-Summary", "uid": "000000012", "version": 1 -} \ No newline at end of file +} diff --git a/metrics/meta.go b/metrics/meta.go index 3b305247cbfe5..de587662947f7 100644 --- a/metrics/meta.go +++ b/metrics/meta.go @@ -28,7 +28,7 @@ var ( Subsystem: "autoid", Name: "operation_duration_seconds", Help: "Bucketed histogram of processing time (s) of handled autoid.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 500us ~ 2097s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 0.5ms ~ 1048s }, []string{LblType, LblResult}) GetSchemaDiff = "get_schema_diff" @@ -43,6 +43,6 @@ var ( Subsystem: "meta", Name: "operation_duration_seconds", Help: "Bucketed histogram of processing time (s) of tidb meta data operations.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 500us ~ 2097s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 0.5ms ~ 1048s }, []string{LblType, LblResult}) ) diff --git a/metrics/metrics.go b/metrics/metrics.go index 183203f031153..f00d1aeae68aa 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -43,7 +43,7 @@ const ( opSucc = "ok" opFailed = "err" - LableScope = "scope" + LabelScope = "scope" ScopeGlobal = "global" ScopeSession = "session" ) @@ -76,7 +76,7 @@ func RegisterMetrics() { prometheus.MustRegister(DDLWorkerHistogram) prometheus.MustRegister(DeploySyncerHistogram) prometheus.MustRegister(DistSQLPartialCountHistogram) - prometheus.MustRegister(DistSQLQueryHistgram) + prometheus.MustRegister(DistSQLQueryHistogram) prometheus.MustRegister(DistSQLScanKeysHistogram) prometheus.MustRegister(DistSQLScanKeysPartialHistogram) prometheus.MustRegister(DumpFeedbackCounter) @@ -149,8 +149,11 @@ func RegisterMetrics() { prometheus.MustRegister(TotalCopProcHistogram) prometheus.MustRegister(TotalCopWaitHistogram) prometheus.MustRegister(TiKVPendingBatchRequests) + prometheus.MustRegister(TiKVStatusDuration) + prometheus.MustRegister(TiKVStatusCounter) prometheus.MustRegister(TiKVBatchWaitDuration) prometheus.MustRegister(TiKVBatchClientUnavailable) + prometheus.MustRegister(TiKVBatchClientWaitEstablish) prometheus.MustRegister(TiKVRangeTaskStats) prometheus.MustRegister(TiKVRangeTaskPushDuration) prometheus.MustRegister(HandleSchemaValidate) @@ -159,4 +162,5 @@ func RegisterMetrics() { prometheus.MustRegister(TiKVPessimisticLockKeysDuration) prometheus.MustRegister(GRPCConnTransientFailureCounter) prometheus.MustRegister(TiKVTTLLifeTimeReachCounter) + prometheus.MustRegister(TiKVNoAvailableConnectionCounter) } diff --git a/metrics/oracles.go b/metrics/oracles.go index ab657af46360d..88940c2fcfb9b 100644 --- a/metrics/oracles.go +++ b/metrics/oracles.go @@ -25,6 +25,6 @@ var ( Subsystem: "pdclient", Name: "ts_future_wait_seconds", Help: "Bucketed histogram of seconds cost for waiting timestamp future.", - Buckets: prometheus.ExponentialBuckets(0.000005, 2, 20), // 5us ~ 5s + Buckets: prometheus.ExponentialBuckets(0.000005, 2, 24), // 5us ~ 40s }) ) diff --git a/metrics/owner.go b/metrics/owner.go index b5318be12239b..595fce11b42a8 100644 --- a/metrics/owner.go +++ b/metrics/owner.go @@ -25,7 +25,7 @@ var ( Subsystem: "owner", Name: "new_session_duration_seconds", Help: "Bucketed histogram of processing time (s) of new session.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 500us ~ 2097s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 0.5ms ~ 1048s }, []string{LblType, LblResult}) WatcherClosed = "watcher_closed" diff --git a/metrics/server.go b/metrics/server.go index 2fc36748bd898..cf8fc95fe8214 100644 --- a/metrics/server.go +++ b/metrics/server.go @@ -34,7 +34,7 @@ var ( Subsystem: "server", Name: "handle_query_duration_seconds", Help: "Bucketed histogram of processing time (s) of handled queries.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 500us ~ 2097s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 22), // 0.5ms ~ 1048s }, []string{LblSQLType}) QueryTotalCounter = prometheus.NewCounterVec( @@ -128,7 +128,7 @@ var ( Subsystem: "server", Name: "get_token_duration_seconds", Help: "Duration (us) for getting token, it should be small until concurrency limit is reached.", - Buckets: prometheus.ExponentialBuckets(1, 2, 26), // 1us ~ 67s + Buckets: prometheus.ExponentialBuckets(1, 2, 26), // 1us ~ 33s }) TotalQueryProcHistogram = prometheus.NewHistogram( @@ -137,7 +137,7 @@ var ( Subsystem: "server", Name: "slow_query_process_duration_seconds", Help: "Bucketed histogram of processing time (s) of of slow queries.", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 4096s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 2048s }) TotalCopProcHistogram = prometheus.NewHistogram( prometheus.HistogramOpts{ @@ -145,7 +145,7 @@ var ( Subsystem: "server", Name: "slow_query_cop_duration_seconds", Help: "Bucketed histogram of all cop processing time (s) of of slow queries.", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 4096s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 2048s }) TotalCopWaitHistogram = prometheus.NewHistogram( prometheus.HistogramOpts{ @@ -153,7 +153,7 @@ var ( Subsystem: "server", Name: "slow_query_wait_duration_seconds", Help: "Bucketed histogram of all cop waiting time (s) of of slow queries.", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 4096s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 2048s }) ) diff --git a/metrics/session.go b/metrics/session.go index 3c661af213937..d80d1b5eb7da3 100644 --- a/metrics/session.go +++ b/metrics/session.go @@ -23,7 +23,7 @@ var ( Subsystem: "session", Name: "parse_duration_seconds", Help: "Bucketed histogram of processing time (s) in parse SQL.", - Buckets: prometheus.ExponentialBuckets(0.00004, 2, 22), // 40us ~ 168s + Buckets: prometheus.ExponentialBuckets(0.00004, 2, 22), // 40us ~ 84s }, []string{LblSQLType}) SessionExecuteCompileDuration = prometheus.NewHistogramVec( prometheus.HistogramOpts{ @@ -32,7 +32,7 @@ var ( Name: "compile_duration_seconds", Help: "Bucketed histogram of processing time (s) in query optimize.", // Build plan may execute the statement, or allocate table ID, so it might take a long time. - Buckets: prometheus.ExponentialBuckets(0.00004, 2, 22), // 40us ~ 168s + Buckets: prometheus.ExponentialBuckets(0.00004, 2, 22), // 40us ~ 84s }, []string{LblSQLType}) SessionExecuteRunDuration = prometheus.NewHistogramVec( prometheus.HistogramOpts{ @@ -40,7 +40,7 @@ var ( Subsystem: "session", Name: "execute_duration_seconds", Help: "Bucketed histogram of processing time (s) in running executor.", - Buckets: prometheus.ExponentialBuckets(0.0001, 2, 22), // 100us ~ 419s + Buckets: prometheus.ExponentialBuckets(0.0001, 2, 22), // 100us ~ 209s }, []string{LblSQLType}) SchemaLeaseErrorCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ @@ -55,7 +55,7 @@ var ( Subsystem: "session", Name: "retry_num", Help: "Bucketed histogram of session retry count.", - Buckets: prometheus.LinearBuckets(0, 1, 20), // 0 ~ 20 + Buckets: prometheus.LinearBuckets(0, 1, 21), // 0 ~ 20 }) SessionRetryErrorCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ @@ -79,7 +79,7 @@ var ( Subsystem: "session", Name: "transaction_statement_num", Help: "Bucketed histogram of statements count in each transaction.", - Buckets: prometheus.ExponentialBuckets(1, 2, 16), // 1 ~ 65536 + Buckets: prometheus.ExponentialBuckets(1, 2, 16), // 1 ~ 32768 }, []string{LblSQLType, LblType}) TransactionDuration = prometheus.NewHistogramVec( @@ -88,7 +88,7 @@ var ( Subsystem: "session", Name: "transaction_duration_seconds", Help: "Bucketed histogram of a transaction execution duration, including retry.", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 1049s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, []string{LblSQLType, LblType}) StatementDeadlockDetectDuration = prometheus.NewHistogram( @@ -97,7 +97,7 @@ var ( Subsystem: "session", Name: "statement_deadlock_detect_duration_seconds", Help: "Bucketed histogram of a statement deadlock detect duration.", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 1049s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, ) @@ -107,7 +107,7 @@ var ( Subsystem: "session", Name: "statement_pessimistic_retry_count", Help: "Bucketed histogram of statement pessimistic retry count", - Buckets: prometheus.ExponentialBuckets(1, 1.5, 14), // 1 ~ 291 + Buckets: prometheus.ExponentialBuckets(1, 2, 16), // 1 ~ 32768 }) StatementLockKeysCount = prometheus.NewHistogram( @@ -116,7 +116,7 @@ var ( Subsystem: "session", Name: "statement_lock_keys_count", Help: "Keys locking for a single statement", - Buckets: prometheus.ExponentialBuckets(1, 2, 21), // 1 ~ 2097152 + Buckets: prometheus.ExponentialBuckets(1, 2, 21), // 1 ~ 1048576 }) ) diff --git a/metrics/stats.go b/metrics/stats.go index d9524a7c3ee1a..2d74caa4bf77b 100644 --- a/metrics/stats.go +++ b/metrics/stats.go @@ -25,7 +25,7 @@ var ( Subsystem: "statistics", Name: "auto_analyze_duration_seconds", Help: "Bucketed histogram of processing time (s) of auto analyze.", - Buckets: prometheus.ExponentialBuckets(0.01, 2, 20), // 10ms ~ 3hours + Buckets: prometheus.ExponentialBuckets(0.01, 2, 20), // 10ms ~ 1.5hours }) AutoAnalyzeCounter = prometheus.NewCounterVec( diff --git a/metrics/tikvclient.go b/metrics/tikvclient.go index 1a34a3377982d..2021273162345 100644 --- a/metrics/tikvclient.go +++ b/metrics/tikvclient.go @@ -23,7 +23,7 @@ var ( Subsystem: "tikvclient", Name: "txn_cmd_duration_seconds", Help: "Bucketed histogram of processing time of txn cmds.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 524s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 262s }, []string{LblType}) TiKVBackoffHistogram = prometheus.NewHistogramVec( @@ -32,7 +32,7 @@ var ( Subsystem: "tikvclient", Name: "backoff_seconds", Help: "total backoff seconds of a single backoffer.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 524s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 262s }, []string{LblType}) TiKVSendReqHistogram = prometheus.NewHistogramVec( @@ -41,7 +41,7 @@ var ( Subsystem: "tikvclient", Name: "request_seconds", Help: "Bucketed histogram of sending request duration.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 524s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 262s }, []string{LblType, LblStore}) TiKVCoprocessorHistogram = prometheus.NewHistogram( @@ -50,7 +50,7 @@ var ( Subsystem: "tikvclient", Name: "cop_duration_seconds", Help: "Run duration of a single coprocessor task, includes backoff time.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 524s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 262s }) TiKVLockResolverCounter = prometheus.NewCounterVec( @@ -75,7 +75,7 @@ var ( Subsystem: "tikvclient", Name: "txn_write_kv_num", Help: "Count of kv pairs to write in a transaction.", - Buckets: prometheus.ExponentialBuckets(1, 2, 21), // 1 ~ 2097152 + Buckets: prometheus.ExponentialBuckets(1, 2, 21), // 1 ~ 1048576 }) TiKVTxnWriteSizeHistogram = prometheus.NewHistogram( @@ -84,7 +84,7 @@ var ( Subsystem: "tikvclient", Name: "txn_write_size_bytes", Help: "Size of kv pairs to write in a transaction.", - Buckets: prometheus.ExponentialBuckets(1, 2, 30), // 1Byte ~ 1GB + Buckets: prometheus.ExponentialBuckets(1, 2, 30), // 1Byte ~ 500MB }) TiKVRawkvCmdHistogram = prometheus.NewHistogramVec( @@ -93,7 +93,7 @@ var ( Subsystem: "tikvclient", Name: "rawkv_cmd_seconds", Help: "Bucketed histogram of processing time of rawkv cmds.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 524s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 262s }, []string{LblType}) TiKVRawkvSizeHistogram = prometheus.NewHistogramVec( @@ -102,7 +102,7 @@ var ( Subsystem: "tikvclient", Name: "rawkv_kv_size_bytes", Help: "Size of key/value to put, in bytes.", - Buckets: prometheus.ExponentialBuckets(1, 2, 23), // 1Byte ~ 8MB + Buckets: prometheus.ExponentialBuckets(1, 2, 24), // 1Byte ~ 8MB }, []string{LblType}) TiKVTxnRegionsNumHistogram = prometheus.NewHistogramVec( @@ -144,7 +144,7 @@ var ( Subsystem: "tikvclient", Name: "local_latch_wait_seconds", Help: "Wait time of a get local latch.", - Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 524s + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 262s }) // TiKVPendingBatchRequests indicates the number of requests pending in the batch channel. @@ -156,12 +156,29 @@ var ( Help: "Pending batch requests", }, []string{"store"}) + TiKVStatusDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "tikvclient", + Name: "kv_status_api_duration", + Help: "duration for kv status api.", + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 20), // 0.5ms ~ 262s + }, []string{"store"}) + + TiKVStatusCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "tikvclient", + Name: "kv_status_api_count", + Help: "Counter of access kv status api.", + }, []string{LblResult}) + TiKVBatchWaitDuration = prometheus.NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "tikvclient", Name: "batch_wait_duration", - Buckets: prometheus.ExponentialBuckets(1, 2, 30), // 1ns ~ 1s + Buckets: prometheus.ExponentialBuckets(1, 2, 34), // 1ns ~ 8s Help: "batch wait duration", }) @@ -170,9 +187,17 @@ var ( Namespace: "tidb", Subsystem: "tikvclient", Name: "batch_client_unavailable_seconds", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 1000s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s Help: "batch client unavailable", }) + TiKVBatchClientWaitEstablish = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "tikvclient", + Name: "batch_client_wait_connection_establish", + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s + Help: "batch client wait new connection establish", + }) TiKVRangeTaskStats = prometheus.NewGaugeVec( prometheus.GaugeOpts{ @@ -187,16 +212,15 @@ var ( Namespace: "tidb", Subsystem: "tikvclient", Name: "range_task_push_duration", - // 1ms ~ 1000s - Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), - Help: "duration to push sub tasks to range task workers", + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s + Help: "duration to push sub tasks to range task workers", }, []string{LblType}) TiKVTokenWaitDuration = prometheus.NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "tikvclient", Name: "batch_executor_token_wait_duration", - Buckets: prometheus.ExponentialBuckets(1, 2, 30), // 1ns ~ 1s + Buckets: prometheus.ExponentialBuckets(1, 2, 34), // 1ns ~ 8s Help: "tidb txn token wait duration to process batches", }) @@ -206,14 +230,14 @@ var ( Subsystem: "tikvclient", Name: "txn_heart_beat", Help: "Bucketed histogram of the txn_heartbeat request duration.", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 18), // 1ms ~ 292s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }, []string{LblType}) TiKVPessimisticLockKeysDuration = prometheus.NewHistogram( prometheus.HistogramOpts{ Namespace: "tidb", Subsystem: "tikvclient", Name: "pessimistic_lock_keys_duration", - Buckets: prometheus.ExponentialBuckets(0.001, 2, 24), // 1ms ~ 16777s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 24), // 1ms ~ 8389s Help: "tidb txn pessimistic lock keys duration", }) @@ -224,4 +248,12 @@ var ( Name: "ttl_lifetime_reach_total", Help: "Counter of ttlManager live too long.", }) + + TiKVNoAvailableConnectionCounter = prometheus.NewCounter( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "tikvclient", + Name: "batch_client_no_available_connection_total", + Help: "Counter of no available batch client.", + }) ) diff --git a/owner/fail_test.go b/owner/fail_test.go index b25be78e99704..577ca1be7f049 100644 --- a/owner/fail_test.go +++ b/owner/fail_test.go @@ -55,7 +55,7 @@ func (s *testSuite) TearDownSuite(c *C) { var ( endpoints = []string{"unix://new_session:12379"} dialTimeout = 5 * time.Second - retryCnt = int(math.MaxInt32) + retryCnt = math.MaxInt32 ) func (s *testSuite) TestFailNewSession(c *C) { diff --git a/planner/cascades/testdata/integration_suite_in.json b/planner/cascades/testdata/integration_suite_in.json index 547c2cee8e486..4a19d038c02ab 100644 --- a/planner/cascades/testdata/integration_suite_in.json +++ b/planner/cascades/testdata/integration_suite_in.json @@ -152,7 +152,8 @@ "select 1 from (select /*+ HASH_JOIN(t1) */ t1.a not in (select t2.a from t2) from t1) x;", // TODO: should use hash join "select /*+ INL_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", "select /*+ INL_HASH_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "select /*+ INL_MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;" + "select /*+ INL_MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "select /*+ MERGE_JOIN(t1, t2) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;" ] } ] diff --git a/planner/cascades/testdata/integration_suite_out.json b/planner/cascades/testdata/integration_suite_out.json index 28155a0a22cc0..bd93d58565b25 100644 --- a/planner/cascades/testdata/integration_suite_out.json +++ b/planner/cascades/testdata/integration_suite_out.json @@ -288,7 +288,7 @@ "Plan": [ "HashAgg_14 1.00 root funcs:avg(Column#6, Column#7)->Column#5", "└─TableReader_15 1.00 root data:HashAgg_16", - " └─HashAgg_16 1.00 cop[tikv] funcs:avg(test.t.b)->Column#6", + " └─HashAgg_16 1.00 cop[tikv] funcs:count(test.t.b)->Column#6, funcs:sum(test.t.b)->Column#7", " └─TableFullScan_12 10000.00 cop[tikv] table:a keep order:false, stats:pseudo" ], "Result": [ @@ -489,7 +489,7 @@ "Projection_8 8000.00 root Column#5, test.t.c, Column#5, Column#6, Column#7, Column#8, Column#9", "└─HashAgg_13 8000.00 root group by:test.t.c, funcs:avg(Column#11, Column#12)->Column#5, funcs:count(distinct test.t.a, test.t.b)->Column#6, funcs:count(distinct test.t.a)->Column#7, funcs:count(distinct test.t.c)->Column#8, funcs:sum(Column#13)->Column#9, funcs:firstrow(test.t.c)->test.t.c", " └─TableReader_14 8000.00 root data:HashAgg_15", - " └─HashAgg_15 8000.00 cop[tikv] group by:test.t.a, test.t.b, test.t.c, funcs:avg(test.t.b)->Column#11, funcs:sum(test.t.b)->Column#12", + " └─HashAgg_15 8000.00 cop[tikv] group by:test.t.a, test.t.b, test.t.c, funcs:count(test.t.b)->Column#11, funcs:sum(test.t.b)->Column#12, funcs:sum(test.t.b)->Column#13", " └─TableFullScan_11 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Result": [ @@ -1281,6 +1281,19 @@ "Result": [ "1 1" ] + }, + { + "SQL": "select /*+ MERGE_JOIN(t1, t2) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "Plan": [ + "HashJoin_9 12500.00 root inner join, equal:[eq(test.t1.a, test.t2.a)]", + "├─TableReader_12(Build) 10000.00 root data:TableFullScan_13", + "│ └─TableFullScan_13 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader_14(Probe) 10000.00 root data:TableFullScan_15", + " └─TableFullScan_15 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Result": [ + "1 1" + ] } ] } diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 530f2733ce670..9296bd9c468d0 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -436,7 +436,7 @@ func (r *PushAggDownGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.Gr AggFuncs: aggFuncs, GroupByItems: gbyItems, Schema: aggSchema, - }) + }, true) // Remove unnecessary FirstRow. partialPref.AggFuncs = plannercore.RemoveUnnecessaryFirstRow(agg.SCtx(), finalPref.AggFuncs, finalPref.GroupByItems, partialPref.AggFuncs, partialPref.GroupByItems, partialPref.Schema, funcMap) @@ -2005,9 +2005,9 @@ func (r *TransformAggregateCaseToSelection) transform(agg *plannercore.LogicalAg caseArgsNum := len(caseArgs) // `case when a>0 then null else a end` should be converted to `case when !(a>0) then a else null end`. - var nullFlip = caseArgsNum == 3 && caseArgs[1].Equal(ctx, expression.Null) && !caseArgs[2].Equal(ctx, expression.Null) + var nullFlip = caseArgsNum == 3 && caseArgs[1].Equal(ctx, expression.NewNull()) && !caseArgs[2].Equal(ctx, expression.NewNull()) // `case when a>0 then 0 else a end` should be converted to `case when !(a>0) then a else 0 end`. - var zeroFlip = !nullFlip && caseArgsNum == 3 && caseArgs[1].Equal(ctx, expression.Zero) + var zeroFlip = !nullFlip && caseArgsNum == 3 && caseArgs[1].Equal(ctx, expression.NewZero()) var outputIdx int if nullFlip || zeroFlip { @@ -2040,8 +2040,8 @@ func (r *TransformAggregateCaseToSelection) transform(agg *plannercore.LogicalAg // => newAggFuncDesc: SUM(cnt), newCondition: x = 'foo' switch { - case r.allowsSelection(aggFuncName) && (caseArgsNum == 2 || caseArgs[3-outputIdx].Equal(ctx, expression.Null)), // Case A1 - aggFuncName == ast.AggFuncSum && caseArgsNum == 3 && caseArgs[3-outputIdx].Equal(ctx, expression.Zero): // Case A2 + case r.allowsSelection(aggFuncName) && (caseArgsNum == 2 || caseArgs[3-outputIdx].Equal(ctx, expression.NewNull())), // Case A1 + aggFuncName == ast.AggFuncSum && caseArgsNum == 3 && caseArgs[3-outputIdx].Equal(ctx, expression.NewZero()): // Case A2 newAggFuncDesc := aggFuncDesc.Clone() newAggFuncDesc.Args = []expression.Expression{caseArgs[outputIdx]} return true, newConditions, []*aggregation.AggFuncDesc{newAggFuncDesc} @@ -2055,7 +2055,7 @@ func (r *TransformAggregateCaseToSelection) allowsSelection(aggFuncName string) } func (r *TransformAggregateCaseToSelection) isOnlyOneNotNull(ctx sessionctx.Context, args []expression.Expression, argsNum int, outputIdx int) bool { - return !args[outputIdx].Equal(ctx, expression.Null) && (argsNum == 2 || args[3-outputIdx].Equal(ctx, expression.Null)) + return !args[outputIdx].Equal(ctx, expression.NewNull()) && (argsNum == 2 || args[3-outputIdx].Equal(ctx, expression.NewNull())) } // TransformAggregateCaseToSelection only support `case when cond then var end` and `case when cond then var1 else var2 end`. diff --git a/planner/core/cache.go b/planner/core/cache.go index a1ddb90eedd6d..c793a635674ad 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -37,9 +37,9 @@ var ( // PreparedPlanCacheCapacity stores the global config "prepared-plan-cache-capacity". PreparedPlanCacheCapacity uint = 100 // PreparedPlanCacheMemoryGuardRatio stores the global config "prepared-plan-cache-memory-guard-ratio". - PreparedPlanCacheMemoryGuardRatio float64 = 0.1 + PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory stores the max memory size defined in the global config "performance-max-memory". - PreparedPlanCacheMaxMemory atomic2.Uint64 = *atomic2.NewUint64(math.MaxUint64) + PreparedPlanCacheMaxMemory = *atomic2.NewUint64(math.MaxUint64) ) const ( diff --git a/planner/core/cacheable_checker.go b/planner/core/cacheable_checker.go index 7369c9ac1cfec..d6c3161417d11 100644 --- a/planner/core/cacheable_checker.go +++ b/planner/core/cacheable_checker.go @@ -16,13 +16,16 @@ package core import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" ) // Cacheable checks whether the input ast is cacheable. // Handle "ignore_plan_cache()" hint // If there are multiple hints, only one will take effect -func Cacheable(node ast.Node) bool { +func Cacheable(node ast.Node, is infoschema.InfoSchema) bool { switch node.(type) { case *ast.SelectStmt: for _, hints := range (node.(*ast.SelectStmt)).TableHints { @@ -48,6 +51,7 @@ func Cacheable(node ast.Node) bool { } checker := cacheableChecker{ cacheable: true, + schema: is, } node.Accept(&checker) return checker.cacheable @@ -60,6 +64,7 @@ func Cacheable(node ast.Node) bool { // NOTE: we can add more rules in the future. type cacheableChecker struct { cacheable bool + schema infoschema.InfoSchema } // Enter implements Visitor interface. @@ -105,10 +110,27 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren checker.cacheable = false return in, true } + case *ast.TableName: + if checker.isPartitionTable(node) { + checker.cacheable = false + return in, true + } } return in, false } +func (checker *cacheableChecker) isPartitionTable(tn *ast.TableName) bool { + tb, err := checker.schema.TableByName(tn.Schema, tn.Name) + if err != nil { + logutil.BgLogger().Error("Error occur in checking cacheable", zap.Error(err)) + return false + } + if tb.Meta().Partition != nil { + return true + } + return false +} + // Leave implements Visitor interface. func (checker *cacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) { return in, checker.cacheable diff --git a/planner/core/cacheable_checker_test.go b/planner/core/cacheable_checker_test.go index d2db2d42392a9..97370f72cdf91 100644 --- a/planner/core/cacheable_checker_test.go +++ b/planner/core/cacheable_checker_test.go @@ -11,14 +11,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -package core +package core_test import ( . "github.com/pingcap/check" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/util/testkit" ) var _ = Suite(&testCacheableSuite{}) @@ -27,20 +30,33 @@ type testCacheableSuite struct { } func (s *testCacheableSuite) TestCacheable(c *C) { + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + defer func() { + dom.Close() + store.Close() + }() + tk.MustExec("use test") + tk.MustExec("create table t1(a int, b int) partition by range(a) ( partition p0 values less than (6), partition p1 values less than (11) )") + tk.MustExec("create table t2(a int, b int) partition by hash(a) partitions 11") + tk.MustExec("create table t3(a int, b int)") + tbl := &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t3")} + is := infoschema.GetInfoSchema(tk.Se) // test non-SelectStmt/-InsertStmt/-DeleteStmt/-UpdateStmt/-SelectStmt var stmt ast.Node = &ast.UnionStmt{} - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) stmt = &ast.ShowStmt{} - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) stmt = &ast.LoadDataStmt{} - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) - tableRefsClause := &ast.TableRefsClause{TableRefs: &ast.Join{Left: &ast.TableSource{Source: &ast.TableName{}}}} + tableRefsClause := &ast.TableRefsClause{TableRefs: &ast.Join{Left: &ast.TableSource{Source: tbl}}} // test InsertStmt stmt = &ast.InsertStmt{Table: tableRefsClause} - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) // test DeleteStmt whereExpr := &ast.FuncCallExpr{} @@ -48,21 +64,21 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Where: whereExpr, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) for funcName := range expression.UnCacheableFunctions { whereExpr.FnName = model.NewCIStr(funcName) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) } whereExpr.FnName = model.NewCIStr(ast.Rand) - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt = &ast.DeleteStmt{ TableRefs: tableRefsClause, Where: &ast.ExistsSubqueryExpr{}, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt := &ast.Limit{ Count: &driver.ParamMarkerExpr{}, @@ -71,7 +87,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{ Offset: &driver.ParamMarkerExpr{}, @@ -80,19 +96,19 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{} stmt = &ast.DeleteStmt{ TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt.(*ast.DeleteStmt).TableHints = append(stmt.(*ast.DeleteStmt).TableHints, &ast.TableOptimizerHint{ - HintName: model.NewCIStr(HintIgnorePlanCache), + HintName: model.NewCIStr(core.HintIgnorePlanCache), }) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) // test UpdateStmt whereExpr = &ast.FuncCallExpr{} @@ -100,21 +116,21 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Where: whereExpr, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) for funcName := range expression.UnCacheableFunctions { whereExpr.FnName = model.NewCIStr(funcName) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) } whereExpr.FnName = model.NewCIStr(ast.Rand) - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt = &ast.UpdateStmt{ TableRefs: tableRefsClause, Where: &ast.ExistsSubqueryExpr{}, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{ Count: &driver.ParamMarkerExpr{}, @@ -123,7 +139,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{ Offset: &driver.ParamMarkerExpr{}, @@ -132,39 +148,39 @@ func (s *testCacheableSuite) TestCacheable(c *C) { TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{} stmt = &ast.UpdateStmt{ TableRefs: tableRefsClause, Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt.(*ast.UpdateStmt).TableHints = append(stmt.(*ast.UpdateStmt).TableHints, &ast.TableOptimizerHint{ - HintName: model.NewCIStr(HintIgnorePlanCache), + HintName: model.NewCIStr(core.HintIgnorePlanCache), }) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) // test SelectStmt whereExpr = &ast.FuncCallExpr{} stmt = &ast.SelectStmt{ Where: whereExpr, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) for funcName := range expression.UnCacheableFunctions { whereExpr.FnName = model.NewCIStr(funcName) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) } whereExpr.FnName = model.NewCIStr(ast.Rand) - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt = &ast.SelectStmt{ Where: &ast.ExistsSubqueryExpr{}, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{ Count: &driver.ParamMarkerExpr{}, @@ -172,7 +188,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { stmt = &ast.SelectStmt{ Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{ Offset: &driver.ParamMarkerExpr{}, @@ -180,33 +196,55 @@ func (s *testCacheableSuite) TestCacheable(c *C) { stmt = &ast.SelectStmt{ Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) limitStmt = &ast.Limit{} stmt = &ast.SelectStmt{ Limit: limitStmt, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) paramExpr := &driver.ParamMarkerExpr{} orderByClause := &ast.OrderByClause{Items: []*ast.ByItem{{Expr: paramExpr}}} stmt = &ast.SelectStmt{ OrderBy: orderByClause, } - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) valExpr := &driver.ValueExpr{} orderByClause = &ast.OrderByClause{Items: []*ast.ByItem{{Expr: valExpr}}} stmt = &ast.SelectStmt{ OrderBy: orderByClause, } - c.Assert(Cacheable(stmt), IsTrue) + c.Assert(core.Cacheable(stmt, is), IsTrue) stmt.(*ast.SelectStmt).TableHints = append(stmt.(*ast.SelectStmt).TableHints, &ast.TableOptimizerHint{ - HintName: model.NewCIStr(HintIgnorePlanCache), + HintName: model.NewCIStr(core.HintIgnorePlanCache), }) - c.Assert(Cacheable(stmt), IsFalse) + c.Assert(core.Cacheable(stmt, is), IsFalse) boundExpr := &ast.FrameBound{Expr: &driver.ParamMarkerExpr{}} - c.Assert(Cacheable(boundExpr), IsFalse) + c.Assert(core.Cacheable(boundExpr, is), IsFalse) + + // Partition table can not be cached. + join := &ast.Join{ + Left: &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t1")}, + Right: &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t2")}, + } + stmt = &ast.SelectStmt{ + From: &ast.TableRefsClause{ + TableRefs: join, + }, + } + c.Assert(core.Cacheable(stmt, is), IsFalse) + + join = &ast.Join{ + Left: &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t3")}, + } + stmt = &ast.SelectStmt{ + From: &ast.TableRefsClause{ + TableRefs: join, + }, + } + c.Assert(core.Cacheable(stmt, is), IsTrue) } diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 97b44f22fcff4..5a84c2a0ad2de 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -355,9 +355,14 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { testKit.MustExec("create view v as select * from t") _, err = testKit.Exec("analyze table v") - c.Assert(err.Error(), Equals, "analyze v is not supported now.") + c.Assert(err.Error(), Equals, "analyze view v is not supported now.") testKit.MustExec("drop view v") + testKit.MustExec("create sequence seq") + _, err = testKit.Exec("analyze table seq") + c.Assert(err.Error(), Equals, "analyze sequence seq is not supported now.") + testKit.MustExec("drop sequence seq") + var input, output []string s.testData.GetTestCases(c, &input, &output) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 1d38e37cb4573..35afbeace4877 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" @@ -263,6 +264,12 @@ func (e *Execute) checkPreparedPriv(ctx context.Context, sctx sessionctx.Context return err } +func (e *Execute) setFoundInPlanCache(sctx sessionctx.Context, opt bool) error { + vars := sctx.GetSessionVars() + err := vars.SetSystemVar(variable.TiDBFoundInPlanCache, variable.BoolToIntStr(opt)) + return err +} + func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *CachedPrepareStmt) error { stmtCtx := sctx.GetSessionVars().StmtCtx prepared := preparedStmt.PreparedAst @@ -282,6 +289,10 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, } else { planCacheCounter.Inc() } + err = e.setFoundInPlanCache(sctx, true) + if err != nil { + return err + } e.names = names e.Plan = plan stmtCtx.PointExec = true @@ -307,12 +318,16 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, } } if planValid { + err := e.setFoundInPlanCache(sctx, true) + if err != nil { + return err + } if metrics.ResettablePlanCacheCounterFortTest { metrics.PlanCacheCounter.WithLabelValues("prepare").Inc() } else { planCacheCounter.Inc() } - err := e.rebuildRange(cachedVal.Plan) + err = e.rebuildRange(cachedVal.Plan) if err != nil { return err } @@ -333,14 +348,18 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, } e.names = names e.Plan = p - isRange := e.isRangePartition(p) _, isTableDual := p.(*PhysicalTableDual) - if !isTableDual && prepared.UseCache && !isRange { + if !isTableDual && prepared.UseCache { + err = e.setFoundInPlanCache(sctx, true) + if err != nil { + return err + } cached := NewPSTMTPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan) preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) sctx.PreparedPlanCache().Put(cacheKey, cached) } + err = e.setFoundInPlanCache(sctx, false) return err } @@ -446,13 +465,15 @@ func (e *Execute) rebuildRange(p Plan) error { } case *PointGetPlan: if x.HandleParam != nil { - x.Handle, err = x.HandleParam.Datum.ToInt64(sc) + var iv int64 + iv, err = x.HandleParam.Datum.ToInt64(sc) if err != nil { return err } + x.Handle = kv.IntHandle(iv) if x.PartitionInfo != nil { num := x.TblInfo.Partition.Num - pos := math.Abs(x.Handle) % int64(num) + pos := math.Abs(iv) % int64(num) x.PartitionInfo = &x.TblInfo.Partition.Definitions[pos] } return nil @@ -471,10 +492,12 @@ func (e *Execute) rebuildRange(p Plan) error { case *BatchPointGetPlan: for i, param := range x.HandleParams { if param != nil { - x.Handles[i], err = param.Datum.ToInt64(sc) + var iv int64 + iv, err = param.Datum.ToInt64(sc) if err != nil { return err } + x.Handles[i] = kv.IntHandle(iv) } } for i, params := range x.IndexValueParams { @@ -510,36 +533,6 @@ func (e *Execute) rebuildRange(p Plan) error { return nil } -func checkRangePartitionInfo(pi *model.PartitionInfo) bool { - if pi != nil && pi.Type == model.PartitionTypeRange { - return true - } - return false -} - -// Prepare plan cache is not support query plan on range partition table. -func (e *Execute) isRangePartition(p Plan) bool { - isRange := false - switch x := p.(type) { - case *PhysicalTableReader: - ts := x.TablePlans[0].(*PhysicalTableScan) - return checkRangePartitionInfo(ts.Table.Partition) - case *PhysicalIndexLookUpReader: - is := x.IndexPlans[0].(*PhysicalIndexScan) - return checkRangePartitionInfo(is.Table.Partition) - case *PhysicalIndexReader: - is := x.IndexPlans[0].(*PhysicalIndexScan) - return checkRangePartitionInfo(is.Table.Partition) - case PhysicalPlan: - for _, child := range x.Children() { - if e.isRangePartition(child) { - isRange = true - } - } - } - return isRange -} - func (e *Execute) buildRangeForIndexScan(sctx sessionctx.Context, is *PhysicalIndexScan) ([]*ranger.Range, error) { if len(is.IdxCols) == 0 { return ranger.FullRange(), nil @@ -565,6 +558,16 @@ type Set struct { VarAssigns []*expression.VarAssignment } +// SetConfig represents a plan for set config stmt. +type SetConfig struct { + baseSchemaProducer + + Type string + Instance string + Name string + Value expression.Expression +} + // SQLBindOpType repreents the SQL bind type type SQLBindOpType int @@ -910,16 +913,10 @@ func (e *Explain) explainPlanInRowFormat(p Plan, taskType, driverSide, indent st err = e.explainPlanInRowFormat(x.indexPlan, "cop[tikv]", "(Build)", childIndent, false) err = e.explainPlanInRowFormat(x.tablePlan, "cop[tikv]", "(Probe)", childIndent, true) case *PhysicalIndexMergeReader: - for i, pchild := range x.partialPlans { - if x.tablePlan != nil || i < len(x.partialPlans)-1 { - err = e.explainPlanInRowFormat(pchild, "cop[tikv]", "(Build)", childIndent, false) - } else { - err = e.explainPlanInRowFormat(pchild, "cop[tikv]", "(Probe)", childIndent, true) - } - } - if x.tablePlan != nil { - err = e.explainPlanInRowFormat(x.tablePlan, "cop[tikv]", "(Probe)", childIndent, true) + for _, pchild := range x.partialPlans { + err = e.explainPlanInRowFormat(pchild, "cop[tikv]", "(Build)", childIndent, false) } + err = e.explainPlanInRowFormat(x.tablePlan, "cop[tikv]", "(Probe)", childIndent, true) case *Insert: if x.SelectPlan != nil { err = e.explainPlanInRowFormat(x.SelectPlan, "root", "", childIndent, true) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 25d0ef615cf4e..561bd0222d399 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -37,16 +37,16 @@ import ( "go.uber.org/zap" ) -func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { if prop.IsFlashOnlyProp() { - return nil + return nil, true } childProp := prop.Clone() us := PhysicalUnionScan{ Conditions: p.conditions, HandleCol: p.handleCol, }.Init(p.ctx, p.stats, p.blockOffset, childProp) - return []PhysicalPlan{us} + return []PhysicalPlan{us}, true } func getMaxSortPrefix(sortCols, allCols []*expression.Column) []int { @@ -1302,25 +1302,9 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ defer func() { // refine error message - if !canForced && needForced { - if hasINLMJHint && len(indexJoins) > 0 && len(prop.Items) > 0 { - containIdxMergeJoin := false - for _, idxJoin := range indexJoins { - if _, ok := idxJoin.(*PhysicalIndexMergeJoin); ok { - containIdxMergeJoin = true - break - } - } - // 1. IndexMergeJoin requires stricter conditions than Index(Hash)Join when the output order is needed. - // 2. IndexMergeJoin requires the same conditions with Index(Hash)Join when the output is unordered. - // 3. If ordered-Index(Hash)Join can be chosen but ordered-IndexMergeJoin can not be chosen, we can build a plan with an enforced sort on IndexMergeJoin. - // 4. Thus we can give up the plans here if IndexMergeJoin is nil when `hasINLMJHint` is true. Because we can make sure that an IndexMeregJoin with enforced sort will be built. - if !containIdxMergeJoin { - canForced = true - indexJoins = nil - return - } - } + // If the required property is not empty, we will enforce it and try the hint again. + // So we only need to generate warning message when the property is empty. + if !canForced && needForced && prop.IsEmpty() { // Construct warning message prefix. var errMsg string switch { @@ -1434,49 +1418,55 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ // Firstly we check the hint, if hint is figured by user, we force to choose the corresponding physical plan. // If the hint is not matched, it will get other candidates. // If the hint is not figured, we will pick all candidates. -func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { if val.(bool) { indexJoins, _ := p.tryToGetIndexJoin(prop) - failpoint.Return(indexJoins) + failpoint.Return(indexJoins, true) } }) if prop.IsFlashOnlyProp() && ((p.preferJoinType&preferMergeJoin) > 0 || (p.preferJoinType&preferHashJoin) > 0) { - return nil + return nil, false } joins := make([]PhysicalPlan, 0, 5) if p.ctx.GetSessionVars().AllowBCJ { broadCastJoins := p.tryToGetBroadCastJoin(prop) if (p.preferJoinType & preferBCJoin) > 0 { logutil.BgLogger().Info("prefer bc join", zap.Int("bc count", len(broadCastJoins))) - return broadCastJoins + return broadCastJoins, true } joins = append(joins, broadCastJoins...) } if prop.IsFlashOnlyProp() { - return joins + return joins, true } mergeJoins := p.GetMergeJoin(prop, p.schema, p.Stats(), p.children[0].statsInfo(), p.children[1].statsInfo()) - if (p.preferJoinType & preferMergeJoin) > 0 { - return mergeJoins + if (p.preferJoinType&preferMergeJoin) > 0 && len(mergeJoins) > 0 { + return mergeJoins, true } joins = append(joins, mergeJoins...) indexJoins, forced := p.tryToGetIndexJoin(prop) if forced { - return indexJoins + return indexJoins, true } joins = append(joins, indexJoins...) hashJoins := p.getHashJoins(prop) - if (p.preferJoinType & preferHashJoin) > 0 { - logutil.BgLogger().Info("prefer hash join") - return hashJoins + if (p.preferJoinType&preferHashJoin) > 0 && len(hashJoins) > 0 { + return hashJoins, true } joins = append(joins, hashJoins...) - return joins + + if p.preferJoinType > 0 { + // If we reach here, it means we have a hint that doesn't work. + // It might be affected by the required property, so we enforce + // this property and try the hint again. + return joins, false + } + return joins, true } func getAllDataSourceTotalRowSize(plan LogicalPlan) float64 { @@ -1570,10 +1560,10 @@ func (p *LogicalProjection) TryToGetChildProp(prop *property.PhysicalProperty) ( return newProp, true } -func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { newProp, ok := p.TryToGetChildProp(prop) if !ok { - return nil + return nil, true } proj := PhysicalProjection{ Exprs: p.Exprs, @@ -1581,7 +1571,7 @@ func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty AvoidColumnEvaluator: p.AvoidColumnEvaluator, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, newProp) proj.SetSchema(p.schema) - return []PhysicalPlan{proj} + return []PhysicalPlan{proj}, true } func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPlan { @@ -1631,11 +1621,11 @@ func MatchItems(p *property.PhysicalProperty, items []*ByItems) bool { return true } -func (lt *LogicalTopN) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (lt *LogicalTopN) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { if MatchItems(prop, lt.ByItems) { - return append(lt.getPhysTopN(prop), lt.getPhysLimits(prop)...) + return append(lt.getPhysTopN(prop), lt.getPhysLimits(prop)...), true } - return nil + return nil, true } // GetHashJoin is public for cascades planner. @@ -1643,9 +1633,9 @@ func (la *LogicalApply) GetHashJoin(prop *property.PhysicalProperty) *PhysicalHa return la.LogicalJoin.getHashJoin(prop, 1, false) } -func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { if !prop.AllColsFromSchema(la.children[0].Schema()) || prop.IsFlashOnlyProp() { // for convenient, we don't pass through any prop - return nil + return nil, true } join := la.GetHashJoin(prop) apply := PhysicalApply{ @@ -1657,19 +1647,19 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) [] &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, Items: prop.Items}, &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64}) apply.SetSchema(la.schema) - return []PhysicalPlan{apply} + return []PhysicalPlan{apply}, true } -func (p *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (p *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { if prop.IsFlashOnlyProp() { - return nil + return nil, true } var byItems []property.Item byItems = append(byItems, p.PartitionBy...) byItems = append(byItems, p.OrderBy...) childProperty := &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, Items: byItems, Enforced: true} if !prop.IsPrefix(childProperty) { - return nil + return nil, true } window := PhysicalWindow{ WindowFuncDescs: p.WindowFuncDescs, @@ -1678,11 +1668,11 @@ func (p *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) [] Frame: p.Frame, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, childProperty) window.SetSchema(p.Schema()) - return []PhysicalPlan{window} + return []PhysicalPlan{window}, true } // exhaustPhysicalPlans is only for implementing interface. DataSource and Dual generate task in `findBestTask` directly. -func (p *baseLogicalPlan) exhaustPhysicalPlans(_ *property.PhysicalProperty) []PhysicalPlan { +func (p *baseLogicalPlan) exhaustPhysicalPlans(_ *property.PhysicalProperty) ([]PhysicalPlan, bool) { panic("baseLogicalPlan.exhaustPhysicalPlans() should never be called.") } @@ -1711,9 +1701,9 @@ func (la *LogicalAggregation) getEnforcedStreamAggs(prop *property.PhysicalPrope taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopDoubleReadTaskType} if la.HasDistinct() { - // TODO: remove this logic after the cost estimation of distinct pushdown is implemented. + // TODO: remove AllowDistinctAggPushDown after the cost estimation of distinct pushdown is implemented. // If AllowDistinctAggPushDown is set to true, we should not consider RootTask. - if !la.ctx.GetSessionVars().AllowDistinctAggPushDown { + if !la.canPushToCop() || !la.ctx.GetSessionVars().AllowDistinctAggPushDown { taskTypes = []property.TaskType{property.RootTaskType} } } else if !la.aggHints.preferAggToCop { @@ -1782,9 +1772,9 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P // property that the stream aggregation required, no need to consider. taskTypes := []property.TaskType{property.CopSingleReadTaskType} if la.HasDistinct() { - // TODO: remove this logic after the cost estimation of distinct pushdown is implemented. + // TODO: remove AllowDistinctAggPushDown after the cost estimation of distinct pushdown is implemented. // If AllowDistinctAggPushDown is set to true, we should not consider RootTask. - if !la.ctx.GetSessionVars().AllowDistinctAggPushDown { + if !la.canPushToCop() || !la.ctx.GetSessionVars().AllowDistinctAggPushDown { taskTypes = []property.TaskType{property.RootTaskType} } else { if !la.distinctArgsMeetsProperty() { @@ -1825,9 +1815,9 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy taskTypes = append(taskTypes, property.CopTiFlashLocalReadTaskType) } if la.HasDistinct() { - // TODO: remove this logic after the cost estimation of distinct pushdown is implemented. + // TODO: remove AllowDistinctAggPushDown after the cost estimation of distinct pushdown is implemented. // If AllowDistinctAggPushDown is set to true, we should not consider RootTask. - if !la.ctx.GetSessionVars().AllowDistinctAggPushDown { + if !la.canPushToCop() || !la.ctx.GetSessionVars().AllowDistinctAggPushDown { taskTypes = []property.TaskType{property.RootTaskType} } } else if !la.aggHints.preferAggToCop { @@ -1859,7 +1849,7 @@ func (la *LogicalAggregation) ResetHintIfConflicted() (preferHash bool, preferSt return } -func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { if la.aggHints.preferAggToCop { if !la.canPushToCop() { errMsg := "Optimizer Hint AGG_TO_COP is inapplicable" @@ -1873,37 +1863,36 @@ func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProper hashAggs := la.getHashAggs(prop) if hashAggs != nil && preferHash { - return hashAggs + return hashAggs, true } streamAggs := la.getStreamAggs(prop) if streamAggs != nil && preferStream { - return streamAggs + return streamAggs, true } - if streamAggs == nil && preferStream { + aggs := append(hashAggs, streamAggs...) + + if streamAggs == nil && preferStream && !prop.IsEmpty() { errMsg := "Optimizer Hint STREAM_AGG is inapplicable" warning := ErrInternal.GenWithStack(errMsg) la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } - aggs := make([]PhysicalPlan, 0, len(hashAggs)+len(streamAggs)) - aggs = append(aggs, hashAggs...) - aggs = append(aggs, streamAggs...) - return aggs + return aggs, !(preferStream || preferHash) } -func (p *LogicalSelection) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (p *LogicalSelection) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { childProp := prop.Clone() sel := PhysicalSelection{ Conditions: p.Conditions, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, childProp) - return []PhysicalPlan{sel} + return []PhysicalPlan{sel}, true } -func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { if !prop.IsEmpty() { - return nil + return nil, true } allTaskTypes := prop.GetAllPossibleChildTaskTypes() ret := make([]PhysicalPlan, 0, len(allTaskTypes)) @@ -1915,12 +1904,12 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) []P }.Init(p.ctx, p.stats, p.blockOffset, resultProp) ret = append(ret, limit) } - return ret + return ret, true } -func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { if prop.IsFlashOnlyProp() { - return nil + return nil, true } childProp := prop.Clone() lock := PhysicalLock{ @@ -1928,13 +1917,13 @@ func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) []Ph TblID2Handle: p.tblID2Handle, PartitionedTable: p.partitionedTable, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) - return []PhysicalPlan{lock} + return []PhysicalPlan{lock}, true } -func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { // TODO: UnionAll can not pass any order, but we can change it to sort merge to keep order. if !prop.IsEmpty() || prop.IsFlashOnlyProp() { - return nil + return nil, true } chReqProps := make([]*property.PhysicalProperty, 0, len(p.children)) for range p.children { @@ -1942,7 +1931,7 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) } ua := PhysicalUnionAll{}.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) ua.SetSchema(p.Schema()) - return []PhysicalPlan{ua} + return []PhysicalPlan{ua}, true } func (ls *LogicalSort) getPhysicalSort(prop *property.PhysicalProperty) *PhysicalSort { @@ -1961,7 +1950,7 @@ func (ls *LogicalSort) getNominalSort(reqProp *property.PhysicalProperty) *Nomin return ps } -func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { if MatchItems(prop, ls.ByItems) { ret := make([]PhysicalPlan, 0, 2) ret = append(ret, ls.getPhysicalSort(prop)) @@ -1969,15 +1958,15 @@ func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) []P if ns != nil { ret = append(ret, ns) } - return ret + return ret, true } - return nil + return nil, true } -func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { +func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { if !prop.IsEmpty() || prop.IsFlashOnlyProp() { - return nil + return nil, true } mor := PhysicalMaxOneRow{}.Init(p.ctx, p.stats, p.blockOffset, &property.PhysicalProperty{ExpectedCnt: 2}) - return []PhysicalPlan{mor} + return []PhysicalPlan{mor}, true } diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index b034cb97ae01b..74b7117c89b14 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -38,8 +38,8 @@ import ( "github.com/pingcap/tidb/util/stringutil" ) -// EvalSubquery evaluates incorrelated subqueries once. -var EvalSubquery func(ctx context.Context, p PhysicalPlan, is infoschema.InfoSchema, sctx sessionctx.Context) ([][]types.Datum, error) +// EvalSubqueryFirstRow evaluates incorrelated subqueries once, and get first row. +var EvalSubqueryFirstRow func(ctx context.Context, p PhysicalPlan, is infoschema.InfoSchema, sctx sessionctx.Context) (row []types.Datum, err error) // evalAstExpr evaluates ast expression directly. func evalAstExpr(sctx sessionctx.Context, expr ast.ExprNode) (types.Datum, error) { @@ -277,7 +277,7 @@ func (er *expressionRewriter) constructBinaryOpFunction(l expression.Expression, if err != nil { return nil, err } - expr5, err = er.newFunction(ast.If, types.NewFieldType(mysql.TypeTiny), expr3, expression.Null, expr4) + expr5, err = er.newFunction(ast.If, types.NewFieldType(mysql.TypeTiny), expr3, expression.NewNull(), expr4) if err != nil { return nil, err } @@ -545,10 +545,10 @@ func (er *expressionRewriter) buildQuantifierPlan(plan4Agg *LogicalAggregation, } plan4Agg.AggFuncs = append(plan4Agg.AggFuncs, funcSum) plan4Agg.schema.Append(colSum) - innerHasNull := expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), colSum, expression.Zero) + innerHasNull := expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), colSum, expression.NewZero()) // Build `count(1)` aggregation to check if subquery is empty. - funcCount, err := aggregation.NewAggFuncDesc(er.sctx, ast.AggFuncCount, []expression.Expression{expression.One}, false) + funcCount, err := aggregation.NewAggFuncDesc(er.sctx, ast.AggFuncCount, []expression.Expression{expression.NewOne()}, false) if err != nil { er.err = err return @@ -563,22 +563,22 @@ func (er *expressionRewriter) buildQuantifierPlan(plan4Agg *LogicalAggregation, if all { // All of the inner record set should not contain null value. So for t.id < all(select s.id from s), it // should be rewrote to t.id < min(s.id) and if(sum(s.id is null) != 0, null, true). - innerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), innerHasNull, expression.Null, expression.One) + innerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), innerHasNull, expression.NewNull(), expression.NewOne()) cond = expression.ComposeCNFCondition(er.sctx, cond, innerNullChecker) // If the subquery is empty, it should always return true. - emptyChecker := expression.NewFunctionInternal(er.sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), colCount, expression.Zero) + emptyChecker := expression.NewFunctionInternal(er.sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), colCount, expression.NewZero()) // If outer key is null, and subquery is not empty, it should always return null, even when it is `null = all (1, 2)`. - outerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), outerIsNull, expression.Null, expression.Zero) + outerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), outerIsNull, expression.NewNull(), expression.NewZero()) cond = expression.ComposeDNFCondition(er.sctx, cond, emptyChecker, outerNullChecker) } else { // For "any" expression, if the subquery has null and the cond returns false, the result should be NULL. // Specifically, `t.id < any (select s.id from s)` would be rewrote to `t.id < max(s.id) or if(sum(s.id is null) != 0, null, false)` - innerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), innerHasNull, expression.Null, expression.Zero) + innerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), innerHasNull, expression.NewNull(), expression.NewZero()) cond = expression.ComposeDNFCondition(er.sctx, cond, innerNullChecker) // If the subquery is empty, it should always return false. - emptyChecker := expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), colCount, expression.Zero) + emptyChecker := expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), colCount, expression.NewZero()) // If outer key is null, and subquery is not empty, it should return null. - outerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), outerIsNull, expression.Null, expression.One) + outerNullChecker := expression.NewFunctionInternal(er.sctx, ast.If, types.NewFieldType(mysql.TypeTiny), outerIsNull, expression.NewNull(), expression.NewOne()) cond = expression.ComposeCNFCondition(er.sctx, cond, emptyChecker, outerNullChecker) } @@ -640,7 +640,7 @@ func (er *expressionRewriter) handleNEAny(lexpr, rexpr expression.Expression, np } plan4Agg.names = append(plan4Agg.names, types.EmptyName, types.EmptyName) plan4Agg.SetSchema(expression.NewSchema(firstRowResultCol, count)) - gtFunc := expression.NewFunctionInternal(er.sctx, ast.GT, types.NewFieldType(mysql.TypeTiny), count, expression.One) + gtFunc := expression.NewFunctionInternal(er.sctx, ast.GT, types.NewFieldType(mysql.TypeTiny), count, expression.NewOne()) neCond := expression.NewFunctionInternal(er.sctx, ast.NE, types.NewFieldType(mysql.TypeTiny), lexpr, firstRowResultCol) cond := expression.ComposeDNFCondition(er.sctx, gtFunc, neCond) er.buildQuantifierPlan(plan4Agg, cond, lexpr, rexpr, false) @@ -677,7 +677,7 @@ func (er *expressionRewriter) handleEQAll(lexpr, rexpr expression.Expression, np RetType: countFunc.RetTp, } plan4Agg.SetSchema(expression.NewSchema(firstRowResultCol, count)) - leFunc := expression.NewFunctionInternal(er.sctx, ast.LE, types.NewFieldType(mysql.TypeTiny), count, expression.One) + leFunc := expression.NewFunctionInternal(er.sctx, ast.LE, types.NewFieldType(mysql.TypeTiny), count, expression.NewOne()) eqCond := expression.NewFunctionInternal(er.sctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), lexpr, firstRowResultCol) cond := expression.ComposeCNFCondition(er.sctx, leFunc, eqCond) er.buildQuantifierPlan(plan4Agg, cond, lexpr, rexpr, true) @@ -707,15 +707,15 @@ func (er *expressionRewriter) handleExistSubquery(ctx context.Context, v *ast.Ex er.err = err return v, true } - rows, err := EvalSubquery(ctx, physicalPlan, er.b.is, er.b.ctx) + row, err := EvalSubqueryFirstRow(ctx, physicalPlan, er.b.is, er.b.ctx) if err != nil { er.err = err return v, true } - if (len(rows) > 0 && !v.Not) || (len(rows) == 0 && v.Not) { - er.ctxStackAppend(expression.One.Clone(), types.EmptyName) + if (row != nil && !v.Not) || (row == nil && v.Not) { + er.ctxStackAppend(expression.NewOne(), types.EmptyName) } else { - er.ctxStackAppend(expression.Zero.Clone(), types.EmptyName) + er.ctxStackAppend(expression.NewZero(), types.EmptyName) } } return v, true @@ -877,14 +877,14 @@ func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, v *ast.S er.err = err return v, true } - rows, err := EvalSubquery(ctx, physicalPlan, er.b.is, er.b.ctx) + row, err := EvalSubqueryFirstRow(ctx, physicalPlan, er.b.is, er.b.ctx) if err != nil { er.err = err return v, true } if np.Schema().Len() > 1 { newCols := make([]expression.Expression, 0, np.Schema().Len()) - for i, data := range rows[0] { + for i, data := range row { newCols = append(newCols, &expression.Constant{ Value: data, RetType: np.Schema().Columns[i].GetType()}) @@ -897,7 +897,7 @@ func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, v *ast.S er.ctxStackAppend(expr, types.EmptyName) } else { er.ctxStackAppend(&expression.Constant{ - Value: rows[0][0], + Value: row[0], RetType: np.Schema().Columns[0].GetType(), }, types.EmptyName) } @@ -1240,7 +1240,7 @@ func (er *expressionRewriter) inToExpression(lLen int, not bool, tp *types.Field leftEt, leftIsNull := leftFt.EvalType(), leftFt.Tp == mysql.TypeNull if leftIsNull { er.ctxStackPop(lLen + 1) - er.ctxStackAppend(expression.Null.Clone(), types.EmptyName) + er.ctxStackAppend(expression.NewNull(), types.EmptyName) return } if leftEt == types.ETInt { @@ -1649,7 +1649,7 @@ func (er *expressionRewriter) evalDefaultExpr(v *ast.DefaultExpr) { switch { case isCurrentTimestamp && col.Tp == mysql.TypeDatetime: // for DATETIME column with current_timestamp, use NULL to be compatible with MySQL 5.7 - val = expression.Null + val = expression.NewNull() case isCurrentTimestamp && col.Tp == mysql.TypeTimestamp: // for TIMESTAMP column with current_timestamp, use 0 to be compatible with MySQL 5.7 zero := types.NewTime(types.ZeroCoreTime, mysql.TypeTimestamp, int8(col.Decimal)) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index c8af681c70b46..ba8cce5cc7ee1 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -128,46 +128,9 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty) (task return &rootTask{p: pShow}, nil } -// findBestTask implements LogicalPlan interface. -func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTask task, err error) { - // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, - // and set inner child prop nil, so here we do nothing. - if prop == nil { - return nil, nil - } - // Look up the task with this prop in the task map. - // It's used to reduce double counting. - bestTask = p.getTask(prop) - if bestTask != nil { - return bestTask, nil - } - - if prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiFlashLocalReadTaskType && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { - // Currently all plan cannot totally push down. - p.storeTask(prop, invalidTask) - return invalidTask, nil - } - - bestTask = invalidTask +func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPlan, prop *property.PhysicalProperty) (task, error) { + var bestTask task = invalidTask childTasks := make([]task, 0, len(p.children)) - - // If prop.enforced is true, cols of prop as parameter in exhaustPhysicalPlans should be nil - // And reset it for enforcing task prop and storing map - oldPropCols := prop.Items - if prop.Enforced { - // First, get the bestTask without enforced prop - prop.Enforced = false - bestTask, err = p.findBestTask(prop) - if err != nil { - return nil, err - } - prop.Enforced = true - // Next, get the bestTask with enforced prop - prop.Items = []property.Item{} - } - physicalPlans := p.self.exhaustPhysicalPlans(prop) - prop.Items = oldPropCols - for _, pp := range physicalPlans { // find best child tasks firstly. childTasks = childTasks[:0] @@ -207,10 +170,84 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTas } // get the most efficient one. - if curTask.cost() < bestTask.cost() { + if curTask.cost() < bestTask.cost() || (bestTask.invalid() && !curTask.invalid()) { bestTask = curTask } } + return bestTask, nil +} + +// findBestTask implements LogicalPlan interface. +func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty) (bestTask task, err error) { + // If p is an inner plan in an IndexJoin, the IndexJoin will generate an inner plan by itself, + // and set inner child prop nil, so here we do nothing. + if prop == nil { + return nil, nil + } + // Look up the task with this prop in the task map. + // It's used to reduce double counting. + bestTask = p.getTask(prop) + if bestTask != nil { + return bestTask, nil + } + + if prop.TaskTp != property.RootTaskType && prop.TaskTp != property.CopTiFlashLocalReadTaskType && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { + // Currently all plan cannot totally push down. + p.storeTask(prop, invalidTask) + return invalidTask, nil + } + + bestTask = invalidTask + // prop should be read only because its cached hashcode might be not consistent + // when it is changed. So we clone a new one for the temporary changes. + newProp := prop.Clone() + newProp.Enforced = prop.Enforced + var plansFitsProp, plansNeedEnforce []PhysicalPlan + var hintWorksWithProp bool + // Maybe the plan can satisfy the required property, + // so we try to get the task without the enforced sort first. + plansFitsProp, hintWorksWithProp = p.self.exhaustPhysicalPlans(newProp) + if !hintWorksWithProp && !newProp.IsEmpty() { + // If there is a hint in the plan and the hint cannot satisfy the property, + // we enforce this property and try to generate the PhysicalPlan again to + // make sure the hint can work. + newProp.Enforced = true + } + + if newProp.Enforced { + // Then, we use the empty property to get physicalPlans and + // try to get the task with an enforced sort. + newProp.Items = []property.Item{} + newProp.ExpectedCnt = math.MaxFloat64 + var hintCanWork bool + plansNeedEnforce, hintCanWork = p.self.exhaustPhysicalPlans(newProp) + if hintCanWork && !hintWorksWithProp { + // If the hint can work with the empty property, but cannot work with + // the required property, we give up `plansFitProp` to make sure the hint + // can work. + plansFitsProp = nil + } + if !hintCanWork && !hintWorksWithProp && !prop.Enforced { + // If the original property is not enforced and hint cannot + // work anyway, we give up `plansNeedEnforce` for efficiency, + plansNeedEnforce = nil + } + newProp.Items = prop.Items + newProp.ExpectedCnt = prop.ExpectedCnt + } + + newProp.Enforced = false + if bestTask, err = p.enumeratePhysicalPlans4Task(plansFitsProp, newProp); err != nil { + return nil, err + } + newProp.Enforced = true + curTask, err := p.enumeratePhysicalPlans4Task(plansNeedEnforce, newProp) + if err != nil { + return nil, err + } + if curTask.cost() < bestTask.cost() || (bestTask.invalid() && !curTask.invalid()) { + bestTask = curTask + } p.storeTask(prop, bestTask) return bestTask, nil @@ -562,27 +599,22 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c indexPlanFinished: true, tblColHists: ds.TblColHists, } - allCovered := true for _, partPath := range path.PartialIndexPaths { var scan PhysicalPlan var partialCost, rowCount float64 - var tempCovered bool if partPath.IsTablePath { - scan, partialCost, rowCount, tempCovered = ds.convertToPartialTableScan(prop, partPath) + scan, partialCost, rowCount = ds.convertToPartialTableScan(prop, partPath) } else { - scan, partialCost, rowCount, tempCovered = ds.convertToPartialIndexScan(prop, partPath) + scan, partialCost, rowCount = ds.convertToPartialIndexScan(prop, partPath) } scans = append(scans, scan) totalCost += partialCost totalRowCount += rowCount - allCovered = allCovered && tempCovered } - if !allCovered || len(path.TableFilters) > 0 { - ts, partialCost := ds.buildIndexMergeTableScan(prop, path.TableFilters, totalRowCount) - totalCost += partialCost - cop.tablePlan = ts - } + ts, partialCost := ds.buildIndexMergeTableScan(prop, path.TableFilters, totalRowCount) + totalCost += partialCost + cop.tablePlan = ts cop.idxMergePartPlans = scans cop.cst = totalCost task = finishCopTask(ds.ctx, cop) @@ -592,8 +624,7 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c func (ds *DataSource) convertToPartialIndexScan(prop *property.PhysicalProperty, path *util.AccessPath) ( indexPlan PhysicalPlan, partialCost float64, - rowCount float64, - isCovered bool) { + rowCount float64) { idx := path.Index is, partialCost, rowCount := ds.getOriginalPhysicalIndexScan(prop, path, false, false) rowSize := is.indexScanRowSize(idx, ds, false) @@ -615,18 +646,17 @@ func (ds *DataSource) convertToPartialIndexScan(prop *property.PhysicalProperty, indexPlan := PhysicalSelection{Conditions: indexConds}.Init(is.ctx, stats, ds.blockOffset) indexPlan.SetChildren(is) partialCost += rowCount * rowSize * sessVars.NetworkFactor - return indexPlan, partialCost, rowCount, false + return indexPlan, partialCost, rowCount } partialCost += rowCount * rowSize * sessVars.NetworkFactor indexPlan = is - return indexPlan, partialCost, rowCount, false + return indexPlan, partialCost, rowCount } func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, path *util.AccessPath) ( tablePlan PhysicalPlan, partialCost float64, - rowCount float64, - isCovered bool) { + rowCount float64) { ts, partialCost, rowCount := ds.getOriginalPhysicalTableScan(prop, path, false) rowSize := ds.TblColHists.GetAvgRowSize(ds.ctx, ds.TblCols, false, false) sessVars := ds.ctx.GetSessionVars() @@ -640,11 +670,11 @@ func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, tablePlan.SetChildren(ts) partialCost += rowCount * sessVars.CopCPUFactor partialCost += selectivity * rowCount * rowSize * sessVars.NetworkFactor - return tablePlan, partialCost, rowCount, true + return tablePlan, partialCost, rowCount } partialCost += rowCount * rowSize * sessVars.NetworkFactor tablePlan = ts - return tablePlan, partialCost, rowCount, true + return tablePlan, partialCost, rowCount } func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, tableFilters []expression.Expression, totalRowCount float64) (PhysicalPlan, float64) { @@ -840,6 +870,13 @@ func (is *PhysicalIndexScan) addPushedDownSelection(copTask *copTask, p *DataSou tableConds, copTask.rootTaskConds = SplitSelCondsWithVirtualColumn(tableConds) + var newRootConds []expression.Expression + indexConds, newRootConds = expression.PushDownExprs(is.ctx.GetSessionVars().StmtCtx, indexConds, is.ctx.GetClient(), kv.TiKV) + copTask.rootTaskConds = append(copTask.rootTaskConds, newRootConds...) + + tableConds, newRootConds = expression.PushDownExprs(is.ctx.GetSessionVars().StmtCtx, tableConds, is.ctx.GetClient(), kv.TiKV) + copTask.rootTaskConds = append(copTask.rootTaskConds, newRootConds...) + sessVars := is.ctx.GetSessionVars() if indexConds != nil { copTask.cst += copTask.count() * sessVars.CopCPUFactor @@ -1163,7 +1200,7 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida rTsk := &rootTask{p: pointGetPlan} var cost float64 if candidate.path.IsTablePath { - pointGetPlan.Handle = candidate.path.Ranges[0].LowVal[0].GetInt64() + pointGetPlan.Handle = kv.IntHandle(candidate.path.Ranges[0].LowVal[0].GetInt64()) pointGetPlan.UnsignedHandle = mysql.HasUnsignedFlag(ds.getHandleCol().RetType.Flag) pointGetPlan.PartitionInfo = partitionInfo cost = pointGetPlan.GetCost(ds.TblCols) @@ -1224,7 +1261,7 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca var cost float64 if candidate.path.IsTablePath { for _, ran := range candidate.path.Ranges { - batchPointGetPlan.Handles = append(batchPointGetPlan.Handles, ran.LowVal[0].GetInt64()) + batchPointGetPlan.Handles = append(batchPointGetPlan.Handles, kv.IntHandle(ran.LowVal[0].GetInt64())) } cost = batchPointGetPlan.GetCost(ds.TblCols) // Add filter condition to table plan now. diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go new file mode 100644 index 0000000000000..701376b035c17 --- /dev/null +++ b/planner/core/find_best_task_test.go @@ -0,0 +1,274 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "fmt" + "math" + + . "github.com/pingcap/check" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/sessionctx" +) + +var _ = Suite(&testFindBestTaskSuite{}) + +type testFindBestTaskSuite struct { + ctx sessionctx.Context +} + +func (s *testFindBestTaskSuite) SetUpSuite(c *C) { + s.ctx = MockContext() +} + +type mockDataSource struct { + baseLogicalPlan +} + +func (ds mockDataSource) Init(ctx sessionctx.Context) *mockDataSource { + ds.baseLogicalPlan = newBaseLogicalPlan(ctx, "mockDS", &ds, 0) + return &ds +} + +func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty) (task, error) { + // It can satisfy any of the property! + // Just use a TableDual for convenience. + p := PhysicalTableDual{}.Init(ds.ctx, &property.StatsInfo{RowCount: 1}, 0) + task := &rootTask{ + p: p, + cst: 10000, + } + return task, nil +} + +// mockLogicalPlan4Test is a LogicalPlan which is used for unit test. +// The basic assumption: +// 1. mockLogicalPlan4Test can generate tow kinds of physical plan: physicalPlan1 and +// physicalPlan2. physicalPlan1 can pass the property only when they are the same +// order; while physicalPlan2 cannot match any of the property(in other words, we can +// generate it only when then property is empty). +// 2. We have a hint for physicalPlan2. +// 3. If the property is empty, we still need to check `canGeneratePlan2` to decide +// whether it can generate physicalPlan2. +type mockLogicalPlan4Test struct { + baseLogicalPlan + // hasHintForPlan2 indicates whether this mockPlan contains hint. + // This hint is used to generate physicalPlan2. See the implementation + // of exhaustPhysicalPlans(). + hasHintForPlan2 bool + // canGeneratePlan2 indicates whether this plan can generate physicalPlan2. + canGeneratePlan2 bool + // costOverflow indicates whether this plan will generate physical plan whose cost is overflowed. + costOverflow bool +} + +func (p mockLogicalPlan4Test) Init(ctx sessionctx.Context) *mockLogicalPlan4Test { + p.baseLogicalPlan = newBaseLogicalPlan(ctx, "mockPlan", &p, 0) + return &p +} + +func (p *mockLogicalPlan4Test) getPhysicalPlan1(prop *property.PhysicalProperty) PhysicalPlan { + physicalPlan1 := mockPhysicalPlan4Test{planType: 1, costOverflow: p.costOverflow}.Init(p.ctx) + physicalPlan1.stats = &property.StatsInfo{RowCount: 1} + physicalPlan1.childrenReqProps = make([]*property.PhysicalProperty, 1) + physicalPlan1.childrenReqProps[0] = prop.Clone() + return physicalPlan1 +} + +func (p *mockLogicalPlan4Test) getPhysicalPlan2(prop *property.PhysicalProperty) PhysicalPlan { + physicalPlan2 := mockPhysicalPlan4Test{planType: 2, costOverflow: p.costOverflow}.Init(p.ctx) + physicalPlan2.stats = &property.StatsInfo{RowCount: 1} + physicalPlan2.childrenReqProps = make([]*property.PhysicalProperty, 1) + physicalPlan2.childrenReqProps[0] = property.NewPhysicalProperty(prop.TaskTp, nil, false, prop.ExpectedCnt, false) + return physicalPlan2 +} + +func (p *mockLogicalPlan4Test) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { + plan1 := make([]PhysicalPlan, 0, 1) + plan2 := make([]PhysicalPlan, 0, 1) + if prop.IsEmpty() && p.canGeneratePlan2 { + // Generate PhysicalPlan2 when the property is empty. + plan2 = append(plan2, p.getPhysicalPlan2(prop)) + if p.hasHintForPlan2 { + return plan2, true + } + } + if all, _ := prop.AllSameOrder(); all { + // Generate PhysicalPlan1 when properties are the same order. + plan1 = append(plan1, p.getPhysicalPlan1(prop)) + } + if p.hasHintForPlan2 { + // The hint cannot work. + if prop.IsEmpty() { + p.ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("the hint is inapplicable for plan2")) + } + return plan1, false + } + return append(plan1, plan2...), true +} + +type mockPhysicalPlan4Test struct { + basePhysicalPlan + // 1 or 2 for physicalPlan1 or physicalPlan2. + // See the comment of mockLogicalPlan4Test. + planType int + costOverflow bool +} + +func (p mockPhysicalPlan4Test) Init(ctx sessionctx.Context) *mockPhysicalPlan4Test { + p.basePhysicalPlan = newBasePhysicalPlan(ctx, "mockPlan", &p, 0) + return &p +} + +func (p *mockPhysicalPlan4Test) attach2Task(tasks ...task) task { + t := tasks[0].copy() + attachPlan2Task(p, t) + if p.costOverflow { + t.addCost(math.MaxFloat64) + } else { + t.addCost(1) + } + return t +} + +func (s *testFindBestTaskSuite) TestCostOverflow(c *C) { + ctx := MockContext() + // Plan Tree: mockPlan -> mockDataSource + mockPlan := mockLogicalPlan4Test{costOverflow: true}.Init(ctx) + mockDS := mockDataSource{}.Init(ctx) + mockPlan.SetChildren(mockDS) + // An empty property is enough for this test. + prop := property.NewPhysicalProperty(property.RootTaskType, nil, false, 0, false) + t, err := mockPlan.findBestTask(prop) + c.Assert(err, IsNil) + // The cost should be overflowed, but the task shouldn't be invalid. + c.Assert(t.invalid(), IsFalse) + c.Assert(t.cost(), Equals, math.MaxFloat64) +} + +func (s *testFindBestTaskSuite) TestEnforcedProperty(c *C) { + ctx := MockContext() + // PlanTree : mockLogicalPlan -> mockDataSource + mockPlan := mockLogicalPlan4Test{}.Init(ctx) + mockDS := mockDataSource{}.Init(ctx) + mockPlan.SetChildren(mockDS) + + col0 := &expression.Column{UniqueID: 1} + col1 := &expression.Column{UniqueID: 2} + // Use different order, so that mockLogicalPlan cannot generate any of the + // physical plans. + item0 := property.Item{Col: col0, Desc: false} + item1 := property.Item{Col: col1, Desc: true} + items := []property.Item{item0, item1} + + prop0 := &property.PhysicalProperty{ + Items: items, + Enforced: false, + } + // should return invalid task because no physical plan can match this property. + task, err := mockPlan.findBestTask(prop0) + c.Assert(err, IsNil) + c.Assert(task.invalid(), IsTrue) + + prop1 := &property.PhysicalProperty{ + Items: items, + Enforced: true, + } + // should return the valid task when the property is enforced. + task, err = mockPlan.findBestTask(prop1) + c.Assert(err, IsNil) + c.Assert(task.invalid(), IsFalse) +} + +func (s *testFindBestTaskSuite) TestHintCannotFitProperty(c *C) { + ctx := MockContext() + // PlanTree : mockLogicalPlan -> mockDataSource + mockPlan0 := mockLogicalPlan4Test{ + hasHintForPlan2: true, + canGeneratePlan2: true, + }.Init(ctx) + mockDS := mockDataSource{}.Init(ctx) + mockPlan0.SetChildren(mockDS) + + col0 := &expression.Column{UniqueID: 1} + item0 := property.Item{Col: col0} + items := []property.Item{item0} + // case 1, The property is not empty and enforced, should enforce a sort. + prop0 := &property.PhysicalProperty{ + Items: items, + Enforced: true, + } + task, err := mockPlan0.findBestTask(prop0) + c.Assert(err, IsNil) + c.Assert(task.invalid(), IsFalse) + _, enforcedSort := task.plan().(*PhysicalSort) + c.Assert(enforcedSort, IsTrue) + plan2 := task.plan().Children()[0] + mockPhysicalPlan, ok := plan2.(*mockPhysicalPlan4Test) + c.Assert(ok, IsTrue) + c.Assert(mockPhysicalPlan.planType, Equals, 2) + + // case 2, The property is not empty but not enforced, still need to enforce a sort + // to ensure the hint can work + prop1 := &property.PhysicalProperty{ + Items: items, + Enforced: false, + } + task, err = mockPlan0.findBestTask(prop1) + c.Assert(err, IsNil) + c.Assert(task.invalid(), IsFalse) + _, enforcedSort = task.plan().(*PhysicalSort) + c.Assert(enforcedSort, IsTrue) + plan2 = task.plan().Children()[0] + mockPhysicalPlan, ok = plan2.(*mockPhysicalPlan4Test) + c.Assert(ok, IsTrue) + c.Assert(mockPhysicalPlan.planType, Equals, 2) + + // case 3, The hint cannot work even if the property is empty, should return a warning + // and generate physicalPlan1. + prop2 := &property.PhysicalProperty{ + Items: items, + Enforced: false, + } + mockPlan1 := mockLogicalPlan4Test{ + hasHintForPlan2: true, + canGeneratePlan2: false, + }.Init(ctx) + mockPlan1.SetChildren(mockDS) + task, err = mockPlan1.findBestTask(prop2) + c.Assert(err, IsNil) + c.Assert(task.invalid(), IsFalse) + c.Assert(ctx.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) + // Because physicalPlan1 can match the property, so we should get it. + mockPhysicalPlan, ok = task.plan().(*mockPhysicalPlan4Test) + c.Assert(ok, IsTrue) + c.Assert(mockPhysicalPlan.planType, Equals, 1) + + // case 4, Similar to case 3, but the property is enforced now. Ths result should be + // the same with case 3. + ctx.GetSessionVars().StmtCtx.SetWarnings(nil) + prop3 := &property.PhysicalProperty{ + Items: items, + Enforced: true, + } + task, err = mockPlan1.findBestTask(prop3) + c.Assert(err, IsNil) + c.Assert(task.invalid(), IsFalse) + c.Assert(ctx.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) + // Because physicalPlan1 can match the property, so we don't need to enforce a sort. + mockPhysicalPlan, ok = task.plan().(*mockPhysicalPlan4Test) + c.Assert(ok, IsTrue) + c.Assert(mockPhysicalPlan.planType, Equals, 1) +} diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 781dfe82a44ae..dd96e235058dd 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -387,8 +387,8 @@ func (p PhysicalIndexMergeReader) Init(ctx sessionctx.Context, offset int) *Phys for _, partPlan := range p.partialPlans { totalRowCount += partPlan.StatsCount() } - p.stats.StatsVersion = p.partialPlans[0].statsInfo().StatsVersion p.stats = p.partialPlans[0].statsInfo().ScaleByExpectCnt(totalRowCount) + p.stats.StatsVersion = p.partialPlans[0].statsInfo().StatsVersion } p.PartialPlans = make([][]PhysicalPlan, 0, len(p.partialPlans)) for _, partialPlan := range p.partialPlans { diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 99db04f075223..240c5484cf790 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -14,10 +14,13 @@ package core_test import ( + "fmt" + . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -695,7 +698,7 @@ func (s *testIntegrationSuite) TestIndexMerge(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, unique index(a), unique index(b))") + tk.MustExec("create table t(a int, b int, c int, unique index(a), unique index(b), primary key(c))") var input []string var output []struct { @@ -812,6 +815,37 @@ func (s *testIntegrationSuite) TestIssue15546(c *C) { tk.MustQuery("select * from pt, vt where pt.a = vt.a").Check(testkit.Rows("1 1 1 1")) } +func (s *testIntegrationSuite) TestHintWithRequiredProperty(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 primary key, b int, c int, key b(b))") + var input []string + var output []struct { + SQL string + Plan []string + Warnings []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() + output[i].Warnings = make([]string, len(warnings)) + for j, warning := range warnings { + output[i].Warnings[j] = warning.Err.Error() + } + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() + c.Assert(len(warnings), Equals, len(output[i].Warnings)) + for j, warning := range warnings { + c.Assert(output[i].Warnings[j], Equals, warning.Err.Error()) + } + } +} + func (s *testIntegrationSuite) TestIssue15813(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -824,6 +858,17 @@ func (s *testIntegrationSuite) TestIssue15813(c *C) { tk.MustQuery("select /*+ MERGE_JOIN(t0, t1) */ * from t0, t1 where t0.c0 = t1.c0").Check(testkit.Rows()) } +func (s *testIntegrationSuite) TestFullGroupByOrderBy(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)") + tk.MustQuery("select count(a) as b from t group by a order by b").Check(testkit.Rows()) + err := tk.ExecToErr("select count(a) as cnt from t group by a order by b") + c.Assert(terror.ErrorEqual(err, core.ErrFieldNotInGroupBy), IsTrue) +} + func (s *testIntegrationSuite) TestHintWithoutTableWarning(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -889,3 +934,55 @@ func (s *testIntegrationSuite) TestIssue15846(c *C) { tk.MustExec("INSERT INTO t0(t0) VALUES (NULL), (NULL);") tk.MustQuery("SELECT t1.c0 FROM t1 LEFT JOIN t0 ON 1;").Check(testkit.Rows("0", "0")) } + +func (s *testIntegrationSuite) TestFloorUnixTimestampPruning(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists floor_unix_timestamp") + tk.MustExec(`create table floor_unix_timestamp (ts timestamp(3)) +partition by range (floor(unix_timestamp(ts))) ( +partition p0 values less than (unix_timestamp('2020-04-05 00:00:00')), +partition p1 values less than (unix_timestamp('2020-04-12 00:00:00')), +partition p2 values less than (unix_timestamp('2020-04-15 00:00:00')))`) + tk.MustExec("insert into floor_unix_timestamp values ('2020-04-04 00:00:00')") + tk.MustExec("insert into floor_unix_timestamp values ('2020-04-04 23:59:59.999')") + tk.MustExec("insert into floor_unix_timestamp values ('2020-04-05 00:00:00')") + tk.MustExec("insert into floor_unix_timestamp values ('2020-04-05 00:00:00.001')") + tk.MustExec("insert into floor_unix_timestamp values ('2020-04-12 01:02:03.456')") + tk.MustExec("insert into floor_unix_timestamp values ('2020-04-14 00:00:42')") + tk.MustQuery("select count(*) from floor_unix_timestamp where '2020-04-05 00:00:00.001' = ts").Check(testkit.Rows("1")) + tk.MustQuery("select * from floor_unix_timestamp where ts > '2020-04-05 00:00:00' order by ts").Check(testkit.Rows("2020-04-05 00:00:00.001", "2020-04-12 01:02:03.456", "2020-04-14 00:00:42.000")) + tk.MustQuery("select count(*) from floor_unix_timestamp where ts <= '2020-04-05 23:00:00'").Check(testkit.Rows("4")) + tk.MustQuery("select * from floor_unix_timestamp partition(p1, p2) where ts > '2020-04-14 00:00:00'").Check(testkit.Rows("2020-04-14 00:00:42.000")) +} + +func (s *testIntegrationSuite) TestIssue16290And16292(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, primary key(a));") + tk.MustExec("insert into t values(1, 1);") + + for i := 0; i <= 1; i++ { + tk.MustExec(fmt.Sprintf("set session tidb_opt_agg_push_down = %v", i)) + + tk.MustQuery("select avg(a) from (select * from t ta union all select * from t tb) t;").Check(testkit.Rows("1.0000")) + tk.MustQuery("select avg(b) from (select * from t ta union all select * from t tb) t;").Check(testkit.Rows("1.0000")) + tk.MustQuery("select count(distinct a) from (select * from t ta union all select * from t tb) t;").Check(testkit.Rows("1")) + tk.MustQuery("select count(distinct b) from (select * from t ta union all select * from t tb) t;").Check(testkit.Rows("1")) + } +} + +func (s *testIntegrationSerialSuite) TestIssue16837(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 int,d int,e int,unique key idx_ab(a,b),unique key(c),unique key(d))") + tk.MustQuery("explain select /*+ use_index_merge(t,c,idx_ab) */ * from t where a = 1 or (e = 1 and c = 1)").Check(testkit.Rows( + "TableReader_7 8000.00 root data:Selection_6", + "└─Selection_6 8000.00 cop[tikv] or(eq(test.t.a, 1), and(eq(test.t.e, 1), eq(test.t.c, 1)))", + " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 IndexMerge is inapplicable or disabled")) + tk.MustExec("insert into t values (2, 1, 1, 1, 2)") + tk.MustQuery("select /*+ use_index_merge(t,c,idx_ab) */ * from t where a = 1 or (e = 1 and c = 1)").Check(testkit.Rows()) +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index e48f08b745605..1ec9fa765c837 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -922,7 +922,7 @@ func (b *PlanBuilder) buildProjection(ctx context.Context, p LogicalPlan, fields newNames = append(newNames, p.OutputNames()[i]) continue } else if !considerWindow && isWindowFuncField { - expr := expression.Zero + expr := expression.NewZero() proj.Exprs = append(proj.Exprs, expr) col, name, err := b.buildProjectionField(ctx, p, field, expr) if err != nil { @@ -2009,6 +2009,16 @@ func (b *PlanBuilder) checkOnlyFullGroupByWithGroupClause(p LogicalPlan, sel *as if sel.OrderBy != nil { for offset, item := range sel.OrderBy.Items { + if colName, ok := item.Expr.(*ast.ColumnNameExpr); ok { + index, err := resolveFromSelectFields(colName, sel.Fields.Fields, false) + if err != nil { + return err + } + // If the ByItem is in fields list, it has been checked already in above. + if index >= 0 { + continue + } + } checkExprInGroupBy(p, item.Expr, offset, ErrExprInOrderBy, gbyColNames, gbyExprs, notInGbyColNames) } } @@ -2140,6 +2150,8 @@ func (b *PlanBuilder) resolveGbyExprs(ctx context.Context, p LogicalPlan, gby *a } for _, item := range gby.Items { resolver.inExpr = false + resolver.exprDepth = 0 + resolver.isParam = false retExpr, _ := item.Expr.Accept(resolver) if resolver.err != nil { return nil, nil, errors.Trace(resolver.err) @@ -2706,7 +2718,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as var indexMergeHints []*ast.IndexHint if hints := b.TableHints(); hints != nil { for i, hint := range hints.indexMergeHintList { - if hint.tblName.L == tblName.L { + if hint.tblName.L == tblName.L && hint.dbName.L == dbName.L { hints.indexMergeHintList[i].matched = true // check whether the index names in IndexMergeHint are valid. invalidIdxNames := make([]string, 0, len(hint.indexHint.IndexNames)) @@ -2836,7 +2848,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as if err != nil { return nil, err } - colExpr.VirtualExpr = expr + colExpr.VirtualExpr = expr.Clone() } } } @@ -3250,6 +3262,9 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( if t.TableInfo.IsView() { return nil, errors.Errorf("update view %s is not supported now.", t.Name.O) } + if t.TableInfo.IsSequence() { + return nil, errors.Errorf("update sequence %s is not supported now.", t.Name.O) + } b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, dbName, t.Name.L, "", nil) } @@ -3261,7 +3276,10 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( } } if b.ctx.GetSessionVars().TxnCtx.IsPessimistic { - if !update.MultipleTable { + if update.TableRefs.TableRefs.Right == nil { + // buildSelectLock is an optimization that can reduce RPC call. + // We only need do this optimization for single table update which is the most common case. + // When TableRefs.Right is nil, it is single table update. p = b.buildSelectLock(p, ast.SelectLockForUpdate) } } @@ -3609,6 +3627,9 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( if tn.TableInfo.IsView() { return nil, errors.Errorf("delete view %s is not supported now.", tn.Name.O) } + if tn.TableInfo.IsSequence() { + return nil, errors.Errorf("delete sequence %s is not supported now.", tn.Name.O) + } b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DeletePriv, tn.Schema.L, tn.TableInfo.Name.L, "", nil) } } else { @@ -3617,6 +3638,9 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( if v.TableInfo.IsView() { return nil, errors.Errorf("delete view %s is not supported now.", v.Name.O) } + if v.TableInfo.IsSequence() { + return nil, errors.Errorf("delete sequence %s is not supported now.", v.Name.O) + } dbName := v.Schema.L if dbName == "" { dbName = b.ctx.GetSessionVars().CurrentDB diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 2e127fc1f001c..cf12f699baa91 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1579,3 +1579,23 @@ func (s *testPlanSuite) TestConflictedJoinTypeHints(c *C) { c.Assert(join.hintInfo, IsNil) c.Assert(join.preferJoinType, Equals, uint(0)) } + +func (s *testPlanSuite) TestSimplyOuterJoinWithOnlyOuterExpr(c *C) { + defer testleak.AfterTest(c)() + sql := "select * from t t1 right join t t0 ON TRUE where CONCAT_WS(t0.e=t0.e, 0, NULL) IS NULL" + ctx := context.TODO() + stmt, err := s.ParseOneStmt(sql, "", "") + c.Assert(err, IsNil) + Preprocess(s.ctx, stmt, s.is) + builder := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) + p, err := builder.Build(ctx, stmt) + c.Assert(err, IsNil) + p, err = logicalOptimize(ctx, builder.optFlag, p.(LogicalPlan)) + c.Assert(err, IsNil) + proj, ok := p.(*LogicalProjection) + c.Assert(ok, IsTrue) + join, ok := proj.Children()[0].(*LogicalJoin) + c.Assert(ok, IsTrue) + // previous wrong JoinType is InnerJoin + c.Assert(join.JoinType, Equals, RightOuterJoin) +} diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index 76ed43078c942..a62444974071e 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -371,7 +371,7 @@ func (helper extractHelper) extractTimeRange( timezone *time.Location, ) ( remained []expression.Expression, - // unix timestamp in millisecond + // unix timestamp in nanoseconds startTime int64, endTime int64, ) { @@ -398,7 +398,7 @@ func (helper extractHelper) extractTimeRange( if colName == extractColName { timeType := types.NewFieldType(mysql.TypeDatetime) - timeType.Decimal = 3 + timeType.Decimal = 6 timeDatum, err := datums[0].ConvertTo(ctx.GetSessionVars().StmtCtx, timeType) if err != nil || timeDatum.Kind() == types.KindNull { remained = append(remained, expr) @@ -414,7 +414,7 @@ func (helper extractHelper) extractTimeRange( mysqlTime.Second(), mysqlTime.Microsecond()*1000, timezone, - ).UnixNano() / int64(time.Millisecond) + ).UnixNano() switch fnName { case ast.EQ: @@ -425,14 +425,15 @@ func (helper extractHelper) extractTimeRange( endTime = mathutil.MinInt64(endTime, timestamp) } case ast.GT: - startTime = mathutil.MaxInt64(startTime, timestamp+1) + // FixMe: add 1ms is not absolutely correct here, just because the log search precision is millisecond. + startTime = mathutil.MaxInt64(startTime, timestamp+int64(time.Millisecond)) case ast.GE: startTime = mathutil.MaxInt64(startTime, timestamp) case ast.LT: if endTime == 0 { - endTime = timestamp - 1 + endTime = timestamp - int64(time.Millisecond) } else { - endTime = mathutil.MinInt64(endTime, timestamp-1) + endTime = mathutil.MinInt64(endTime, timestamp-int64(time.Millisecond)) } case ast.LE: if endTime == 0 { @@ -495,7 +496,7 @@ func (helper extractHelper) convertToTime(t int64) time.Time { if t == 0 || t == math.MaxInt64 { return time.Now() } - return time.Unix(t/1000, (t%1000)*int64(time.Millisecond)) + return time.Unix(0, t) } // ClusterTableExtractor is used to extract some predicates of cluster table. @@ -589,12 +590,14 @@ func (e *ClusterLogTableExtractor) Extract( } remained, startTime, endTime := e.extractTimeRange(ctx, schema, names, remained, "time", time.Local) - if endTime == 0 { - endTime = math.MaxInt64 - } + // The time unit for search log is millisecond. + startTime = startTime / int64(time.Millisecond) + endTime = endTime / int64(time.Millisecond) e.StartTime = startTime e.EndTime = endTime - e.SkipRequest = startTime > endTime + if startTime != 0 && endTime != 0 { + e.SkipRequest = startTime > endTime + } if e.SkipRequest { return nil @@ -610,12 +613,12 @@ func (e *ClusterLogTableExtractor) explainInfo(p *PhysicalMemTable) string { return "skip_request: true" } r := new(bytes.Buffer) - st, et := e.GetTimeRange(false) + st, et := e.StartTime, e.EndTime if st > 0 { st := time.Unix(0, st*1e6) r.WriteString(fmt.Sprintf("start_time:%v, ", st.In(p.ctx.GetSessionVars().StmtCtx.TimeZone).Format(MetricTableTimeFormat))) } - if et < math.MaxInt64 { + if et > 0 { et := time.Unix(0, et*1e6) r.WriteString(fmt.Sprintf("end_time:%v, ", et.In(p.ctx.GetSessionVars().StmtCtx.TimeZone).Format(MetricTableTimeFormat))) } @@ -637,27 +640,6 @@ func (e *ClusterLogTableExtractor) explainInfo(p *PhysicalMemTable) string { return s } -// GetTimeRange extract startTime and endTime -func (e *ClusterLogTableExtractor) GetTimeRange(isFailpointTestModeSkipCheck bool) (int64, int64) { - startTime := e.StartTime - endTime := e.EndTime - if endTime == 0 { - endTime = math.MaxInt64 - } - if !isFailpointTestModeSkipCheck { - // Just search the recent half an hour logs if the user doesn't specify the start time - const defaultSearchLogDuration = 30 * time.Minute / time.Millisecond - if startTime == 0 { - if endTime == math.MaxInt64 { - startTime = time.Now().UnixNano()/int64(time.Millisecond) - int64(defaultSearchLogDuration) - } else { - startTime = endTime - int64(defaultSearchLogDuration) - } - } - } - return startTime, endTime -} - // MetricTableExtractor is used to extract some predicates of metrics_schema tables. type MetricTableExtractor struct { extractHelper diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index 8a4f22cbdf075..8f176d99d4218 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -15,7 +15,6 @@ package core_test import ( "context" - "math" "regexp" "sort" "time" @@ -451,14 +450,12 @@ func (s *extractorSuite) TestClusterLogTableExtractor(c *C) { nodeTypes: set.NewStringSet(), instances: set.NewStringSet(), startTime: timestamp(c, "2019-10-10 10:10:10"), - endTime: math.MaxInt64, }, { sql: "select * from information_schema.cluster_log where time>='2019-10-10 10:10:10' and time>='2019-10-11 10:10:10' and time>='2019-10-12 10:10:10'", nodeTypes: set.NewStringSet(), instances: set.NewStringSet(), startTime: timestamp(c, "2019-10-12 10:10:10"), - endTime: math.MaxInt64, }, { sql: "select * from information_schema.cluster_log where time>='2019-10-10 10:10:10' and time>='2019-10-11 10:10:10' and time>='2019-10-12 10:10:10' and time='2019-10-13 10:10:10'", @@ -487,7 +484,6 @@ func (s *extractorSuite) TestClusterLogTableExtractor(c *C) { nodeTypes: set.NewStringSet(), instances: set.NewStringSet(), startTime: timestamp(c, "2019-10-10 10:10:10"), - endTime: math.MaxInt64, patterns: []string{".*a.*"}, }, { diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 7994e7999fa2e..da394fcf62cba 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -121,6 +121,10 @@ func CheckTableLock(ctx sessionctx.Context, is infoschema.InfoSchema, vs []visit // DoOptimize optimizes a logical plan to a physical plan. func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (PhysicalPlan, float64, error) { + // if there is something after flagPrunColumns, do flagPrunColumnsAgain + if flag&flagPrunColumns > 0 && flag-flagPrunColumns > flagPrunColumns { + flag |= flagPrunColumnsAgain + } logic, err := logicalOptimize(ctx, flag, logic) if err != nil { return nil, 0, err diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 73681c6a70bdd..13ab04b563796 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -858,6 +858,7 @@ func (s *testPlanSuite) TestPushdownDistinctEnable(c *C) { s.testData.GetTestCases(c, &input, &output) vars := []string{ fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), + "set session tidb_opt_agg_push_down = 1", } s.doTestPushdownDistinct(c, vars, input, output) } @@ -875,6 +876,24 @@ func (s *testPlanSuite) TestPushdownDistinctDisable(c *C) { s.testData.GetTestCases(c, &input, &output) vars := []string{ fmt.Sprintf("set @@session.%s = 0", variable.TiDBOptDistinctAggPushDown), + "set session tidb_opt_agg_push_down = 1", + } + s.doTestPushdownDistinct(c, vars, input, output) +} + +func (s *testPlanSuite) TestPushdownDistinctEnableAggPushDownDisable(c *C) { + var ( + input []string + output []struct { + SQL string + Plan []string + Result []string + } + ) + s.testData.GetTestCases(c, &input, &output) + vars := []string{ + fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), + "set session tidb_opt_agg_push_down = 0", } s.doTestPushdownDistinct(c, vars, input, output) } @@ -892,9 +911,24 @@ func (s *testPlanSuite) doTestPushdownDistinct(c *C, vars, input []string, outpu }() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") + tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, c int, index(c))") tk.MustExec("insert into t values (1, 1, 1), (1, 1, 3), (1, 2, 3), (2, 1, 3), (1, 2, NULL);") + + tk.MustExec("drop table if exists pt") + tk.MustExec(`CREATE TABLE pt (a int, b int) PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (2), + PARTITION p1 VALUES LESS THAN (100) + );`) + + tk.MustExec("drop table if exists ta") + tk.MustExec("create table ta(a int);") + tk.MustExec("insert into ta values(1), (1);") + tk.MustExec("drop table if exists tb") + tk.MustExec("create table tb(a int);") + tk.MustExec("insert into tb values(1), (1);") + tk.MustExec("set session sql_mode=''") tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggPartialConcurrency)) tk.MustExec(fmt.Sprintf("set session %s=1", variable.TiDBHashAggFinalConcurrency)) diff --git a/planner/core/plan.go b/planner/core/plan.go index 4df8a3044370f..5ca07968fa28d 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -15,12 +15,12 @@ package core import ( "fmt" - "github.com/pingcap/tidb/kv" "math" "strconv" "github.com/cznic/mathutil" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" @@ -178,7 +178,10 @@ type LogicalPlan interface { PreparePossibleProperties(schema *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column // exhaustPhysicalPlans generates all possible plans that can match the required property. - exhaustPhysicalPlans(*property.PhysicalProperty) []PhysicalPlan + // It will return: + // 1. All possible plans that can match the required property. + // 2. Whether the SQL hint can work. Return true if there is no hint. + exhaustPhysicalPlans(*property.PhysicalProperty) (physicalPlans []PhysicalPlan, hintCanWork bool) // ExtractCorrelatedCols extracts correlated columns inside the LogicalPlan. ExtractCorrelatedCols() []*expression.CorrelatedColumn diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 890cf6f50895b..7eaa161652ba2 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -39,8 +39,12 @@ func (p *basePhysicalPlan) ToPB(_ sessionctx.Context, _ kv.StoreType) (*tipb.Exe func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { sc := ctx.GetSessionVars().StmtCtx client := ctx.GetClient() + groupByExprs, err := expression.ExpressionsToPBList(sc, p.GroupByItems, client) + if err != nil { + return nil, err + } aggExec := &tipb.Aggregation{ - GroupBy: expression.ExpressionsToPBList(sc, p.GroupByItems, client), + GroupBy: groupByExprs, } for _, aggFunc := range p.AggFuncs { aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc)) @@ -61,8 +65,12 @@ func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) ( func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { sc := ctx.GetSessionVars().StmtCtx client := ctx.GetClient() + groupByExprs, err := expression.ExpressionsToPBList(sc, p.GroupByItems, client) + if err != nil { + return nil, err + } aggExec := &tipb.Aggregation{ - GroupBy: expression.ExpressionsToPBList(sc, p.GroupByItems, client), + GroupBy: groupByExprs, } for _, aggFunc := range p.AggFuncs { aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc)) @@ -83,8 +91,12 @@ func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) func (p *PhysicalSelection) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { sc := ctx.GetSessionVars().StmtCtx client := ctx.GetClient() + conditions, err := expression.ExpressionsToPBList(sc, p.Conditions, client) + if err != nil { + return nil, err + } selExec := &tipb.Selection{ - Conditions: expression.ExpressionsToPBList(sc, p.Conditions, client), + Conditions: conditions, } executorId := "" if storeType == kv.TiFlash { @@ -233,12 +245,20 @@ func (p *PhysicalBroadCastJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreT return nil, errors.Trace(err) } + left, leftErr := expression.ExpressionsToPBList(sc, leftJoinKeys, client) + if leftErr != nil { + return nil, leftErr + } + right, rightErr := expression.ExpressionsToPBList(sc, rightJoinKeys, client) + if rightErr != nil { + return nil, rightErr + } join := &tipb.Join{ JoinType: tipb.JoinType_TypeInnerJoin, JoinExecType: tipb.JoinExecType_TypeHashJoin, InnerIdx: int64(p.InnerChildIdx), - LeftJoinKeys: expression.ExpressionsToPBList(sc, leftJoinKeys, client), - RightJoinKeys: expression.ExpressionsToPBList(sc, rightJoinKeys, client), + LeftJoinKeys: left, + RightJoinKeys: right, Children: []*tipb.Executor{lChildren, rChildren}, } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 2708daed44e98..cdbf6cfcb0616 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -466,12 +466,6 @@ func NewPlanBuilder(sctx sessionctx.Context, is infoschema.InfoSchema, processor // Build builds the ast node to a Plan. func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (Plan, error) { b.optFlag |= flagPrunColumns - defer func() { - // if there is something after flagPrunColumns, do flagPrunColumnsAgain - if b.optFlag&flagPrunColumns > 0 && b.optFlag-flagPrunColumns > flagPrunColumns { - b.optFlag |= flagPrunColumnsAgain - } - }() switch x := node.(type) { case *ast.AdminStmt: return b.buildAdmin(ctx, x) @@ -512,9 +506,11 @@ func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (Plan, error) { return b.buildDo(ctx, x) case *ast.SetStmt: return b.buildSet(ctx, x) + case *ast.SetConfigStmt: + return b.buildSetConfig(ctx, x) case *ast.AnalyzeTableStmt: return b.buildAnalyze(x) - case *ast.BinlogStmt, *ast.FlushStmt, *ast.UseStmt, + case *ast.BinlogStmt, *ast.FlushStmt, *ast.UseStmt, *ast.BRIEStmt, *ast.BeginStmt, *ast.CommitStmt, *ast.RollbackStmt, *ast.CreateUserStmt, *ast.SetPwdStmt, *ast.AlterInstanceStmt, *ast.GrantStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.RevokeStmt, *ast.KillStmt, *ast.DropStatsStmt, *ast.GrantRoleStmt, *ast.RevokeRoleStmt, *ast.SetRoleStmt, *ast.SetDefaultRoleStmt, *ast.ShutdownStmt: @@ -533,6 +529,14 @@ func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (Plan, error) { return nil, ErrUnsupportedType.GenWithStack("Unsupported type %T", node) } +func (b *PlanBuilder) buildSetConfig(ctx context.Context, v *ast.SetConfigStmt) (Plan, error) { + privErr := ErrSpecificAccessDenied.GenWithStackByArgs("CONFIG") + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ConfigPriv, "", "", "", privErr) + mockTablePlan := LogicalTableDual{}.Init(b.ctx, b.getSelectOffset()) + expr, _, err := b.rewrite(ctx, v.Value, mockTablePlan, nil, true) + return &SetConfig{Name: v.Name, Type: v.Type, Instance: v.Instance, Value: expr}, err +} + func (b *PlanBuilder) buildChange(v *ast.ChangeStmt) (Plan, error) { exe := &Change{ ChangeStmt: v, @@ -1327,7 +1331,10 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A p := &Analyze{Opts: opts} for _, tbl := range as.TableNames { if tbl.TableInfo.IsView() { - return nil, errors.Errorf("analyze %s is not supported now.", tbl.Name.O) + return nil, errors.Errorf("analyze view %s is not supported now.", tbl.Name.O) + } + if tbl.TableInfo.IsSequence() { + return nil, errors.Errorf("analyze sequence %s is not supported now.", tbl.Name.O) } idxInfo, colInfo, pkInfo := getColsInfo(tbl) physicalIDs, names, err := getPhysicalIDsAndPartitionNames(tbl.TableInfo, as.PartitionNames) @@ -1487,6 +1494,7 @@ func buildShowNextRowID() (*expression.Schema, types.NameSlice) { schema.Append(buildColumnWithName("", "TABLE_NAME", mysql.TypeVarchar, mysql.MaxTableNameLength)) schema.Append(buildColumnWithName("", "COLUMN_NAME", mysql.TypeVarchar, mysql.MaxColumnNameLength)) schema.Append(buildColumnWithName("", "NEXT_GLOBAL_ROW_ID", mysql.TypeLonglong, 4)) + schema.Append(buildColumnWithName("", "ID_TYPE", mysql.TypeVarchar, 15)) return schema.col2Schema(), schema.names } @@ -1591,6 +1599,19 @@ func buildCancelDDLJobsFields() (*expression.Schema, types.NameSlice) { return schema.col2Schema(), schema.names } +func buildBRIESchema() (*expression.Schema, types.NameSlice) { + longlongSize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeLonglong) + datetimeSize, _ := mysql.GetDefaultFieldLengthAndDecimal(mysql.TypeDatetime) + + schema := newColumnsWithNames(5) + schema.Append(buildColumnWithName("", "Destination", mysql.TypeVarchar, 255)) + schema.Append(buildColumnWithName("", "Size", mysql.TypeLonglong, longlongSize)) + schema.Append(buildColumnWithName("", "BackupTS", mysql.TypeLonglong, longlongSize)) + schema.Append(buildColumnWithName("", "Queue Time", mysql.TypeDatetime, datetimeSize)) + schema.Append(buildColumnWithName("", "Execution Time", mysql.TypeDatetime, datetimeSize)) + return schema.col2Schema(), schema.names +} + func buildColumnWithName(tableName, name string, tp byte, size int) (*expression.Column, *types.FieldName) { cs, cl := types.DefaultCharsetForType(tp) flag := mysql.UnsignedFlag @@ -1691,6 +1712,9 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, case ast.ShowCreateView: err := ErrSpecificAccessDenied.GenWithStackByArgs("SHOW VIEW") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShowViewPriv, show.Table.Schema.L, show.Table.Name.L, "", err) + case ast.ShowBackups, ast.ShowRestores: + err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER") + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err) case ast.ShowTableNextRowId: p := &ShowNextRowID{TableName: show.Table} p.setSchemaAndNames(buildShowNextRowID()) @@ -1769,14 +1793,15 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { } } b.visitInfo = collectVisitInfoFromGrantStmt(b.ctx, b.visitInfo, raw) - case *ast.GrantRoleStmt: + case *ast.BRIEStmt: + p.setSchemaAndNames(buildBRIESchema()) err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err) - case *ast.RevokeStmt: - b.visitInfo = collectVisitInfoFromRevokeStmt(b.ctx, b.visitInfo, raw) - case *ast.RevokeRoleStmt: + case *ast.GrantRoleStmt, *ast.RevokeRoleStmt: err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err) + case *ast.RevokeStmt: + b.visitInfo = collectVisitInfoFromRevokeStmt(b.ctx, b.visitInfo, raw) case *ast.KillStmt: // If you have the SUPER privilege, you can kill all threads and statements. // Otherwise, you can kill only your own threads and statements. @@ -1945,6 +1970,13 @@ func (b *PlanBuilder) buildInsert(ctx context.Context, insert *ast.InsertStmt) ( } return nil, err } + if tableInfo.IsSequence() { + err := errors.Errorf("insert into sequence %s is not supported now.", tableInfo.Name.O) + if insert.IsReplace { + err = errors.Errorf("replace into sequence %s is not supported now.", tableInfo.Name.O) + } + return nil, err + } // Build Schema with DBName otherwise ColumnRef with DBName cannot match any Column in Schema. schema, names := expression.TableInfo2SchemaAndNames(b.ctx, tn.Schema, tableInfo) tableInPlan, ok := b.is.TableByID(tableInfo.ID) @@ -2141,7 +2173,12 @@ func (b *PlanBuilder) buildSetValuesOfInsert(ctx context.Context, insert *ast.In return ErrBadGeneratedColumn.GenWithStackByArgs(assign.Column.Name.O, tableInfo.Name.O) } b.curClause = fieldList - expr, _, err := b.rewriteWithPreprocess(ctx, assign.Expr, mockTablePlan, nil, nil, true, checkRefColumn) + // subquery in insert values should not reference upper scope + usingPlan := mockTablePlan + if _, ok := assign.Expr.(*ast.SubqueryExpr); ok { + usingPlan = LogicalTableDual{}.Init(b.ctx, b.getSelectOffset()) + } + expr, _, err := b.rewriteWithPreprocess(ctx, assign.Expr, usingPlan, nil, nil, true, checkRefColumn) if err != nil { return err } @@ -2214,7 +2251,12 @@ func (b *PlanBuilder) buildValuesListOfInsert(ctx context.Context, insert *ast.I } default: b.curClause = fieldList - expr, _, err = b.rewriteWithPreprocess(ctx, valueItem, mockTablePlan, nil, nil, true, checkRefColumn) + // subquery in insert values should not reference upper scope + usingPlan := mockTablePlan + if _, ok := valueItem.(*ast.SubqueryExpr); ok { + usingPlan = LogicalTableDual{}.Init(b.ctx, b.getSelectOffset()) + } + expr, _, err = b.rewriteWithPreprocess(ctx, valueItem, usingPlan, nil, nil, true, checkRefColumn) } if err != nil { return err @@ -2311,6 +2353,12 @@ func (b *PlanBuilder) buildLoadData(ctx context.Context, ld *ast.LoadDataStmt) ( LinesInfo: ld.LinesInfo, IgnoreLines: ld.IgnoreLines, } + user := b.ctx.GetSessionVars().User + var insertErr error + if user != nil { + insertErr = ErrTableaccessDenied.GenWithStackByArgs("INSERT", user.AuthUsername, user.AuthHostname, p.Table.Name.O) + } + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, p.Table.Schema.O, p.Table.Name.O, "", insertErr) tableInfo := p.Table.TableInfo tableInPlan, ok := b.is.TableByID(tableInfo.ID) if !ok { @@ -2926,6 +2974,8 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp return buildTableRegionsSchema() case ast.ShowEngines: names = []string{"Engine", "Support", "Comment", "Transactions", "XA", "Savepoints"} + case ast.ShowConfig: + names = []string{"Type", "Instance", "Name", "Value"} case ast.ShowDatabases: names = []string{"Database"} case ast.ShowOpenTables: @@ -3035,6 +3085,9 @@ func buildShowSchema(s *ast.ShowStmt, isView bool, isSequence bool) (schema *exp case ast.ShowBuiltins: names = []string{"Supported_builtin_functions"} ftypes = []byte{mysql.TypeVarchar} + case ast.ShowBackups, ast.ShowRestores: + names = []string{"Destination", "State", "Progress", "Queue_time", "Execution_time", "Finish_time", "Connection"} + ftypes = []byte{mysql.TypeVarchar, mysql.TypeVarchar, mysql.TypeDouble, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeDatetime, mysql.TypeLonglong} } schema = expression.NewSchema(make([]*expression.Column, 0, len(names))...) diff --git a/planner/core/planbuilder_test.go b/planner/core/planbuilder_test.go index 92cd04cbe6577..c3e9a16dfdf55 100644 --- a/planner/core/planbuilder_test.go +++ b/planner/core/planbuilder_test.go @@ -57,6 +57,8 @@ func (s *testPlanBuilderSuite) TestShow(c *C) { ast.ShowCreateDatabase, ast.ShowEvents, ast.ShowMasterStatus, + ast.ShowBackups, + ast.ShowRestores, } for _, tp := range tps { node.Tp = tp diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 0aaa79feb7485..36378bbdfb962 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/opcode" @@ -47,7 +48,7 @@ type PointGetPlan struct { TblInfo *model.TableInfo IndexInfo *model.IndexInfo PartitionInfo *model.PartitionDefinition - Handle int64 + Handle kv.Handle HandleParam *driver.ParamMarkerExpr IndexValues []types.Datum IndexValueParams []*driver.ParamMarkerExpr @@ -139,9 +140,9 @@ func (p *PointGetPlan) OperatorInfo(normalized bool) string { fmt.Fprintf(buffer, "handle:?, ") } else { if p.UnsignedHandle { - fmt.Fprintf(buffer, "handle:%d, ", uint64(p.Handle)) + fmt.Fprintf(buffer, "handle:%d, ", uint64(p.Handle.IntValue())) } else { - fmt.Fprintf(buffer, "handle:%d, ", p.Handle) + fmt.Fprintf(buffer, "handle:%s, ", p.Handle) } } } @@ -224,7 +225,7 @@ type BatchPointGetPlan struct { dbName string TblInfo *model.TableInfo IndexInfo *model.IndexInfo - Handles []int64 + Handles []kv.Handle HandleParams []*driver.ParamMarkerExpr IndexValues [][]types.Datum IndexValueParams [][]*driver.ParamMarkerExpr @@ -420,7 +421,7 @@ func newBatchPointGetPlan( return nil } if handleCol != nil { - var handles = make([]int64, len(patternInExpr.List)) + var handles = make([]kv.Handle, len(patternInExpr.List)) var handleParams = make([]*driver.ParamMarkerExpr, len(patternInExpr.List)) for i, item := range patternInExpr.List { // SELECT * FROM t WHERE (key) in ((1), (2)) @@ -450,7 +451,7 @@ func newBatchPointGetPlan( if err != nil || cmp != 0 { return nil } - handles[i] = intDatum.GetInt64() + handles[i] = kv.IntHandle(intDatum.GetInt64()) handleParams[i] = param } return BatchPointGetPlan{ @@ -652,6 +653,9 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP return nil } pi := tbl.GetPartitionInfo() + if pi != nil && pi.Type != model.PartitionTypeHash { + return nil + } for _, col := range tbl.Columns { // Do not handle generated columns. if col.IsGenerated() { @@ -662,53 +666,40 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP return nil } } + schema, names := buildSchemaFromFields(tblName.Schema, tbl, tblAlias, selStmt.Fields.Fields) + if schema == nil { + return nil + } + dbName := tblName.Schema.L + if dbName == "" { + dbName = ctx.GetSessionVars().CurrentDB + } + pairs := make([]nameValuePair, 0, 4) - pairs = getNameValuePairs(pairs, tblAlias, selStmt.Where) - if pairs == nil { + pairs, isTableDual := getNameValuePairs(ctx.GetSessionVars().StmtCtx, tbl, tblAlias, pairs, selStmt.Where) + if pairs == nil && !isTableDual { return nil } var partitionInfo *model.PartitionDefinition var pos int if pi != nil { - if pi.Type != model.PartitionTypeHash { - return nil - } partitionInfo, pos = getPartitionInfo(ctx, tbl, pairs) if partitionInfo == nil { return nil } } + handlePair, fieldType := findPKHandle(tbl, pairs) if handlePair.value.Kind() != types.KindNull && len(pairs) == 1 { - schema, names := buildSchemaFromFields(tblName.Schema, tbl, tblAlias, selStmt.Fields.Fields) - if schema == nil { - return nil - } - dbName := tblName.Schema.L - if dbName == "" { - dbName = ctx.GetSessionVars().CurrentDB - } - p := newPointGetPlan(ctx, dbName, schema, tbl, names) - intDatum, err := handlePair.value.ConvertTo(ctx.GetSessionVars().StmtCtx, fieldType) - if err != nil { - if terror.ErrorEqual(types.ErrOverflow, err) { - p.IsTableDual = true - return p - } - // some scenarios cast to int with error, but we may use this value in point get - if !terror.ErrorEqual(types.ErrTruncatedWrongVal, err) { - return nil - } - } - cmp, err := intDatum.CompareDatum(ctx.GetSessionVars().StmtCtx, &handlePair.value) - if err != nil { - return nil - } else if cmp != 0 { + if isTableDual { + p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names) p.IsTableDual = true return p } - p.Handle = intDatum.GetInt64() + + p := newPointGetPlan(ctx, dbName, schema, tbl, names) + p.Handle = kv.IntHandle(handlePair.value.GetInt64()) p.UnsignedHandle = mysql.HasUnsignedFlag(fieldType.Flag) p.HandleParam = handlePair.param p.PartitionInfo = partitionInfo @@ -722,18 +713,16 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP if idxInfo.State != model.StatePublic { continue } + if isTableDual { + p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names) + p.IsTableDual = true + return p + } + idxValues, idxValueParams := getIndexValues(idxInfo, pairs) if idxValues == nil { continue } - schema, names := buildSchemaFromFields(tblName.Schema, tbl, tblAlias, selStmt.Fields.Fields) - if schema == nil { - return nil - } - dbName := tblName.Schema.L - if dbName == "" { - dbName = ctx.GetSessionVars().CurrentDB - } p := newPointGetPlan(ctx, dbName, schema, tbl, names) p.IndexInfo = idxInfo p.IndexValues = idxValues @@ -864,21 +853,22 @@ func getSingleTableNameAndAlias(tableRefs *ast.TableRefsClause) (tblName *ast.Ta } // getNameValuePairs extracts `column = constant/paramMarker` conditions from expr as name value pairs. -func getNameValuePairs(nvPairs []nameValuePair, tblName model.CIStr, expr ast.ExprNode) []nameValuePair { +func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, tblName model.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) ( + pairs []nameValuePair, isTableDual bool) { binOp, ok := expr.(*ast.BinaryOperationExpr) if !ok { - return nil + return nil, false } if binOp.Op == opcode.LogicAnd { - nvPairs = getNameValuePairs(nvPairs, tblName, binOp.L) - if nvPairs == nil { - return nil + nvPairs, isTableDual = getNameValuePairs(stmtCtx, tbl, tblName, nvPairs, binOp.L) + if nvPairs == nil || isTableDual { + return nil, isTableDual } - nvPairs = getNameValuePairs(nvPairs, tblName, binOp.R) - if nvPairs == nil { - return nil + nvPairs, isTableDual = getNameValuePairs(stmtCtx, tbl, tblName, nvPairs, binOp.R) + if nvPairs == nil || isTableDual { + return nil, isTableDual } - return nvPairs + return nvPairs, isTableDual } else if binOp.Op == opcode.EQ { var d types.Datum var colName *ast.ColumnNameExpr @@ -901,17 +891,44 @@ func getNameValuePairs(nvPairs []nameValuePair, tblName model.CIStr, expr ast.Ex param = x } } else { - return nil + return nil, false } if d.IsNull() { - return nil + return nil, false + } + // Views' columns have no FieldType. + if tbl.IsView() { + return nil, false } if colName.Name.Table.L != "" && colName.Name.Table.L != tblName.L { - return nil + return nil, false + } + col := model.FindColumnInfo(tbl.Cols(), colName.Name.Name.L) + if col == nil || // Handling the case when the column is _tidb_rowid. + (col.Tp == mysql.TypeString && col.Collate == charset.CollationBin) { // This type we needn't to pad `\0` in here. + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}), false } - return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}) + dVal, err := d.ConvertTo(stmtCtx, &col.FieldType) + if err != nil { + if terror.ErrorEqual(types.ErrOverflow, err) { + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}), true + } + // Some scenarios cast to int with error, but we may use this value in point get. + if !terror.ErrorEqual(types.ErrTruncatedWrongVal, err) { + return nil, false + } + } + // The converted result must be same as original datum. + cmp, err := d.CompareDatum(stmtCtx, &dVal) + if err != nil { + return nil, false + } else if cmp != 0 { + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), true + } + + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), false } - return nil + return nil, false } func findPKHandle(tblInfo *model.TableInfo, pairs []nameValuePair) (handlePair nameValuePair, fieldType *types.FieldType) { diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index e47046033748a..401259bae257c 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -16,7 +16,6 @@ package core_test import ( "context" "fmt" - "github.com/pingcap/tidb/util/testutil" "math" "strings" @@ -31,6 +30,7 @@ import ( "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" + "github.com/pingcap/tidb/util/testutil" dto "github.com/prometheus/client_model/go" ) diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 41100c92083b6..79532ea2de819 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -392,7 +392,7 @@ func (s *testPrepareSuite) TestPrepareForGroupByItems(c *C) { tk.MustQuery("execute s1 using @a;").Check(testkit.Rows("3")) } -func (s *testPrepareSuite) TestPrepareCacheForPartition(c *C) { +func (s *testPrepareSerialSuite) TestPrepareCacheForPartition(c *C) { defer testleak.AfterTest(c)() store, dom, err := newStoreWithBootstrap() c.Assert(err, IsNil) @@ -675,3 +675,76 @@ func (s *testPlanSerialSuite) TestPlanCacheUnionScan(c *C) { cnt = pb.GetCounter().GetValue() c.Check(cnt, Equals, float64(6)) } + +func (s *testPlanSerialSuite) TestPlanCacheHitInfo(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + orgEnable := core.PreparedPlanCacheEnabled() + defer func() { + dom.Close() + store.Close() + core.SetPreparedPlanCache(orgEnable) + }() + core.SetPreparedPlanCache(true) + + tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + c.Assert(err, IsNil) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int)") + tk.MustExec("insert into t values (1),(2),(3),(4)") + tk.MustExec("prepare stmt from 'select * from t where id=?'") + tk.MustExec("prepare stmt2 from 'select /*+ ignore_plan_cache() */ * from t where id=?'") + tk.MustExec("set @doma = 1") + // Test if last_plan_from_cache is appropriately initialized. + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustQuery("execute stmt using @doma").Check(testkit.Rows("1")) + // Test if last_plan_from_cache is updated after a plan cache hit. + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + tk.MustQuery("execute stmt using @doma").Check(testkit.Rows("1")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustQuery("execute stmt2 using @doma").Check(testkit.Rows("1")) + // Test if last_plan_from_cache is updated after a plan cache miss caused by a prepared statement. + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + // Test if last_plan_from_cache is updated after a plan cache miss caused by a usual statement. + tk.MustQuery("execute stmt using @doma").Check(testkit.Rows("1")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + tk.MustQuery("select * from t where id=1").Check(testkit.Rows("1")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) +} + +func (s *testPrepareSuite) TestPrepareForGroupByMultiItems(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + defer func() { + dom.Close() + store.Close() + }() + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, c int , index idx(a));") + tk.MustExec("insert into t values(1,2, -1), (1,2, 1), (1,2, -1), (4,4,3);") + tk.MustExec("set @a=1") + tk.MustExec("set @b=3") + tk.MustExec(`set sql_mode=""`) + tk.MustExec(`prepare stmt from "select a, sum(b), c from t group by ?, ? order by ?, ?"`) + tk.MustQuery("select a, sum(b), c from t group by 1,3 order by 1,3;").Check(testkit.Rows("1 4 -1", "1 2 1", "4 4 3")) + tk.MustQuery(`execute stmt using @a, @b, @a, @b`).Check(testkit.Rows("1 4 -1", "1 2 1", "4 4 3")) + + tk.MustExec("set @c=10") + err = tk.ExecToErr("execute stmt using @a, @c, @a, @c") + c.Assert(err.Error(), Equals, "Unknown column '10' in 'group statement'") + + tk.MustExec("set @v1=1.0") + tk.MustExec("set @v2=3.0") + tk.MustExec(`prepare stmt2 from "select sum(b) from t group by ?, ?"`) + tk.MustQuery(`execute stmt2 using @v1, @v2`).Check(testkit.Rows("10")) +} diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 3080205b021f8..8d9f7e8a2b88d 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/types/parser_driver" + driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/domainutil" ) @@ -154,6 +154,10 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { if node.FnName.L == ast.NextVal || node.FnName.L == ast.LastVal || node.FnName.L == ast.SetVal { p.flag |= inSequenceFunction } + case *ast.BRIEStmt: + if node.Kind == ast.BRIEKindRestore { + p.flag |= inCreateOrDropTable + } default: p.flag &= ^parentIsJoin } @@ -247,6 +251,10 @@ func (p *preprocessor) Leave(in ast.Node) (out ast.Node, ok bool) { p.flag &= ^inRepairTable case *ast.CreateSequenceStmt: p.flag &= ^inCreateOrDropTable + case *ast.BRIEStmt: + if x.Kind == ast.BRIEKindRestore { + p.flag &= ^inCreateOrDropTable + } } return in, p.err == nil diff --git a/planner/core/rule_aggregation_elimination.go b/planner/core/rule_aggregation_elimination.go index 639e0e400ef6f..7af0511c12864 100644 --- a/planner/core/rule_aggregation_elimination.go +++ b/planner/core/rule_aggregation_elimination.go @@ -109,7 +109,7 @@ func rewriteCount(ctx sessionctx.Context, exprs []expression.Expression, targetT isNullExprs := make([]expression.Expression, 0, len(exprs)) for _, expr := range exprs { if mysql.HasNotNullFlag(expr.GetType().Flag) { - isNullExprs = append(isNullExprs, expression.Zero) + isNullExprs = append(isNullExprs, expression.NewZero()) } else { isNullExpr := expression.NewFunctionInternal(ctx, ast.IsNull, types.NewFieldType(mysql.TypeTiny), expr) isNullExprs = append(isNullExprs, isNullExpr) @@ -117,7 +117,7 @@ func rewriteCount(ctx sessionctx.Context, exprs []expression.Expression, targetT } innerExpr := expression.ComposeDNFCondition(ctx, isNullExprs...) - newExpr := expression.NewFunctionInternal(ctx, ast.If, targetTp, innerExpr, expression.Zero, expression.One) + newExpr := expression.NewFunctionInternal(ctx, ast.If, targetTp, innerExpr, expression.NewZero(), expression.NewOne()) return newExpr } @@ -127,7 +127,7 @@ func rewriteBitFunc(ctx sessionctx.Context, funcType string, arg expression.Expr outerCast := wrapCastFunction(ctx, innerCast, targetTp) var finalExpr expression.Expression if funcType != ast.AggFuncBitAnd { - finalExpr = expression.NewFunctionInternal(ctx, ast.Ifnull, targetTp, outerCast, expression.Zero.Clone()) + finalExpr = expression.NewFunctionInternal(ctx, ast.Ifnull, targetTp, outerCast, expression.NewZero()) } else { finalExpr = expression.NewFunctionInternal(ctx, ast.Ifnull, outerCast.GetType(), outerCast, &expression.Constant{Value: types.NewUintDatum(math.MaxUint64), RetType: targetTp}) } diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 7091bd453d72e..2a8a5109e9891 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -244,6 +244,9 @@ func (a *aggregationPushDownSolver) checkAnyCountAndSum(aggFuncs []*aggregation. return false } +// TODO: +// 1. https://github.com/pingcap/tidb/issues/16355, push avg & distinct functions across join +// 2. remove this method and use splitPartialAgg instead for clean code. func (a *aggregationPushDownSolver) makeNewAgg(ctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc, gbyCols []*expression.Column, aggHints aggHintInfo, blockOffset int) (*LogicalAggregation, error) { agg := LogicalAggregation{ GroupByItems: expression.Column2Exprs(gbyCols), @@ -275,6 +278,27 @@ func (a *aggregationPushDownSolver) makeNewAgg(ctx sessionctx.Context, aggFuncs return agg, nil } +func (a *aggregationPushDownSolver) splitPartialAgg(agg *LogicalAggregation) (pushedAgg *LogicalAggregation) { + partial, final, _ := BuildFinalModeAggregation(agg.ctx, &AggInfo{ + AggFuncs: agg.AggFuncs, + GroupByItems: agg.GroupByItems, + Schema: agg.schema, + }, false) + agg.SetSchema(final.Schema) + agg.AggFuncs = final.AggFuncs + agg.GroupByItems = final.GroupByItems + agg.collectGroupByColumns() + + pushedAgg = LogicalAggregation{ + AggFuncs: partial.AggFuncs, + GroupByItems: partial.GroupByItems, + aggHints: agg.aggHints, + }.Init(agg.ctx, agg.blockOffset) + pushedAgg.SetSchema(partial.Schema) + pushedAgg.collectGroupByColumns() + return +} + // pushAggCrossUnion will try to push the agg down to the union. If the new aggregation's group-by columns doesn't contain unique key. // We will return the new aggregation. Otherwise we will transform the aggregation to projection. func (a *aggregationPushDownSolver) pushAggCrossUnion(agg *LogicalAggregation, unionSchema *expression.Schema, unionChild LogicalPlan) LogicalPlan { @@ -379,12 +403,7 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan) (_ LogicalPlan, e projChild := proj.children[0] agg.SetChildren(projChild) } else if union, ok1 := child.(*LogicalUnionAll); ok1 { - var gbyCols []*expression.Column - gbyCols = expression.ExtractColumnsFromExpressions(gbyCols, agg.GroupByItems, nil) - pushedAgg, err := a.makeNewAgg(agg.ctx, agg.AggFuncs, gbyCols, agg.aggHints, agg.blockOffset) - if err != nil { - return nil, err - } + pushedAgg := a.splitPartialAgg(agg) newChildren := make([]LogicalPlan, 0, len(union.children)) for _, child := range union.children { newChild := a.pushAggCrossUnion(pushedAgg, union.Schema(), child) diff --git a/planner/core/rule_build_key_info.go b/planner/core/rule_build_key_info.go index aaf9de1efc5fc..20863e95c3042 100644 --- a/planner/core/rule_build_key_info.go +++ b/planner/core/rule_build_key_info.go @@ -15,6 +15,7 @@ package core import ( "context" + "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index c35f17a6a0cdf..2ec433bdffa62 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -100,7 +100,7 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) } if len(la.AggFuncs) == 0 { // If all the aggregate functions are pruned, we should add an aggregate function to keep the correctness. - one, err := aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncFirstRow, []expression.Expression{expression.One}, false) + one, err := aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncFirstRow, []expression.Expression{expression.NewOne()}, false) if err != nil { return err } @@ -124,7 +124,7 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) // If all the group by items are pruned, we should add a constant 1 to keep the correctness. // Because `select count(*) from t` is different from `select count(*) from t group by 1`. if len(la.GroupByItems) == 0 { - la.GroupByItems = []expression.Expression{expression.One} + la.GroupByItems = []expression.Expression{expression.NewOne()} } } return child.PruneColumns(selfUsedCols) @@ -138,7 +138,7 @@ func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column) error { for i := len(ls.ByItems) - 1; i >= 0; i-- { cols := expression.ExtractColumns(ls.ByItems[i].Expr) if len(cols) == 0 { - if expression.IsMutableEffectsExpr(ls.ByItems[i].Expr) { + if !expression.IsRuntimeConstExpr(ls.ByItems[i].Expr) { continue } ls.ByItems = append(ls.ByItems[:i], ls.ByItems[i+1:]...) diff --git a/planner/core/rule_decorrelate.go b/planner/core/rule_decorrelate.go index 37f12cf5ba170..7027a44f78994 100644 --- a/planner/core/rule_decorrelate.go +++ b/planner/core/rule_decorrelate.go @@ -99,7 +99,7 @@ func (s *decorrelateSolver) aggDefaultValueMap(agg *LogicalAggregation) map[int] for i, f := range agg.AggFuncs { switch f.Name { case ast.AggFuncBitOr, ast.AggFuncBitXor, ast.AggFuncCount: - defaultValueMap[i] = expression.Zero.Clone().(*expression.Constant) + defaultValueMap[i] = expression.NewZero() case ast.AggFuncBitAnd: defaultValueMap[i] = &expression.Constant{Value: types.NewUintDatum(math.MaxUint64), RetType: types.NewFieldType(mysql.TypeLonglong)} } diff --git a/planner/core/rule_generate_column_substitute.go b/planner/core/rule_generate_column_substitute.go index 3b3e75c929a46..5b65cefb93fae 100644 --- a/planner/core/rule_generate_column_substitute.go +++ b/planner/core/rule_generate_column_substitute.go @@ -62,7 +62,7 @@ func collectGenerateColumn(lp LogicalPlan, exprToColumn ExprColumnMap) { if colInfo.IsGenerated() && !colInfo.GeneratedStored { s := ds.schema.Columns col := expression.ColInfo2Col(s, colInfo) - if col != nil { + if col != nil && col.GetType().Equal(col.VirtualExpr.GetType()) { exprToColumn[col.VirtualExpr] = col } } diff --git a/planner/core/rule_inject_extra_projection.go b/planner/core/rule_inject_extra_projection.go index b13278750d72e..ec0dbd31c0239 100644 --- a/planner/core/rule_inject_extra_projection.go +++ b/planner/core/rule_inject_extra_projection.go @@ -66,9 +66,10 @@ func wrapCastForAggFuncs(sctx sessionctx.Context, aggFuncs []*aggregation.AggFun } } -// InjectProjBelowAgg injects a ProjOperator below AggOperator. If all the args -// of `aggFuncs`, and all the item of `groupByItems` are columns or constants, -// we do not need to build the `proj`. +// InjectProjBelowAgg injects a ProjOperator below AggOperator. So that All +// scalar functions in aggregation may speed up by vectorized evaluation in +// the `proj`. If all the args of `aggFuncs`, and all the item of `groupByItems` +// are columns or constants, we do not need to build the `proj`. func InjectProjBelowAgg(aggPlan PhysicalPlan, aggFuncs []*aggregation.AggFuncDesc, groupByItems []expression.Expression) PhysicalPlan { hasScalarFunc := false diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index b0933425af6e0..a415879092717 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -369,9 +369,28 @@ func makePartitionByFnCol(sctx sessionctx.Context, columns []*expression.Column, var fn *expression.ScalarFunction switch raw := partExpr.(type) { case *expression.ScalarFunction: + // Special handle for floor(unix_timestamp(ts)) as partition expression. + // This pattern is so common for timestamp(3) column as partition expression that it deserve an optimization. + if raw.FuncName.L == ast.Floor { + if ut, ok := raw.GetArgs()[0].(*expression.ScalarFunction); ok && ut.FuncName.L == ast.UnixTimestamp { + args := ut.GetArgs() + if len(args) == 1 { + if c, ok1 := args[0].(*expression.Column); ok1 { + return c, raw, nil + } + } + } + } + if _, ok := monotoneIncFuncs[raw.FuncName.L]; ok { fn = raw - col = fn.GetArgs()[0].(*expression.Column) + args := fn.GetArgs() + if len(args) > 0 { + arg0 := args[0] + if c, ok1 := arg0.(*expression.Column); ok1 { + col = c + } + } } case *expression.Column: col = raw @@ -501,10 +520,8 @@ func (p *rangePruner) extractDataForPrune(sctx sessionctx.Context, expr expressi var constExpr expression.Expression if p.partFn != nil { // If the partition expression is fn(col), change constExpr to fn(constExpr). - // No 'copy on write' for the expression here, this is a dangerous operation. - args := p.partFn.GetArgs() - args[0] = con - constExpr = p.partFn + constExpr = replaceColumnWithConst(p.partFn, con) + // Sometimes we need to relax the condition, < to <=, > to >=. // For example, the following case doesn't hold: // col < '2020-02-11 17:34:11' => to_days(col) < to_days(2020-02-11 17:34:11) @@ -523,6 +540,25 @@ func (p *rangePruner) extractDataForPrune(sctx sessionctx.Context, expr expressi return ret, false } +// replaceColumnWithConst change fn(col) to fn(const) +func replaceColumnWithConst(partFn *expression.ScalarFunction, con *expression.Constant) *expression.ScalarFunction { + args := partFn.GetArgs() + // The partition function may be floor(unix_timestamp(ts)) instead of a simple fn(col). + if partFn.FuncName.L == ast.Floor { + ut := args[0].(*expression.ScalarFunction) + if ut.FuncName.L == ast.UnixTimestamp { + args = ut.GetArgs() + args[0] = con + return partFn + } + } + + // No 'copy on write' for the expression here, this is a dangerous operation. + args[0] = con + return partFn + +} + // opposite turns > to <, >= to <= and so on. func opposite(op string) string { switch op { diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index b5d0291ee01fc..0d1951ab35521 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -316,6 +316,10 @@ func simplifyOuterJoin(p *LogicalJoin, predicates []expression.Expression) { // then simplify embedding outer join. canBeSimplified := false for _, expr := range predicates { + // avoid the case where the expr only refers to the schema of outerTable + if expression.ExprFromSchema(expr, outerTable.Schema()) { + continue + } isOk := isNullRejected(p.ctx, innerTable.Schema(), expr) if isOk { canBeSimplified = true @@ -333,7 +337,7 @@ func simplifyOuterJoin(p *LogicalJoin, predicates []expression.Expression) { // If it is a conjunction containing a null-rejected condition as a conjunct. // If it is a disjunction of null-rejected conditions. func isNullRejected(ctx sessionctx.Context, schema *expression.Schema, expr expression.Expression) bool { - expr = expression.PushDownNot(nil, expr) + expr = expression.PushDownNot(ctx, expr) sc := ctx.GetSessionVars().StmtCtx sc.InNullRejectCheck = true result := expression.EvaluateExprWithNull(ctx, schema, expr) diff --git a/planner/core/stats.go b/planner/core/stats.go index 0d87d579dad03..17d3f3264f09d 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -378,6 +378,14 @@ func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, us logutil.BgLogger().Debug("can not derive statistics of a path", zap.Error(err)) continue } + if len(path.TableFilters) > 0 || len(path.AccessConds) == 0 { + // If AccessConds is empty or tableFilter is not empty, we ignore the access path. + // Now these conditions are too strict. + // For example, a sql `select * from t where a > 1 or (b < 2 and c > 3)` and table `t` with indexes + // on a and b separately. we can generate a `IndexMergePath` with table filter `a > 1 or (b < 2 and c > 3)`. + // TODO: solve the above case + continue + } // If we have point or empty range, just remove other possible paths. if noIntervalRanges || len(path.Ranges) == 0 { if len(results) == 0 { @@ -399,6 +407,14 @@ func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, us continue } noIntervalRanges := ds.deriveIndexPathStats(path, conditions, true) + if len(path.TableFilters) > 0 || len(path.AccessConds) == 0 { + // If AccessConds is empty or tableFilter is not empty, we ignore the access path. + // Now these conditions are too strict. + // For example, a sql `select * from t where a > 1 or (b < 2 and c > 3)` and table `t` with indexes + // on a and b separately. we can generate a `IndexMergePath` with table filter `a > 1 or (b < 2 and c > 3)`. + // TODO: solve the above case + continue + } // If we have empty range, or point range on unique index, just remove other possible paths. if (noIntervalRanges && path.Index.Unique) || len(path.Ranges) == 0 { if len(results) == 0 { @@ -410,14 +426,6 @@ func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, us break } } - // If AccessConds is empty or tableFilter is not empty, we ignore the access path. - // Now these conditions are too strict. - // For example, a sql `select * from t where a > 1 or (b < 2 and c > 3)` and table `t` with indexes - // on a and b separately. we can generate a `IndexMergePath` with table filter `a > 1 or (b < 2 and c > 3)`. - // TODO: solve the above case - if len(path.TableFilters) > 0 || len(path.AccessConds) == 0 { - continue - } results = append(results, path) } return results diff --git a/planner/core/task.go b/planner/core/task.go index d61312df139dd..f5e5ed3d7f411 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -53,8 +53,8 @@ type copTask struct { indexPlanFinished bool // keepOrder indicates if the plan scans data by order. keepOrder bool - // In double read case, it may output one more column for handle(row id). - // We need to prune it, so we add a project do this. + // doubleReadNeedProj means an extra prune is needed because + // in double read case, it may output one more column for handle(row id). doubleReadNeedProj bool extraHandleCol *expression.Column @@ -657,45 +657,12 @@ func (p *PhysicalMergeJoin) attach2Task(tasks ...task) task { lTask := finishCopTask(p.ctx, tasks[0].copy()) rTask := finishCopTask(p.ctx, tasks[1].copy()) p.SetChildren(lTask.plan(), rTask.plan()) - p.schema = BuildPhysicalJoinSchema(p.JoinType, p) return &rootTask{ p: p, cst: lTask.cost() + rTask.cost() + p.GetCost(lTask.count(), rTask.count()), } } -// splitCopAvg2CountAndSum splits the cop avg function to count and sum. -// Now it's only used for TableReader. -func splitCopAvg2CountAndSum(p PhysicalPlan) { - var baseAgg *basePhysicalAgg - if agg, ok := p.(*PhysicalStreamAgg); ok { - baseAgg = &agg.basePhysicalAgg - } - if agg, ok := p.(*PhysicalHashAgg); ok { - baseAgg = &agg.basePhysicalAgg - } - if baseAgg == nil { - return - } - - schemaCursor := len(baseAgg.Schema().Columns) - len(baseAgg.GroupByItems) - for i := len(baseAgg.AggFuncs) - 1; i >= 0; i-- { - f := baseAgg.AggFuncs[i] - schemaCursor-- - if f.Name == ast.AggFuncAvg { - schemaCursor-- - sumAgg := *f - sumAgg.Name = ast.AggFuncSum - sumAgg.RetTp = baseAgg.Schema().Columns[schemaCursor+1].RetType - cntAgg := *f - cntAgg.Name = ast.AggFuncCount - cntAgg.RetTp = baseAgg.Schema().Columns[schemaCursor].RetType - cntAgg.RetTp.Flag = f.RetTp.Flag - baseAgg.AggFuncs = append(baseAgg.AggFuncs[:i], append([]*aggregation.AggFuncDesc{&cntAgg, &sumAgg}, baseAgg.AggFuncs[i+1:]...)...) - } - } -} - func buildIndexLookUpTask(ctx sessionctx.Context, t *copTask) *rootTask { newTask := &rootTask{cst: t.cst} sessVars := ctx.GetSessionVars() @@ -786,7 +753,6 @@ func finishCopTask(ctx sessionctx.Context, task task) task { newTask.p = p } else { tp := t.tablePlan - splitCopAvg2CountAndSum(tp) for len(tp.Children()) > 0 { if len(tp.Children()) == 1 { tp = tp.Children()[0] @@ -1117,9 +1083,10 @@ type AggInfo struct { } // BuildFinalModeAggregation splits either LogicalAggregation or PhysicalAggregation to finalAgg and partial1Agg, -// returns the body of finalAgg and the schema of partialAgg. +// returns the information of partial and final agg. +// partialIsCop means whether partial agg is a cop task. func BuildFinalModeAggregation( - sctx sessionctx.Context, original *AggInfo) (partial, final *AggInfo, funcMap map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc) { + sctx sessionctx.Context, original *AggInfo, partialIsCop bool) (partial, final *AggInfo, funcMap map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc) { funcMap = make(map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc, len(original.AggFuncs)) partial = &AggInfo{ @@ -1189,11 +1156,25 @@ func BuildFinalModeAggregation( } } partialGbySchema.Append(gbyCol) + if !partialIsCop { + // if partial is a cop task, firstrow function is redundant since group by items are outputted + // by group by schema, and final functions use group by schema as their arguments. + // if partial agg is not cop, we must append firstrow function & schema, to output the group by + // items. + // maybe we can unify them sometime. + firstRow, err := aggregation.NewAggFuncDesc(sctx, ast.AggFuncFirstRow, []expression.Expression{gbyCol}, false) + if err != nil { + panic("NewAggFuncDesc FirstRow meets error: " + err.Error()) + } + partial.AggFuncs = append(partial.AggFuncs, firstRow) + newCol, _ := gbyCol.Clone().(*expression.Column) + newCol.RetType = firstRow.RetTp + partial.Schema.Append(newCol) + partialCursor++ + } args = append(args, gbyCol) } } - // Just use groupBy items in Schema of partialAgg as arguments, - // no need to spawn FirstRow function. finalAggFunc.HasDistinct = true finalAggFunc.Mode = aggregation.CompleteMode @@ -1216,7 +1197,18 @@ func BuildFinalModeAggregation( args = append(args, partial.Schema.Columns[partialCursor]) partialCursor++ } - partial.AggFuncs = append(partial.AggFuncs, aggFunc) + if aggFunc.Name == ast.AggFuncAvg { + cntAgg := *aggFunc + cntAgg.Name = ast.AggFuncCount + cntAgg.RetTp = partial.Schema.Columns[partialCursor-2].GetType() + cntAgg.RetTp.Flag = aggFunc.RetTp.Flag + sumAgg := *aggFunc + sumAgg.Name = ast.AggFuncSum + sumAgg.RetTp = partial.Schema.Columns[partialCursor-1].GetType() + partial.AggFuncs = append(partial.AggFuncs, &cntAgg, &sumAgg) + } else { + partial.AggFuncs = append(partial.AggFuncs, aggFunc) + } finalAggFunc.Mode = aggregation.FinalMode funcMap[aggFunc] = finalAggFunc @@ -1239,7 +1231,7 @@ func (p *basePhysicalAgg) newPartialAggregate(copTaskType kv.StoreType) (partial AggFuncs: p.AggFuncs, GroupByItems: p.GroupByItems, Schema: p.Schema().Clone(), - }) + }, true) if p.tp == plancodec.TypeStreamAgg && len(partialPref.GroupByItems) != len(finalPref.GroupByItems) { return nil, p.self } diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index bd28fa364d0bd..2428560505ad6 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -207,7 +207,7 @@ "Plan": [ "StreamAgg_20 1.00 root funcs:avg(Column#7, Column#8)->Column#4", "└─IndexReader_21 1.00 root index:StreamAgg_8", - " └─StreamAgg_8 1.00 cop[tikv] funcs:avg(test.t.a)->Column#7", + " └─StreamAgg_8 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", " └─IndexFullScan_19 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" ], "Warn": [ @@ -219,7 +219,7 @@ "Plan": [ "StreamAgg_20 1.00 root funcs:avg(Column#7, Column#8)->Column#4", "└─IndexReader_21 1.00 root index:StreamAgg_8", - " └─StreamAgg_8 1.00 cop[tikv] funcs:avg(test.t.a)->Column#7", + " └─StreamAgg_8 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", " └─IndexFullScan_19 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" ], "Warn": null @@ -229,7 +229,7 @@ "Plan": [ "StreamAgg_20 1.00 root funcs:avg(Column#7, Column#8)->Column#4", "└─IndexReader_21 1.00 root index:StreamAgg_8", - " └─StreamAgg_8 1.00 cop[tikv] funcs:avg(test.t.a)->Column#7", + " └─StreamAgg_8 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", " └─IndexFullScan_19 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" ], "Warn": [ diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 5f9c9cc1fffae..18c95deffdc73 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -55,7 +55,9 @@ { "name": "TestIndexMerge", "cases": [ - "explain select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2" + "explain select /*+ USE_INDEX_MERGE(t, a, b) */ * from t where a = 1 or b = 2", + "explain select /*+ USE_INDEX_MERGE(t, primary) */ * from t where 1 or t.c", + "explain select /*+ USE_INDEX_MERGE(t, a, b, c) */ * from t where 1 or t.a = 1 or t.b = 2" ] }, { @@ -71,6 +73,18 @@ "desc select /*+ TIDB_INLJ(t2)*/ * from t1, t2 where t1.a = t2.a and t1.b = t2.a and t1.b = t2.b" ] }, + { + "name": "TestHintWithRequiredProperty", + "cases": [ + "desc select /*+ INL_JOIN(t2) */ * from t t1, t t2 where t1.a = t2.b order by t2.a", + "desc select /*+ INL_HASH_JOIN(t2) */ * from t t1, t t2 where t1.a = t2.b order by t2.a", + "desc select /*+ INL_MERGE_JOIN(t2)*/ t1.a, t2.a from t t1, t t2 ,t t3 where t1.a = t2.a and t3.a=t2.a", + "desc select * from t t1, (select /*+ HASH_AGG() */ b, max(a) from t t2 group by b) t2 where t1.b = t2.b order by t1.b", + "desc select /*+ INL_HASH_JOIN(t2) */ distinct t2.a from t t1 join t t2 on t1.a = t2.a", + // This hint cannot work, so choose another plan. + "desc select /*+ INL_JOIN(t2) */ * from t t1, t t2 where t1.a = t2.c order by t1.a" + ] + }, { "name": "TestIndexHintWarning", "cases": [ diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 4e4e3a3b17b6f..81faaa6637461 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -50,14 +50,14 @@ { "SQL": "explain select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b);", "Plan": [ - "HashJoin_8 9990.00 root inner join, equal:[eq(test.t.a, test.t.a)]", - "├─Selection_13(Build) 7992.00 root from_unixtime(cast(test.t.b))", - "│ └─TableReader_16 9990.00 root data:Selection_15", - "│ └─Selection_15 9990.00 cop[tikv] not(isnull(test.t.a))", - "│ └─TableFullScan_14 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─TableReader_12(Probe) 9990.00 root data:Selection_11", - " └─Selection_11 9990.00 cop[tikv] not(isnull(test.t.a))", - " └─TableFullScan_10 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + "HashJoin_7 9990.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─Selection_15(Build) 7992.00 root from_unixtime(cast(test.t.b))", + "│ └─TableReader_14 7992.00 root data:Selection_13", + "│ └─Selection_13 7992.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan_12 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader_11(Probe) 9990.00 root data:Selection_10", + " └─Selection_10 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan_9 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ] } ] @@ -203,6 +203,23 @@ "├─IndexRangeScan_6(Build) 1.00 cop[tikv] table:t, index:b(b) range:[2,2], keep order:false, stats:pseudo", "└─TableRowIDScan_7(Probe) 2.00 cop[tikv] table:t keep order:false, stats:pseudo" ] + }, + { + "SQL": "explain select /*+ USE_INDEX_MERGE(t, primary) */ * from t where 1 or t.c", + "Plan": [ + "IndexMerge_8 10000.00 root ", + "├─TableFullScan_5(Build) 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "├─TableRangeScan_6(Build) 6666.67 cop[tikv] table:t range:[-inf,0), (0,+inf], keep order:false, stats:pseudo", + "└─TableRowIDScan_7(Probe) 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain select /*+ USE_INDEX_MERGE(t, a, b, c) */ * from t where 1 or t.a = 1 or t.b = 2", + "Plan": [ + "TableReader_7 8000.00 root data:Selection_6", + "└─Selection_6 8000.00 cop[tikv] or(or(1, eq(test.t.a, 1)), eq(test.t.b, 2))", + " └─TableFullScan_5 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] } ] }, @@ -255,6 +272,97 @@ } ] }, + { + "Name": "TestHintWithRequiredProperty", + "Cases": [ + { + "SQL": "desc select /*+ INL_JOIN(t2) */ * from t t1, t t2 where t1.a = t2.b order by t2.a", + "Plan": [ + "Sort_7 12487.50 root test.t.a:asc", + "└─IndexJoin_15 12487.50 root inner join, inner:IndexLookUp_14, outer key:test.t.a, inner key:test.t.b", + " ├─TableReader_25(Build) 10000.00 root data:TableFullScan_24", + " │ └─TableFullScan_24 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─IndexLookUp_14(Probe) 1.25 root ", + " ├─Selection_13(Build) 1.25 cop[tikv] not(isnull(test.t.b))", + " │ └─IndexRangeScan_11 1.25 cop[tikv] table:t2, index:b(b) range: decided by [eq(test.t.b, test.t.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan_12(Probe) 1.25 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warnings": [] + }, + { + "SQL": "desc select /*+ INL_HASH_JOIN(t2) */ * from t t1, t t2 where t1.a = t2.b order by t2.a", + "Plan": [ + "Sort_7 12487.50 root test.t.a:asc", + "└─IndexHashJoin_23 12487.50 root inner join, inner:IndexLookUp_14, outer key:test.t.a, inner key:test.t.b", + " ├─TableReader_25(Build) 10000.00 root data:TableFullScan_24", + " │ └─TableFullScan_24 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─IndexLookUp_14(Probe) 1.25 root ", + " ├─Selection_13(Build) 1.25 cop[tikv] not(isnull(test.t.b))", + " │ └─IndexRangeScan_11 1.25 cop[tikv] table:t2, index:b(b) range: decided by [eq(test.t.b, test.t.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan_12(Probe) 1.25 cop[tikv] table:t2 keep order:false, stats:pseudo" + ], + "Warnings": [] + }, + { + "SQL": "desc select /*+ INL_MERGE_JOIN(t2)*/ t1.a, t2.a from t t1, t t2 ,t t3 where t1.a = t2.a and t3.a=t2.a", + "Plan": [ + "HashJoin_21 15625.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─TableReader_60(Build) 10000.00 root data:TableFullScan_59", + "│ └─TableFullScan_59 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", + "└─IndexMergeJoin_56(Probe) 12500.00 root inner join, inner:TableReader_54, outer key:test.t.a, inner key:test.t.a", + " ├─TableReader_45(Build) 10000.00 root data:TableFullScan_44", + " │ └─TableFullScan_44 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader_54(Probe) 1.00 root data:TableRangeScan_53", + " └─TableRangeScan_53 1.00 cop[tikv] table:t2 range: decided by [test.t.a], keep order:true, stats:pseudo" + ], + "Warnings": [] + }, + { + "SQL": "desc select * from t t1, (select /*+ HASH_AGG() */ b, max(a) from t t2 group by b) t2 where t1.b = t2.b order by t1.b", + "Plan": [ + "Sort_10 9990.00 root test.t.b:asc", + "└─Projection_12 9990.00 root test.t.a, test.t.b, test.t.c, test.t.b, Column#7", + " └─HashJoin_27 9990.00 root inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─HashAgg_47(Build) 7992.00 root group by:test.t.b, funcs:max(Column#10)->Column#7, funcs:firstrow(test.t.b)->test.t.b", + " │ └─IndexReader_48 7992.00 root index:HashAgg_44", + " │ └─HashAgg_44 7992.00 cop[tikv] group by:test.t.b, funcs:max(test.t.a)->Column#10", + " │ └─IndexFullScan_37 9990.00 cop[tikv] table:t2, index:b(b) keep order:false, stats:pseudo", + " └─TableReader_51(Probe) 9990.00 root data:Selection_50", + " └─Selection_50 9990.00 cop[tikv] not(isnull(test.t.b))", + " └─TableFullScan_49 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warnings": [] + }, + { + "SQL": "desc select /*+ INL_HASH_JOIN(t2) */ distinct t2.a from t t1 join t t2 on t1.a = t2.a", + "Plan": [ + "HashAgg_8 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", + "└─IndexHashJoin_20 12500.00 root inner join, inner:TableReader_13, outer key:test.t.a, inner key:test.t.a", + " ├─TableReader_22(Build) 10000.00 root data:TableFullScan_21", + " │ └─TableFullScan_21 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader_13(Probe) 1.00 root data:TableRangeScan_12", + " └─TableRangeScan_12 1.00 cop[tikv] table:t2 range: decided by [test.t.a], keep order:false, stats:pseudo" + ], + "Warnings": [] + }, + { + "SQL": "desc select /*+ INL_JOIN(t2) */ * from t t1, t t2 where t1.a = t2.c order by t1.a", + "Plan": [ + "Sort_7 12487.50 root test.t.a:asc", + "└─HashJoin_19 12487.50 root inner join, equal:[eq(test.t.a, test.t.c)]", + " ├─TableReader_23(Build) 9990.00 root data:Selection_22", + " │ └─Selection_22 9990.00 cop[tikv] not(isnull(test.t.c))", + " │ └─TableFullScan_21 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader_25(Probe) 10000.00 root data:TableFullScan_24", + " └─TableFullScan_24 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warnings": [ + "[planner:1815]Optimizer Hint /*+ INL_JOIN(t2) */ or /*+ TIDB_INLJ(t2) */ is inapplicable", + "[planner:1815]Optimizer Hint /*+ INL_JOIN(t2) */ or /*+ TIDB_INLJ(t2) */ is inapplicable" + ] + } + ] + }, { "Name": "TestIndexHintWarning", "Cases": [ diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index ef04e0cd3fbcc..31dab48fa1bcc 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -60,7 +60,9 @@ "select /*+ NO_INDEX_MERGE(), USE_INDEX_MERGE(t, primary, f_g, c_d_e) */ * from t where a < 1 or f > 2", "select /*+ USE_INDEX_MERGE(t1, c_d_e, f_g) */ * from t where c < 1 or f > 2", "select /*+ NO_INDEX_MERGE(), USE_INDEX_MERGE(t, primary, f_g, c_d_e) */ * from t where a < 1 or f > 2", - "select /*+ USE_INDEX_MERGE(t) USE_INDEX_MERGE(t) */ * from t where c < 1 or f > 2" + "select /*+ USE_INDEX_MERGE(t) USE_INDEX_MERGE(t) */ * from t where c < 1 or f > 2", + "select /*+ USE_INDEX_MERGE(db2.t) */ * from t where c < 1 or f > 2", + "select /*+ USE_INDEX_MERGE(db2.t, c_d_e, f_g) */ * from t where c < 1 or f > 2" ] }, { @@ -544,7 +546,10 @@ "select /*+ STREAM_AGG() */ count(distinct c) from t;", // can not push down because c is not in group by "select /*+ HASH_AGG() */ count(distinct c) from t;", // can push down "select count(distinct c) from t group by c;", - "select count(distinct c) from t;" // should not use streamAgg because c is not in group by + "select count(distinct c) from t;", // should not use streamAgg because c is not in group by + + "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;" ] }, { @@ -557,7 +562,17 @@ "select /*+ STREAM_AGG(), AGG_TO_COP() */ count(distinct c) from t group by c;", "select /*+ STREAM_AGG(), AGG_TO_COP() */ count(distinct c) from t;", "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct c) from t;", - "select /*+ AGG_TO_COP() */ count(distinct c) from t group by c;" + "select /*+ AGG_TO_COP() */ count(distinct c) from t group by c;", + + "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;" + ] + }, + { + "name": "TestPushdownDistinctEnableAggPushDownDisable", + "cases": [ + "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;" ] }, { @@ -599,7 +614,8 @@ "select 1 from (select /*+ HASH_JOIN(t1) */ t1.a not in (select t2.a from t2) from t1) x;", "select /*+ INL_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", "select /*+ INL_HASH_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", - "select /*+ INL_MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;" + "select /*+ INL_MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "select /*+ MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;" ] }, { diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 92fb917ac8a79..652c55f7a5623 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -4,15 +4,15 @@ "Cases": [ { "SQL": "select /*+ MERGE_JOIN(t1) */ t1.a, t1.b from t t1, (select /*+ INL_JOIN(t3) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - "Best": "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)}(test.t.a,test.t.a)->Projection" + "Best": "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)}(test.t.a,test.t.a)" }, { "SQL": "select /*+ MERGE_JOIN(test.t1) */ t1.a, t1.b from t t1, (select /*+ INL_JOIN(test.t3) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - "Best": "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)}(test.t.a,test.t.a)->Projection" + "Best": "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)}(test.t.a,test.t.a)" }, { "SQL": "select /*+ MERGE_JOIN(t1) */ t1.a, t1.b from t t1, (select /*+ HASH_JOIN(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - "Best": "MergeInnerJoin{TableReader(Table(t))->LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)->Sort}(test.t.a,test.t.a)->Projection" + "Best": "MergeInnerJoin{TableReader(Table(t))->LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)->Sort}(test.t.a,test.t.a)" }, { "SQL": "select /*+ INL_JOIN(t1) */ t1.a, t1.b from t t1, (select /*+ HASH_JOIN(t2) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", @@ -40,7 +40,7 @@ }, { "SQL": "select /*+ MERGE_JOIN(t1) */ t1.a, t1.b from t t1, (select t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - "Best": "MergeInnerJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)}(test.t.a,test.t.a)->Projection" + "Best": "MergeInnerJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)}(test.t.a,test.t.a)" }, { "SQL": "select /*+ INL_JOIN(t1) */ t1.a, t1.b from t t1, (select t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", @@ -229,6 +229,18 @@ "Best": "IndexMergeReader(PartialPlans->[Index(t.c_d_e)[[-inf,1)], Index(t.f_g)[(2,+inf]]], TablePlan->Table(t))", "HasWarn": false, "Hints": "use_index_merge(@`sel_1` `t` `c_d_e`, `f_g`)" + }, + { + "SQL": "select /*+ USE_INDEX_MERGE(db2.t) */ * from t where c < 1 or f > 2", + "Best": "TableReader(Table(t)->Sel([or(lt(test.t.c, 1), gt(test.t.f, 2))]))", + "HasWarn": true, + "Hints": "use_index(@`sel_1` `test`.`t` )" + }, + { + "SQL": "select /*+ USE_INDEX_MERGE(db2.t, c_d_e, f_g) */ * from t where c < 1 or f > 2", + "Best": "TableReader(Table(t)->Sel([or(lt(test.t.c, 1), gt(test.t.f, 2))]))", + "HasWarn": true, + "Hints": "use_index(@`sel_1` `test`.`t` )" } ] }, @@ -579,7 +591,7 @@ }, { "SQL": "select * from t where a in (select s.a from t s) order by t.a", - "Best": "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.t.a)->Projection" + "Best": "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.t.a)" }, { "SQL": "select * from t where exists (select s.a from t s where s.c in (select c from t as k where k.d = s.d) having sum(s.a) = t.a )", @@ -927,7 +939,7 @@ }, { "SQL": "select a from t where not a", - "Best": "IndexReader(Index(t.f)[[NULL,+inf]]->Sel([not(test.t.a)]))" + "Best": "PointGet(Handle(t.a)0)" }, { "SQL": "select a from t where c in (1)", @@ -1229,7 +1241,7 @@ { "SQL": "select /*+ STREAM_AGG() */ sum(t1.a) from t t1 join t t2 on t1.b = t2.b group by t1.b", "Best": "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))->Sort->Projection->StreamAgg}(test.t.b,test.t.b)->HashAgg", - "Warning": "[planner:1815]Optimizer Hint STREAM_AGG is inapplicable" + "Warning": "" }, { "SQL": "select /*+ STREAM_AGG() */ e, sum(b) from t group by e", @@ -1243,12 +1255,12 @@ "Cases": [ { "SQL": "select /*+ MERGE_JOIN(@sel_1 t1), INL_JOIN(@sel_2 t3) */ t1.a, t1.b from t t1, (select t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - "Plan": "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)}(test.t.a,test.t.a)->Projection", + "Plan": "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)}(test.t.a,test.t.a)", "Hints": "use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_2` `test`.`t2` ), use_index(@`sel_2` `test`.`t3` `c_d_e`), inl_join(@`sel_2` `test`.`t3`), merge_join(@`sel_1` `test`.`t1`)" }, { "SQL": "select /*+ MERGE_JOIN(@sel_1 t1), INL_JOIN(@qb t3) */ t1.a, t1.b from t t1, (select /*+ QB_NAME(qb) */ t2.a from t t2, t t3 where t2.a = t3.c) s where t1.a=s.a", - "Plan": "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)}(test.t.a,test.t.a)->Projection", + "Plan": "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t.a,test.t.c)}(test.t.a,test.t.a)", "Hints": "use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_2` `test`.`t2` ), use_index(@`sel_2` `test`.`t3` `c_d_e`), inl_join(@`sel_2` `test`.`t3`), merge_join(@`sel_1` `test`.`t1`)" }, { @@ -1538,6 +1550,43 @@ "Result": [ "2" ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "Plan": [ + "HashAgg_11 1.00 root funcs:sum(distinct Column#7)->Column#4", + "└─Projection_23 16000.00 root cast(test.pt.b, decimal(65,0) BINARY)->Column#7", + " └─Union_12 16000.00 root ", + " ├─HashAgg_16 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", + " │ └─TableReader_17 8000.00 root data:HashAgg_13", + " │ └─HashAgg_13 8000.00 cop[tikv] group by:test.pt.b, ", + " │ └─TableFullScan_15 10000.00 cop[tikv] table:pt, partition:p0 keep order:false, stats:pseudo", + " └─HashAgg_21 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", + " └─TableReader_22 8000.00 root data:HashAgg_18", + " └─HashAgg_18 8000.00 cop[tikv] group by:test.pt.b, ", + " └─TableFullScan_20 10000.00 cop[tikv] table:pt, partition:p1 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;", + "Plan": [ + "HashAgg_14 1.00 root funcs:count(distinct Column#5)->Column#6", + "└─Union_15 16000.00 root ", + " ├─HashAgg_19 8000.00 root group by:test.ta.a, funcs:firstrow(test.ta.a)->Column#5", + " │ └─TableReader_20 8000.00 root data:HashAgg_16", + " │ └─HashAgg_16 8000.00 cop[tikv] group by:test.ta.a, ", + " │ └─TableFullScan_18 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", + " └─HashAgg_24 8000.00 root group by:test.tb.a, funcs:firstrow(test.tb.a)->Column#5", + " └─TableReader_25 8000.00 root data:HashAgg_21", + " └─HashAgg_21 8000.00 cop[tikv] group by:test.tb.a, ", + " └─TableFullScan_23 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] } ] }, @@ -1630,6 +1679,77 @@ "1", "1" ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "Plan": [ + "HashAgg_11 1.00 root funcs:sum(distinct Column#7)->Column#4", + "└─Projection_23 16000.00 root cast(test.pt.b, decimal(65,0) BINARY)->Column#7", + " └─Union_12 16000.00 root ", + " ├─HashAgg_16 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", + " │ └─TableReader_17 8000.00 root data:HashAgg_13", + " │ └─HashAgg_13 8000.00 cop[tikv] group by:test.pt.b, ", + " │ └─TableFullScan_15 10000.00 cop[tikv] table:pt, partition:p0 keep order:false, stats:pseudo", + " └─HashAgg_21 8000.00 root group by:test.pt.b, funcs:firstrow(test.pt.b)->test.pt.b", + " └─TableReader_22 8000.00 root data:HashAgg_18", + " └─HashAgg_18 8000.00 cop[tikv] group by:test.pt.b, ", + " └─TableFullScan_20 10000.00 cop[tikv] table:pt, partition:p1 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;", + "Plan": [ + "HashAgg_14 1.00 root funcs:count(distinct Column#5)->Column#6", + "└─Union_15 16000.00 root ", + " ├─HashAgg_19 8000.00 root group by:test.ta.a, funcs:firstrow(test.ta.a)->Column#5", + " │ └─TableReader_20 8000.00 root data:HashAgg_16", + " │ └─HashAgg_16 8000.00 cop[tikv] group by:test.ta.a, ", + " │ └─TableFullScan_18 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", + " └─HashAgg_24 8000.00 root group by:test.tb.a, funcs:firstrow(test.tb.a)->Column#5", + " └─TableReader_25 8000.00 root data:HashAgg_21", + " └─HashAgg_21 8000.00 cop[tikv] group by:test.tb.a, ", + " └─TableFullScan_23 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] + } + ] + }, + { + "Name": "TestPushdownDistinctEnableAggPushDownDisable", + "Cases": [ + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ sum(distinct b) from pt;", + "Plan": [ + "HashAgg_8 1.00 root funcs:sum(distinct Column#5)->Column#4", + "└─Projection_14 20000.00 root cast(test.pt.b, decimal(65,0) BINARY)->Column#5", + " └─Union_9 20000.00 root ", + " ├─TableReader_11 10000.00 root data:TableFullScan_10", + " │ └─TableFullScan_10 10000.00 cop[tikv] table:pt, partition:p0 keep order:false, stats:pseudo", + " └─TableReader_13 10000.00 root data:TableFullScan_12", + " └─TableFullScan_12 10000.00 cop[tikv] table:pt, partition:p1 keep order:false, stats:pseudo" + ], + "Result": [ + "" + ] + }, + { + "SQL": "select /*+ HASH_AGG(), AGG_TO_COP() */ count(distinct a) from (select * from ta union all select * from tb) t;", + "Plan": [ + "HashAgg_11 1.00 root funcs:count(distinct Column#5)->Column#6", + "└─Union_12 20000.00 root ", + " ├─TableReader_15 10000.00 root data:TableFullScan_14", + " │ └─TableFullScan_14 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", + " └─TableReader_18 10000.00 root data:TableFullScan_17", + " └─TableFullScan_17 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo" + ], + "Result": [ + "1" + ] } ] }, @@ -1661,12 +1781,11 @@ { "SQL": "select /*+ TIDB_SMJ(t1, t2) */ t1.a from t t1, t t2 where t1.a = t2.b order by t1.a", "Plan": [ - "Projection_20 12487.50 root test.t.a", - "└─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", - " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", - " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + "MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + "├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", + "│ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + "└─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", + " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" ], "Result": [ "1", @@ -1679,12 +1798,11 @@ "Plan": [ "Projection_24 12487.50 root test.t.a", "└─Projection_23 12487.50 root test.t.a, plus(test.t.a, 1)->Column#7", - " └─Projection_20 12487.50 root test.t.a", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", - " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", - " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", + " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", + " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" ], "Result": [ "1", @@ -1697,12 +1815,11 @@ "Plan": [ "Projection_24 12487.50 root test.t.a", "└─Projection_23 12487.50 root test.t.a, minus(test.t.a, 1)->Column#7", - " └─Projection_20 12487.50 root test.t.a", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", - " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", - " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", + " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", + " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" ], "Result": [ "1", @@ -1715,12 +1832,11 @@ "Plan": [ "Projection_32 12487.50 root test.t.a", "└─Projection_31 12487.50 root test.t.a, unaryminus(test.t.a)->Column#7", - " └─Projection_20 12487.50 root test.t.a", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", - " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", - " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", - " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", + " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", + " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", + " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" ], "Result": [ "2", @@ -1733,12 +1849,11 @@ "Plan": [ "Projection_32 12487.50 root test.t.a", "└─Projection_31 12487.50 root test.t.a, plus(unaryminus(test.t.a), 3)->Column#7", - " └─Projection_20 12487.50 root test.t.a", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", - " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", - " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", - " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", + " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", + " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", + " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" ], "Result": [ "2", @@ -1751,12 +1866,11 @@ "Plan": [ "Projection_24 12487.50 root test.t.a", "└─Projection_23 12487.50 root test.t.a, plus(1, test.t.a)->Column#7", - " └─Projection_20 12487.50 root test.t.a", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", - " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", - " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", + " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", + " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" ], "Result": [ "1", @@ -1769,12 +1883,11 @@ "Plan": [ "Projection_32 12487.50 root test.t.a", "└─Projection_31 12487.50 root test.t.a, minus(1, test.t.a)->Column#7", - " └─Projection_20 12487.50 root test.t.a", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", - " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", - " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", - " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", + " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", + " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", + " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" ], "Result": [ "2", @@ -1787,12 +1900,11 @@ "Plan": [ "Projection_32 12487.50 root test.t.a", "└─Projection_31 12487.50 root test.t.a, plus(minus(1, test.t.a), 3)->Column#7", - " └─Projection_20 12487.50 root test.t.a", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", - " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", - " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", - " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_26(Build) 9990.00 root index:IndexFullScan_25", + " │ └─IndexFullScan_25 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, desc, stats:pseudo", + " └─IndexReader_24(Probe) 9990.00 root index:IndexFullScan_23", + " └─IndexFullScan_23 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, desc, stats:pseudo" ], "Result": [ "2", @@ -1805,12 +1917,11 @@ "Plan": [ "Projection_24 12487.50 root test.t.a", "└─Projection_23 12487.50 root test.t.a, plus(plus(1, test.t.a), 3)->Column#7", - " └─Projection_20 12487.50 root test.t.a", - " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", - " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", - " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + " └─MergeJoin_21 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_15(Build) 9990.00 root index:IndexFullScan_14", + " │ └─IndexFullScan_14 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + " └─IndexReader_13(Probe) 9990.00 root index:IndexFullScan_12", + " └─IndexFullScan_12 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" ], "Result": [ "1", @@ -1824,12 +1935,11 @@ "Projection_19 12487.50 root test.t.a", "└─Sort_7 12487.50 root Column#7:asc", " └─Projection_20 12487.50 root test.t.a, mul(3, test.t.a)->Column#7", - " └─Projection_8 12487.50 root test.t.a", - " └─MergeJoin_9 12487.50 root inner join, left key:test.t.a, right key:test.t.b", - " ├─IndexReader_14(Build) 9990.00 root index:IndexFullScan_13", - " │ └─IndexFullScan_13 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", - " └─IndexReader_12(Probe) 9990.00 root index:IndexFullScan_11", - " └─IndexFullScan_11 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" + " └─MergeJoin_9 12487.50 root inner join, left key:test.t.a, right key:test.t.b", + " ├─IndexReader_14(Build) 9990.00 root index:IndexFullScan_13", + " │ └─IndexFullScan_13 9990.00 cop[tikv] table:t2, index:idx_b(b) keep order:true, stats:pseudo", + " └─IndexReader_12(Probe) 9990.00 root index:IndexFullScan_11", + " └─IndexFullScan_11 9990.00 cop[tikv] table:t1, index:idx_a(a) keep order:true, stats:pseudo" ], "Result": [ "1", @@ -1886,6 +1996,11 @@ "SQL": "select /*+ INL_MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", "Plan": "IndexMergeJoin{IndexLookUp(Index(t1.idx_a)[[NULL,+inf]]->Sel([not(isnull(test.t1.a))]), Table(t1))->Projection->TableReader(Table(t2)->Sel([not(isnull(test.t2.a))]))}(test.t2.a,test.t1.a)", "Hints": "use_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` )" + }, + { + "SQL": "select /*+ MERGE_JOIN(t1) */ t1.b, t2.b from t1 inner join t2 on t1.a = t2.a;", + "Plan": "MergeInnerJoin{IndexLookUp(Index(t1.idx_a)[[-inf,+inf]], Table(t1))->Projection->IndexLookUp(Index(t2.idx_a)[[-inf,+inf]], Table(t2))->Projection}(test.t1.a,test.t2.a)", + "Hints": "use_index(@`sel_1` `test`.`t1` `idx_a`), use_index(@`sel_1` `test`.`t2` `idx_a`)" } ] }, diff --git a/planner/core/testdata/plan_suite_unexported_in.json b/planner/core/testdata/plan_suite_unexported_in.json index d92bac11542ba..8b2fe43e88f01 100644 --- a/planner/core/testdata/plan_suite_unexported_in.json +++ b/planner/core/testdata/plan_suite_unexported_in.json @@ -21,7 +21,10 @@ "select max(a.b), max(b.b) from t a join t b on a.a = b.a group by a.c", "select max(c.b) from (select * from t a union all select * from t b) c group by c.a", "select max(a.c) from t a join t b on a.a=b.a and a.b=b.b group by a.b", - "select t1.a, count(t2.b) from t t1, t t2 where t1.a = t2.a group by t1.a" + "select t1.a, count(t2.b) from t t1, t t2 where t1.a = t2.a group by t1.a", + "select avg(a) from (select * from t t1 union all select * from t t2) t", + "select count(distinct a) from (select * from t t1 union all select * from t t2) t", + "select count(distinct b) from (select * from t t1 union all select * from t t2) t" ] }, { diff --git a/planner/core/testdata/plan_suite_unexported_out.json b/planner/core/testdata/plan_suite_unexported_out.json index 6afb5519c320f..5115811b4f84c 100644 --- a/planner/core/testdata/plan_suite_unexported_out.json +++ b/planner/core/testdata/plan_suite_unexported_out.json @@ -16,12 +16,15 @@ "Join{DataScan(a)->Aggr(sum(test.t.a),firstrow(test.t.c))->DataScan(b)}(test.t.c,test.t.c)->Aggr(sum(Column#26))->Projection", "Join{DataScan(a)->Aggr(sum(test.t.a),firstrow(test.t.c))->DataScan(b)}(test.t.c,test.t.c)->Aggr(sum(Column#26))->Projection", "DataScan(t)->Aggr(sum(test.t.a))->Projection", - "UnionAll{DataScan(a)->Projection->Aggr(sum(test.t.c),firstrow(test.t.d))->DataScan(b)->Projection->Aggr(sum(test.t.a),firstrow(test.t.b))->DataScan(c)->Projection->Aggr(sum(test.t.b),firstrow(test.t.e))}->Aggr(sum(Column#40))->Projection", + "UnionAll{DataScan(a)->Projection->Aggr(sum(test.t.c))->DataScan(b)->Projection->Aggr(sum(test.t.a))->DataScan(c)->Projection->Aggr(sum(test.t.b))}->Aggr(sum(Column#40))->Projection", "Join{DataScan(a)->DataScan(b)->Aggr(max(test.t.b),firstrow(test.t.c))}(test.t.c,test.t.c)->Projection->Projection", "Join{DataScan(a)->DataScan(b)}(test.t.a,test.t.a)->Aggr(max(test.t.b),max(test.t.b))->Projection", "UnionAll{DataScan(a)->Projection->Projection->Projection->DataScan(b)->Projection->Projection->Projection}->Aggr(max(Column#38))->Projection", "Join{DataScan(a)->DataScan(b)}(test.t.a,test.t.a)(test.t.b,test.t.b)->Aggr(max(test.t.c))->Projection", - "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->Projection->Projection" + "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->Projection->Projection", + "UnionAll{DataScan(t1)->Projection->Aggr(count(test.t.a),sum(test.t.a))->DataScan(t2)->Projection->Aggr(count(test.t.a),sum(test.t.a))}->Aggr(avg(Column#38, Column#39))->Projection", + "UnionAll{DataScan(t1)->Projection->Projection->Projection->DataScan(t2)->Projection->Projection->Projection}->Aggr(count(distinct Column#25))->Projection", + "UnionAll{DataScan(t1)->Projection->Aggr(firstrow(test.t.b))->DataScan(t2)->Projection->Aggr(firstrow(test.t.b))}->Aggr(count(distinct Column#26))->Projection" ] }, { @@ -46,7 +49,7 @@ "Join{DataScan(t1)->DataScan(t2)}->Projection", "Join{DataScan(t1)->DataScan(t2)}->Projection", "LeftHashJoin{LeftHashJoin{TableReader(Table(t))->IndexLookUp(Index(t.c_d_e)[[666,666]], Table(t))}(test.t.a,test.t.b)->IndexReader(Index(t.c_d_e)[[42,42]])}(test.t.b,test.t.a)->Sel([or(Column#25, Column#38)])->Projection->Delete", - "LeftHashJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])->HashAgg}(test.t.b,test.t.c)->Projection->Update" + "LeftHashJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])->HashAgg}(test.t.b,test.t.c)->Update" ] }, { @@ -897,8 +900,8 @@ "DataScan(t1)->Aggr(firstrow(test.t.a),firstrow(test.t.b))", "DataScan(t2)->Aggr(firstrow(test.t.a),firstrow(test.t.b))", "DataScan(t1)->Aggr(max(test.t.a),min(test.t.b))->Projection", - "DataScan(t1)->Aggr(sum(test.t.a))->Projection", - "DataScan(t1)->Aggr(count(test.t.a, test.t.b))->Projection", + "DataScan(t1)->Aggr(sum(distinct test.t.a))->Projection", + "DataScan(t1)->Aggr(count(distinct test.t.a, test.t.b))->Projection", "DataScan(t1)->Projection", "DataScan(t2)->Projection", "Join{Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->DataScan(t3)->TopN([test.t.b true],0,1)}(test.t.b,test.t.b)->TopN([test.t.b true],0,1)->Aggr(max(test.t.b))->Projection", diff --git a/planner/implementation/join.go b/planner/implementation/join.go index 3b801ccae3704..35abf19e8086c 100644 --- a/planner/implementation/join.go +++ b/planner/implementation/join.go @@ -64,7 +64,6 @@ func (impl *MergeJoinImpl) CalcCost(outCount float64, children ...memo.Implement func (impl *MergeJoinImpl) AttachChildren(children ...memo.Implementation) memo.Implementation { mergeJoin := impl.plan.(*plannercore.PhysicalMergeJoin) mergeJoin.SetChildren(children[0].GetPlan(), children[1].GetPlan()) - mergeJoin.SetSchema(plannercore.BuildPhysicalJoinSchema(mergeJoin.JoinType, mergeJoin)) return impl } diff --git a/planner/memo/expr_iterator_test.go b/planner/memo/expr_iterator_test.go index a9ede5d43155a..5825f8f17d5ee 100644 --- a/planner/memo/expr_iterator_test.go +++ b/planner/memo/expr_iterator_test.go @@ -60,26 +60,26 @@ func (s *testMemoSuite) TestNewExprIterFromGroupElem(c *C) { func (s *testMemoSuite) TestExprIterNext(c *C) { g0 := NewGroupWithSchema(NewGroupExpr( - plannercore.LogicalProjection{Exprs: []expression.Expression{expression.Zero}}.Init(s.sctx, 0)), s.schema) + plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewZero()}}.Init(s.sctx, 0)), s.schema) g0.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 1}.Init(s.sctx, 0))) g0.Insert(NewGroupExpr( - plannercore.LogicalProjection{Exprs: []expression.Expression{expression.One}}.Init(s.sctx, 0))) + plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewOne()}}.Init(s.sctx, 0))) g0.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 2}.Init(s.sctx, 0))) g0.Insert(NewGroupExpr( - plannercore.LogicalProjection{Exprs: []expression.Expression{expression.Null}}.Init(s.sctx, 0))) + plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewNull()}}.Init(s.sctx, 0))) g1 := NewGroupWithSchema(NewGroupExpr( - plannercore.LogicalSelection{Conditions: []expression.Expression{expression.Null}}.Init(s.sctx, 0)), s.schema) + plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewNull()}}.Init(s.sctx, 0)), s.schema) g1.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 3}.Init(s.sctx, 0))) g1.Insert(NewGroupExpr( - plannercore.LogicalSelection{Conditions: []expression.Expression{expression.One}}.Init(s.sctx, 0))) + plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(s.sctx, 0))) g1.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 4}.Init(s.sctx, 0))) g1.Insert(NewGroupExpr( - plannercore.LogicalSelection{Conditions: []expression.Expression{expression.Zero}}.Init(s.sctx, 0))) + plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewZero()}}.Init(s.sctx, 0))) expr := NewGroupExpr(plannercore.LogicalJoin{}.Init(s.sctx, 0)) expr.Children = append(expr.Children, g0) @@ -114,19 +114,19 @@ func (s *testMemoSuite) TestExprIterNext(c *C) { func (s *testMemoSuite) TestExprIterReset(c *C) { g0 := NewGroupWithSchema(NewGroupExpr( - plannercore.LogicalProjection{Exprs: []expression.Expression{expression.Zero}}.Init(s.sctx, 0)), s.schema) + plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewZero()}}.Init(s.sctx, 0)), s.schema) g0.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 1}.Init(s.sctx, 0))) g0.Insert(NewGroupExpr( - plannercore.LogicalProjection{Exprs: []expression.Expression{expression.One}}.Init(s.sctx, 0))) + plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewOne()}}.Init(s.sctx, 0))) g0.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 2}.Init(s.sctx, 0))) g0.Insert(NewGroupExpr( - plannercore.LogicalProjection{Exprs: []expression.Expression{expression.Null}}.Init(s.sctx, 0))) + plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewNull()}}.Init(s.sctx, 0))) - sel1 := NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.Null}}.Init(s.sctx, 0)) - sel2 := NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.One}}.Init(s.sctx, 0)) - sel3 := NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.Zero}}.Init(s.sctx, 0)) + sel1 := NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewNull()}}.Init(s.sctx, 0)) + sel2 := NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(s.sctx, 0)) + sel3 := NewGroupExpr(plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewZero()}}.Init(s.sctx, 0)) g1 := NewGroupWithSchema(sel1, s.schema) g1.Insert(NewGroupExpr(plannercore.LogicalLimit{Count: 3}.Init(s.sctx, 0))) g1.Insert(sel2) @@ -134,15 +134,15 @@ func (s *testMemoSuite) TestExprIterReset(c *C) { g1.Insert(sel3) g2 := NewGroupWithSchema(NewGroupExpr( - plannercore.LogicalSelection{Conditions: []expression.Expression{expression.Null}}.Init(s.sctx, 0)), s.schema) + plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewNull()}}.Init(s.sctx, 0)), s.schema) g2.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 3}.Init(s.sctx, 0))) g2.Insert(NewGroupExpr( - plannercore.LogicalSelection{Conditions: []expression.Expression{expression.One}}.Init(s.sctx, 0))) + plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(s.sctx, 0))) g2.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 4}.Init(s.sctx, 0))) g2.Insert(NewGroupExpr( - plannercore.LogicalSelection{Conditions: []expression.Expression{expression.Zero}}.Init(s.sctx, 0))) + plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewZero()}}.Init(s.sctx, 0))) // link join with Group 0 and 1 expr := NewGroupExpr(plannercore.LogicalJoin{}.Init(s.sctx, 0)) @@ -207,20 +207,20 @@ func countMatchedIter(group *Group, pattern *Pattern) int { func (s *testMemoSuite) TestExprIterWithEngineType(c *C) { g1 := NewGroupWithSchema(NewGroupExpr( - plannercore.LogicalSelection{Conditions: []expression.Expression{expression.One}}.Init(s.sctx, 0)), s.schema).SetEngineType(EngineTiFlash) + plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(s.sctx, 0)), s.schema).SetEngineType(EngineTiFlash) g1.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 1}.Init(s.sctx, 0))) g1.Insert(NewGroupExpr( - plannercore.LogicalProjection{Exprs: []expression.Expression{expression.One}}.Init(s.sctx, 0))) + plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewOne()}}.Init(s.sctx, 0))) g1.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 2}.Init(s.sctx, 0))) g2 := NewGroupWithSchema(NewGroupExpr( - plannercore.LogicalSelection{Conditions: []expression.Expression{expression.One}}.Init(s.sctx, 0)), s.schema).SetEngineType(EngineTiKV) + plannercore.LogicalSelection{Conditions: []expression.Expression{expression.NewOne()}}.Init(s.sctx, 0)), s.schema).SetEngineType(EngineTiKV) g2.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 2}.Init(s.sctx, 0))) g2.Insert(NewGroupExpr( - plannercore.LogicalProjection{Exprs: []expression.Expression{expression.One}}.Init(s.sctx, 0))) + plannercore.LogicalProjection{Exprs: []expression.Expression{expression.NewOne()}}.Init(s.sctx, 0))) g2.Insert(NewGroupExpr( plannercore.LogicalLimit{Count: 3}.Init(s.sctx, 0))) diff --git a/privilege/privilege.go b/privilege/privilege.go index bdcea116bed27..c52ea96da41fe 100644 --- a/privilege/privilege.go +++ b/privilege/privilege.go @@ -15,6 +15,7 @@ package privilege import ( "crypto/tls" + "github.com/pingcap/parser/auth" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/sessionctx" diff --git a/privilege/privileges/cache.go b/privilege/privileges/cache.go index 0d3af938f9e4a..de35b8a482123 100644 --- a/privilege/privileges/cache.go +++ b/privilege/privileges/cache.go @@ -983,13 +983,13 @@ func (p *MySQLPrivilege) DBIsVisible(user, host, db string) bool { func (p *MySQLPrivilege) showGrants(user, host string, roles []*auth.RoleIdentity) []string { var gs []string - var hasGlobalGrant bool = false + var hasGlobalGrant = false // Some privileges may granted from role inheritance. // We should find these inheritance relationship. allRoles := p.FindAllRole(roles) // Show global grants. var currentPriv mysql.PrivilegeType - var hasGrantOptionPriv bool = false + var hasGrantOptionPriv = false var g string for _, record := range p.User { if record.baseRecord.match(user, host) { diff --git a/privilege/privileges/cache_test.go b/privilege/privileges/cache_test.go index bbc8a1462448b..ce0e68f2b3976 100644 --- a/privilege/privileges/cache_test.go +++ b/privilege/privileges/cache_test.go @@ -15,6 +15,7 @@ package privileges_test import ( "fmt" + . "github.com/pingcap/check" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/mysql" @@ -384,6 +385,7 @@ func (s *testCacheSuite) TestAbnormalMySQLTable(c *C) { Shutdown_priv enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N', Process_priv enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N', File_priv enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N', + Config_priv enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N', Grant_priv enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N', References_priv enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N', Index_priv enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N', @@ -419,7 +421,7 @@ func (s *testCacheSuite) TestAbnormalMySQLTable(c *C) { password_expired enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N', PRIMARY KEY (Host,User) ) ENGINE=MyISAM DEFAULT CHARSET=utf8 COLLATE=utf8_bin COMMENT='Users and global privileges';`) - mustExec(c, se, `INSERT INTO user VALUES ('localhost','root','','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','','','','',0,0,0,0,'mysql_native_password','','N'); + mustExec(c, se, `INSERT INTO user VALUES ('localhost','root','','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','Y','','','','',0,0,0,0,'mysql_native_password','','N'); `) var p privileges.MySQLPrivilege err = p.LoadUserTable(se) diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index c817d8a195c55..0117db286141d 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -20,6 +20,7 @@ import ( "crypto/x509" "crypto/x509/pkix" "fmt" + "os" "strings" "testing" @@ -826,6 +827,24 @@ func (s *testPrivilegeSuite) TestCreateDropUser(c *C) { mustExec(c, se, `DROP USER tcd3`) } +func (s *testPrivilegeSuite) TestConfigPrivilege(c *C) { + se := newSession(c, s.store, s.dbName) + mustExec(c, se, `DROP USER IF EXISTS tcd1`) + mustExec(c, se, `CREATE USER tcd1`) + mustExec(c, se, `GRANT ALL ON *.* to tcd1`) + mustExec(c, se, `DROP USER IF EXISTS tcd2`) + mustExec(c, se, `CREATE USER tcd2`) + mustExec(c, se, `GRANT ALL ON *.* to tcd2`) + mustExec(c, se, `REVOKE CONFIG ON *.* FROM tcd2`) + + c.Assert(se.Auth(&auth.UserIdentity{Username: "tcd1", Hostname: "localhost", AuthHostname: "tcd1", AuthUsername: "%"}, nil, nil), IsTrue) + mustExec(c, se, `SET CONFIG TIKV testkey="testval"`) + c.Assert(se.Auth(&auth.UserIdentity{Username: "tcd2", Hostname: "localhost", AuthHostname: "tcd2", AuthUsername: "%"}, nil, nil), IsTrue) + _, err := se.Execute(context.Background(), `SET CONFIG TIKV testkey="testval"`) + c.Assert(err, ErrorMatches, ".*you need \\(at least one of\\) the CONFIG privilege\\(s\\) for this operation") + mustExec(c, se, `DROP USER tcd1, tcd2`) +} + func (s *testPrivilegeSuite) TestShowCreateTable(c *C) { se := newSession(c, s.store, s.dbName) mustExec(c, se, `CREATE USER tsct1, tsct2`) @@ -897,7 +916,7 @@ func (s *testPrivilegeSuite) TestSystemSchema(c *C) { mustExec(c, se, `select * from performance_schema.events_statements_summary_by_digest`) _, err = se.Execute(context.Background(), "drop table performance_schema.events_statements_summary_by_digest") c.Assert(strings.Contains(err.Error(), "denied to user"), IsTrue) - _, err = se.Execute(context.Background(), "update performance_schema.events_statements_summary_by_digest set table_names = 'tst'") + _, err = se.Execute(context.Background(), "update performance_schema.events_statements_summary_by_digest set schema_name = 'tst'") c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) _, err = se.Execute(context.Background(), "delete from performance_schema.events_statements_summary_by_digest") c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) @@ -929,6 +948,35 @@ func (s *testPrivilegeSuite) TestAdminCommand(c *C) { c.Assert(err, IsNil) } +func (s *testPrivilegeSuite) TestLoadDataPrivilege(c *C) { + // Create file. + path := "/tmp/load_data_priv.csv" + fp, err := os.Create(path) + c.Assert(err, IsNil) + c.Assert(fp, NotNil) + defer func() { + err = fp.Close() + c.Assert(err, IsNil) + err = os.Remove(path) + c.Assert(err, IsNil) + }() + fp.WriteString("1\n") + + se := newSession(c, s.store, s.dbName) + c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil), IsTrue) + mustExec(c, se, `CREATE USER 'test_load'@'localhost';`) + mustExec(c, se, `CREATE TABLE t_load(a int)`) + mustExec(c, se, `GRANT SELECT on *.* to 'test_load'@'localhost'`) + c.Assert(se.Auth(&auth.UserIdentity{Username: "test_load", Hostname: "localhost"}, nil, nil), IsTrue) + _, err = se.Execute(context.Background(), "LOAD DATA LOCAL INFILE '/tmp/load_data_priv.csv' INTO TABLE t_load") + c.Assert(strings.Contains(err.Error(), "INSERT command denied to user 'test_load'@'localhost' for table 't_load'"), IsTrue) + c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil), IsTrue) + mustExec(c, se, `GRANT INSERT on *.* to 'test_load'@'localhost'`) + c.Assert(se.Auth(&auth.UserIdentity{Username: "test_load", Hostname: "localhost"}, nil, nil), IsTrue) + _, err = se.Execute(context.Background(), "LOAD DATA LOCAL INFILE '/tmp/load_data_priv.csv' INTO TABLE t_load") + c.Assert(err, IsNil) +} + func (s *testPrivilegeSuite) TestGetEncodedPassword(c *C) { se := newSession(c, s.store, s.dbName) mustExec(c, se, `CREATE USER 'test_encode_u'@'localhost' identified by 'root';`) diff --git a/server/conn.go b/server/conn.go index 2ceccf4cb3afc..a8d3b3945f0a4 100644 --- a/server/conn.go +++ b/server/conn.go @@ -54,6 +54,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser" + "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -176,9 +177,9 @@ func (cc *clientConn) String() string { func (cc *clientConn) handshake(ctx context.Context) error { if err := cc.writeInitialHandshake(); err != nil { if errors.Cause(err) == io.EOF { - logutil.Logger(ctx).Info("Could not send handshake due to connection has be closed by client-side") + logutil.Logger(ctx).Debug("Could not send handshake due to connection has be closed by client-side") } else { - terror.Log(err) + logutil.Logger(ctx).Debug("Write init handshake to client fail", zap.Error(errors.SuspendStack(err))) } return err } @@ -493,9 +494,9 @@ func (cc *clientConn) readOptionalSSLRequestAndHandshakeResponse(ctx context.Con if err != nil { err = errors.SuspendStack(err) if errors.Cause(err) == io.EOF { - logutil.Logger(ctx).Info("wait handshake response fail due to connection has be closed by client-side") + logutil.Logger(ctx).Debug("wait handshake response fail due to connection has be closed by client-side") } else { - logutil.Logger(ctx).Error("wait handshake response fail", zap.Error(err)) + logutil.Logger(ctx).Debug("wait handshake response fail", zap.Error(err)) } return err } @@ -570,7 +571,9 @@ func (cc *clientConn) readOptionalSSLRequestAndHandshakeResponse(ctx context.Con cc.attrs = resp.Attrs err = cc.openSessionAndDoAuth(resp.Auth) - logutil.Logger(ctx).Warn("open new session failure", zap.Error(err)) + if err != nil { + logutil.Logger(ctx).Warn("open new session failure", zap.Error(err)) + } return err } @@ -724,7 +727,7 @@ func (cc *clientConn) Run(ctx context.Context) { if cc.ctx != nil { txnMode = cc.ctx.GetSessionVars().GetReadableTxnMode() } - logutil.Logger(ctx).Warn("command dispatched failed", + logutil.Logger(ctx).Error("command dispatched failed", zap.String("connInfo", cc.String()), zap.String("command", mysql.Command2Str[data[0]]), zap.String("status", cc.SessionStatusToString()), @@ -929,7 +932,11 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { func (cc *clientConn) useDB(ctx context.Context, db string) (err error) { // if input is "use `SELECT`", mysql client just send "SELECT" // so we add `` around db. - _, err = cc.ctx.Execute(ctx, "use `"+db+"`") + stmts, err := cc.ctx.Parse(ctx, "use `"+db+"`") + if err != nil { + return err + } + _, err = cc.ctx.ExecuteStmt(ctx, stmts[0]) if err != nil { return err } @@ -1165,9 +1172,16 @@ func (cc *clientConn) handleLoadData(ctx context.Context, loadDataInfo *executor err = loadDataInfo.CommitWork(ctx) wg.Wait() if err != nil { + if !loadDataInfo.Drained { + logutil.Logger(ctx).Info("not drained yet, try reading left data from client connection") + } // drain the data from client conn util empty packet received, otherwise the connection will be reset for !loadDataInfo.Drained { - logutil.Logger(ctx).Info("not drained yet, try reading left data from client connection") + // check kill flag again, let the draining loop could quit if empty packet could not be received + if atomic.CompareAndSwapUint32(&loadDataInfo.Ctx.GetSessionVars().Killed, 1, 0) { + logutil.Logger(ctx).Warn("receiving kill, stop draining data, connection may be reset") + return executor.ErrQueryInterrupted + } curData, err1 := cc.readPacket() if err1 != nil { logutil.Logger(ctx).Error("drain reading left data encounter errors", zap.Error(err1)) @@ -1254,53 +1268,86 @@ func (cc *clientConn) handleIndexAdvise(ctx context.Context, indexAdviseInfo *ex // There is a special query `load data` that does not return result, which is handled differently. // Query `load stats` does not return result either. func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { - rss, err := cc.ctx.Execute(ctx, sql) + stmts, err := cc.ctx.Parse(ctx, sql) if err != nil { metrics.ExecuteErrorCounter.WithLabelValues(metrics.ExecuteErrorToLabel(err)).Inc() return err } - status := atomic.LoadInt32(&cc.status) - if rss != nil && (status == connStatusShutdown || status == connStatusWaitShutdown) { - for _, rs := range rss { - terror.Call(rs.Close) + + if len(stmts) == 0 { + return cc.writeOK() + } + + for i, stmt := range stmts { + if err = cc.handleStmt(ctx, stmt, i == len(stmts)-1); err != nil { + break } - return executor.ErrQueryInterrupted } - if rss != nil { - if len(rss) == 1 { - err = cc.writeResultset(ctx, rss[0], false, 0, 0) - } else { - err = cc.writeMultiResultset(ctx, rss, false) + + if err != nil { + metrics.ExecuteErrorCounter.WithLabelValues(metrics.ExecuteErrorToLabel(err)).Inc() + } + return err +} + +func (cc *clientConn) handleStmt(ctx context.Context, stmt ast.StmtNode, lastStmt bool) error { + rs, err := cc.ctx.ExecuteStmt(ctx, stmt) + if rs != nil { + defer terror.Call(rs.Close) + } + if err != nil { + return err + } + + status := cc.ctx.Status() + if !lastStmt { + status |= mysql.ServerMoreResultsExists + } + + if rs != nil { + connStatus := atomic.LoadInt32(&cc.status) + if connStatus == connStatusShutdown || connStatus == connStatusWaitShutdown { + return executor.ErrQueryInterrupted + } + + err = cc.writeResultset(ctx, rs, false, status, 0) + if err != nil { + return err } } else { - loadDataInfo := cc.ctx.Value(executor.LoadDataVarKey) - if loadDataInfo != nil { - defer cc.ctx.SetValue(executor.LoadDataVarKey, nil) - if err = cc.handleLoadData(ctx, loadDataInfo.(*executor.LoadDataInfo)); err != nil { - return err - } + err = cc.handleQuerySpecial(ctx, status) + if err != nil { + return err } + } + return nil +} - loadStats := cc.ctx.Value(executor.LoadStatsVarKey) - if loadStats != nil { - defer cc.ctx.SetValue(executor.LoadStatsVarKey, nil) - if err = cc.handleLoadStats(ctx, loadStats.(*executor.LoadStatsInfo)); err != nil { - return err - } +func (cc *clientConn) handleQuerySpecial(ctx context.Context, status uint16) error { + loadDataInfo := cc.ctx.Value(executor.LoadDataVarKey) + if loadDataInfo != nil { + defer cc.ctx.SetValue(executor.LoadDataVarKey, nil) + if err := cc.handleLoadData(ctx, loadDataInfo.(*executor.LoadDataInfo)); err != nil { + return err } + } - indexAdvise := cc.ctx.Value(executor.IndexAdviseVarKey) - if indexAdvise != nil { - defer cc.ctx.SetValue(executor.IndexAdviseVarKey, nil) - err = cc.handleIndexAdvise(ctx, indexAdvise.(*executor.IndexAdviseInfo)) - if err != nil { - return err - } + loadStats := cc.ctx.Value(executor.LoadStatsVarKey) + if loadStats != nil { + defer cc.ctx.SetValue(executor.LoadStatsVarKey, nil) + if err := cc.handleLoadStats(ctx, loadStats.(*executor.LoadStatsInfo)); err != nil { + return err } + } - err = cc.writeOK() + indexAdvise := cc.ctx.Value(executor.IndexAdviseVarKey) + if indexAdvise != nil { + defer cc.ctx.SetValue(executor.IndexAdviseVarKey, nil) + if err := cc.handleIndexAdvise(ctx, indexAdvise.(*executor.IndexAdviseInfo)); err != nil { + return err + } } - return err + return cc.writeOkWith(cc.ctx.LastMessage(), cc.ctx.AffectedRows(), cc.ctx.LastInsertID(), status, cc.ctx.WarningCount()) } // handleFieldList returns the field list for a table. @@ -1335,13 +1382,9 @@ func (cc *clientConn) handleFieldList(sql string) (err error) { // If binary is true, the data would be encoded in BINARY format. // serverStatus, a flag bit represents server information. // fetchSize, the desired number of rows to be fetched each time when client uses cursor. -// resultsets, it's used to support the MULTI_RESULTS capability in mysql protocol. func (cc *clientConn) writeResultset(ctx context.Context, rs ResultSet, binary bool, serverStatus uint16, fetchSize int) (runErr error) { defer func() { // close ResultSet when cursor doesn't exist - if !mysql.HasCursorExistsFlag(serverStatus) { - terror.Call(rs.Close) - } r := recover() if r == nil { return diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 0786b7c990fa0..e1aea78688b7e 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -204,6 +205,7 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e // explicitly flush columnInfo to client. return cc.flush() } + defer terror.Call(rs.Close) err = cc.writeResultset(ctx, rs, true, 0, 0) if err != nil { return errors.Annotate(err, cc.preparedStmt2String(stmtID)) @@ -310,7 +312,7 @@ func parseExecArgs(sc *stmtctx.StatementContext, args []types.Datum, boundParams } if isUnsigned { - args[i] = types.NewUintDatum(uint64(uint8(paramValues[pos]))) + args[i] = types.NewUintDatum(uint64(paramValues[pos])) } else { args[i] = types.NewIntDatum(int64(int8(paramValues[pos]))) } @@ -325,7 +327,7 @@ func parseExecArgs(sc *stmtctx.StatementContext, args []types.Datum, boundParams } valU16 := binary.LittleEndian.Uint16(paramValues[pos : pos+2]) if isUnsigned { - args[i] = types.NewUintDatum(uint64(uint16(valU16))) + args[i] = types.NewUintDatum(uint64(valU16)) } else { args[i] = types.NewIntDatum(int64(int16(valU16))) } @@ -339,7 +341,7 @@ func parseExecArgs(sc *stmtctx.StatementContext, args []types.Datum, boundParams } valU32 := binary.LittleEndian.Uint32(paramValues[pos : pos+4]) if isUnsigned { - args[i] = types.NewUintDatum(uint64(uint32(valU32))) + args[i] = types.NewUintDatum(uint64(valU32)) } else { args[i] = types.NewIntDatum(int64(int32(valU32))) } @@ -387,7 +389,7 @@ func parseExecArgs(sc *stmtctx.StatementContext, args []types.Datum, boundParams } // See https://dev.mysql.com/doc/internals/en/binary-protocol-value.html // for more details. - length := uint8(paramValues[pos]) + length := paramValues[pos] pos++ switch length { case 0: @@ -412,13 +414,13 @@ func parseExecArgs(sc *stmtctx.StatementContext, args []types.Datum, boundParams } // See https://dev.mysql.com/doc/internals/en/binary-protocol-value.html // for more details. - length := uint8(paramValues[pos]) + length := paramValues[pos] pos++ switch length { case 0: tmp = "0" case 8: - isNegative := uint8(paramValues[pos]) + isNegative := paramValues[pos] if isNegative > 1 { err = mysql.ErrMalformPacket return @@ -426,7 +428,7 @@ func parseExecArgs(sc *stmtctx.StatementContext, args []types.Datum, boundParams pos++ pos, tmp = parseBinaryDuration(pos, paramValues, isNegative) case 12: - isNegative := uint8(paramValues[pos]) + isNegative := paramValues[pos] if isNegative > 1 { err = mysql.ErrMalformPacket return @@ -510,20 +512,20 @@ func parseExecArgs(sc *stmtctx.StatementContext, args []types.Datum, boundParams func parseBinaryDate(pos int, paramValues []byte) (int, string) { year := binary.LittleEndian.Uint16(paramValues[pos : pos+2]) pos += 2 - month := uint8(paramValues[pos]) + month := paramValues[pos] pos++ - day := uint8(paramValues[pos]) + day := paramValues[pos] pos++ return pos, fmt.Sprintf("%04d-%02d-%02d", year, month, day) } func parseBinaryDateTime(pos int, paramValues []byte) (int, string) { pos, date := parseBinaryDate(pos, paramValues) - hour := uint8(paramValues[pos]) + hour := paramValues[pos] pos++ - minute := uint8(paramValues[pos]) + minute := paramValues[pos] pos++ - second := uint8(paramValues[pos]) + second := paramValues[pos] pos++ return pos, fmt.Sprintf("%s %02d:%02d:%02d", date, hour, minute, second) } @@ -542,11 +544,11 @@ func parseBinaryDuration(pos int, paramValues []byte, isNegative uint8) (int, st } days := binary.LittleEndian.Uint32(paramValues[pos : pos+4]) pos += 4 - hours := uint8(paramValues[pos]) + hours := paramValues[pos] pos++ - minutes := uint8(paramValues[pos]) + minutes := paramValues[pos] pos++ - seconds := uint8(paramValues[pos]) + seconds := paramValues[pos] pos++ return pos, fmt.Sprintf("%s%d %02d:%02d:%02d", sign, days, hours, minutes, seconds) } diff --git a/server/conn_stmt_test.go b/server/conn_stmt_test.go index 455831d83966a..038e05cb348eb 100644 --- a/server/conn_stmt_test.go +++ b/server/conn_stmt_test.go @@ -216,7 +216,7 @@ func (ts *ConnTestSuite) TestParseStmtFetchCmd(c *C) { } for _, t := range tests { - stmtID, fetchSize, err := parseStmtFetchCmd([]byte(t.arg)) + stmtID, fetchSize, err := parseStmtFetchCmd(t.arg) c.Assert(stmtID, Equals, t.stmtID) c.Assert(fetchSize, Equals, t.fetchSize) c.Assert(err, Equals, t.err) diff --git a/server/conn_test.go b/server/conn_test.go index 4997020c7b21f..48d208669f99e 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -23,7 +23,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" @@ -33,7 +32,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/arena" - "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/testleak" ) @@ -440,41 +438,27 @@ func (ts *ConnTestSuite) TestConnExecutionTimeout(c *C) { type mockTiDBCtx struct { TiDBContext - rs []ResultSet err error } -func (c *mockTiDBCtx) Execute(ctx context.Context, sql string) ([]ResultSet, error) { - return c.rs, c.err -} - -func (c *mockTiDBCtx) ExecuteInternal(ctx context.Context, sql string) ([]ResultSet, error) { - return c.rs, c.err -} - func (c *mockTiDBCtx) GetSessionVars() *variable.SessionVars { return &variable.SessionVars{} } -type mockRecordSet struct{} - -func (m mockRecordSet) Fields() []*ast.ResultField { return nil } -func (m mockRecordSet) Next(ctx context.Context, req *chunk.Chunk) error { return nil } -func (m mockRecordSet) NewChunk() *chunk.Chunk { return nil } -func (m mockRecordSet) Close() error { return nil } - func (ts *ConnTestSuite) TestShutDown(c *C) { cc := &clientConn{} - - rs := &tidbResultSet{recordSet: mockRecordSet{}} + se, err := session.CreateSession4Test(ts.store) + c.Assert(err, IsNil) // mock delay response - cc.ctx = &mockTiDBCtx{rs: []ResultSet{rs}, err: nil} + cc.ctx = &mockTiDBCtx{ + TiDBContext: TiDBContext{session: se}, + err: nil, + } // set killed flag cc.status = connStatusShutdown // assert ErrQueryInterrupted - err := cc.handleQuery(context.Background(), "dummy") + err = cc.handleQuery(context.Background(), "select 1") c.Assert(err, Equals, executor.ErrQueryInterrupted) - c.Assert(rs.closed, Equals, int32(1)) } func (ts *ConnTestSuite) TestShutdownOrNotify(c *C) { diff --git a/server/driver.go b/server/driver.go index f336db8f1356b..389f9f2700173 100644 --- a/server/driver.go +++ b/server/driver.go @@ -19,6 +19,7 @@ import ( "fmt" "time" + "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" @@ -66,11 +67,11 @@ type QueryCtx interface { // CurrentDB returns current DB. CurrentDB() string - // Execute executes a SQL statement. - Execute(ctx context.Context, sql string) ([]ResultSet, error) + // ExecuteStmt executes a SQL statement. + ExecuteStmt(context.Context, ast.StmtNode) (ResultSet, error) - // ExecuteInternal executes a internal SQL statement. - ExecuteInternal(ctx context.Context, sql string) ([]ResultSet, error) + // Parse parses a SQL to statement node. + Parse(ctx context.Context, sql string) ([]ast.StmtNode, error) // SetClientCapability sets client capability flags SetClientCapability(uint32) diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 8413b850dbb26..9857217761ccf 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -243,40 +243,23 @@ func (tc *TiDBContext) WarningCount() uint16 { return tc.session.GetSessionVars().StmtCtx.WarningCount() } -// Execute implements QueryCtx Execute method. -func (tc *TiDBContext) Execute(ctx context.Context, sql string) (rs []ResultSet, err error) { - rsList, err := tc.session.Execute(ctx, sql) +// ExecuteStmt implements QueryCtx interface. +func (tc *TiDBContext) ExecuteStmt(ctx context.Context, stmt ast.StmtNode) (ResultSet, error) { + rs, err := tc.session.ExecuteStmt(ctx, stmt) if err != nil { - return + return nil, err } - if len(rsList) == 0 { // result ok - return + if rs == nil { + return nil, nil } - rs = make([]ResultSet, len(rsList)) - for i := 0; i < len(rsList); i++ { - rs[i] = &tidbResultSet{ - recordSet: rsList[i], - } - } - return + return &tidbResultSet{ + recordSet: rs, + }, nil } -// ExecuteInternal implements QueryCtx ExecuteInternal method. -func (tc *TiDBContext) ExecuteInternal(ctx context.Context, sql string) (rs []ResultSet, err error) { - rsList, err := tc.session.ExecuteInternal(ctx, sql) - if err != nil { - return - } - if len(rsList) == 0 { // result ok - return - } - rs = make([]ResultSet, len(rsList)) - for i := 0; i < len(rsList); i++ { - rs[i] = &tidbResultSet{ - recordSet: rsList[i], - } - } - return +// Parse implements QueryCtx interface. +func (tc *TiDBContext) Parse(ctx context.Context, sql string) ([]ast.StmtNode, error) { + return tc.session.Parse(ctx, sql) } // SetSessionManager implements the QueryCtx interface. diff --git a/server/http_handler.go b/server/http_handler.go index 1c09174464cd5..b85e097025eda 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" @@ -54,7 +55,9 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/pdapi" log "github.com/sirupsen/logrus" @@ -152,7 +155,7 @@ func (t *tikvHandlerTool) getRegionIDByKey(encodedKey []byte) (uint64, error) { } func (t *tikvHandlerTool) getMvccByHandle(tableID, handle int64) (*mvccKV, error) { - encodedKey := tablecodec.EncodeRowKeyWithHandle(tableID, handle) + encodedKey := tablecodec.EncodeRowKeyWithHandle(tableID, kv.IntHandle(handle)) data, err := t.GetMvccByEncodedKey(encodedKey) if err != nil { return nil, err @@ -244,7 +247,7 @@ func (t *tikvHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, if err != nil { return nil, errors.Trace(err) } - encodedKey, _, err := idx.GenIndexKey(sc, idxRow, handle, nil) + encodedKey, _, err := idx.GenIndexKey(sc, idxRow, kv.IntHandle(handle), nil) if err != nil { return nil, errors.Trace(err) } @@ -482,7 +485,7 @@ func (vh valueHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } // Decode a column. m := make(map[int64]*types.FieldType, 1) - m[int64(colID)] = ft + m[colID] = ft loc := time.UTC vals, err := tablecodec.DecodeRow(valData, m, loc) if err != nil { @@ -490,7 +493,7 @@ func (vh valueHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { return } - v := vals[int64(colID)] + v := vals[colID] val, err := v.ToString() if err != nil { writeError(w, err) @@ -720,32 +723,92 @@ func (h flashReplicaHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) replicaInfos := make([]*tableFlashReplicaInfo, 0) allDBs := schema.AllSchemas() for _, db := range allDBs { - tables := schema.SchemaTables(db.Name) - for _, tbl := range tables { - tblInfo := tbl.Meta() - if tblInfo.TiFlashReplica == nil { - continue - } - if pi := tblInfo.GetPartitionInfo(); pi != nil { - for _, p := range pi.Definitions { - replicaInfos = append(replicaInfos, &tableFlashReplicaInfo{ - ID: p.ID, - ReplicaCount: tblInfo.TiFlashReplica.Count, - LocationLabels: tblInfo.TiFlashReplica.LocationLabels, - Available: tblInfo.TiFlashReplica.IsPartitionAvailable(p.ID), - }) - } - continue - } + tbls := schema.SchemaTables(db.Name) + for _, tbl := range tbls { + replicaInfos = h.getTiFlashReplicaInfo(tbl.Meta(), replicaInfos) + } + } + dropedOrTruncateReplicaInfos, err := h.getDropOrTruncateTableTiflash(schema) + if err != nil { + writeError(w, err) + return + } + replicaInfos = append(replicaInfos, dropedOrTruncateReplicaInfos...) + writeData(w, replicaInfos) +} + +func (h flashReplicaHandler) getTiFlashReplicaInfo(tblInfo *model.TableInfo, replicaInfos []*tableFlashReplicaInfo) []*tableFlashReplicaInfo { + if tblInfo.TiFlashReplica == nil { + return replicaInfos + } + if pi := tblInfo.GetPartitionInfo(); pi != nil { + for _, p := range pi.Definitions { replicaInfos = append(replicaInfos, &tableFlashReplicaInfo{ - ID: tblInfo.ID, + ID: p.ID, ReplicaCount: tblInfo.TiFlashReplica.Count, LocationLabels: tblInfo.TiFlashReplica.LocationLabels, - Available: tblInfo.TiFlashReplica.Available, + Available: tblInfo.TiFlashReplica.IsPartitionAvailable(p.ID), }) } + return replicaInfos } - writeData(w, replicaInfos) + replicaInfos = append(replicaInfos, &tableFlashReplicaInfo{ + ID: tblInfo.ID, + ReplicaCount: tblInfo.TiFlashReplica.Count, + LocationLabels: tblInfo.TiFlashReplica.LocationLabels, + Available: tblInfo.TiFlashReplica.Available, + }) + return replicaInfos +} + +func (h flashReplicaHandler) getDropOrTruncateTableTiflash(currentSchema infoschema.InfoSchema) ([]*tableFlashReplicaInfo, error) { + s, err := session.CreateSession(h.Store.(kv.Storage)) + if err != nil { + return nil, errors.Trace(err) + } + + if s != nil { + defer s.Close() + } + + store := domain.GetDomain(s).Store() + txn, err := store.Begin() + if err != nil { + return nil, errors.Trace(err) + } + gcSafePoint, err := gcutil.GetGCSafePoint(s) + if err != nil { + return nil, err + } + replicaInfos := make([]*tableFlashReplicaInfo, 0) + uniqueIDMap := make(map[int64]struct{}) + handleJobAndTableInfo := func(job *model.Job, tblInfo *model.TableInfo) (bool, error) { + // Avoid duplicate table ID info. + if _, ok := currentSchema.TableByID(tblInfo.ID); ok { + return false, nil + } + if _, ok := uniqueIDMap[tblInfo.ID]; ok { + return false, nil + } + uniqueIDMap[tblInfo.ID] = struct{}{} + replicaInfos = h.getTiFlashReplicaInfo(tblInfo, replicaInfos) + return false, nil + } + dom := domain.GetDomain(s) + fn := func(jobs []*model.Job) (bool, error) { + return executor.GetDropOrTruncateTableInfoFromJobs(jobs, gcSafePoint, dom, handleJobAndTableInfo) + } + + err = admin.IterAllDDLJobs(txn, fn) + if err != nil { + if terror.ErrorEqual(variable.ErrSnapshotTooOld, err) { + // The err indicate that current ddl job and remain DDL jobs was been deleted by GC, + // just ignore the error and return directly. + return replicaInfos, nil + } + return nil, err + } + return replicaInfos, nil } type tableFlashReplicaStatus struct { @@ -1540,7 +1603,7 @@ func (h *mvccTxnHandler) handleMvccGetByTxn(params map[string]string) (interface return nil, errors.Trace(err) } startKey := tablecodec.EncodeTablePrefix(tableID) - endKey := tablecodec.EncodeRowKeyWithHandle(tableID, math.MaxInt64) + endKey := tablecodec.EncodeRowKeyWithHandle(tableID, kv.IntHandle(math.MaxInt64)) return h.getMvccByStartTs(uint64(startTS), startKey, endKey) } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 423c756f913d1..74f383135f9bf 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" @@ -95,7 +96,7 @@ func (ts *HTTPHandlerTestSuite) TestRegionIndexRange(c *C) { startKey := tablecodec.EncodeIndexSeekKey(sTableID, sIndex, encodedValue) recordPrefix := tablecodec.GenTableRecordPrefix(eTableID) - endKey := tablecodec.EncodeRecordKey(recordPrefix, recordID) + endKey := tablecodec.EncodeRecordKey(recordPrefix, kv.IntHandle(recordID)) region := &tikv.KeyLocation{ Region: tikv.RegionVerID{}, @@ -508,6 +509,31 @@ func (ts *HTTPHandlerTestSuite) TestTiFlashReplica(c *C) { ts.startServer(c) ts.prepareData(c) defer ts.stopServer(c) + + db, err := sql.Open("mysql", ts.getDSN()) + c.Assert(err, IsNil, Commentf("Error connecting")) + defer db.Close() + dbt := &DBTest{c, db} + + defer func(originGC bool) { + if originGC { + ddl.EmulatorGCEnable() + } else { + ddl.EmulatorGCDisable() + } + }(ddl.IsEmulatorGCEnable()) + + // Disable emulator GC. + // Otherwise emulator GC will delete table record as soon as possible after execute drop table DDL. + ddl.EmulatorGCDisable() + gcTimeFormat := "20060102-15:04:05 -0700 MST" + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) + safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', ''),('tikv_gc_enable','true','') + ON DUPLICATE KEY + UPDATE variable_value = '%[1]s'` + // Set GC safe point and enable GC. + dbt.mustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) + resp, err := ts.fetchStatus("/tiflash/replica") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) @@ -516,11 +542,6 @@ func (ts *HTTPHandlerTestSuite) TestTiFlashReplica(c *C) { c.Assert(err, IsNil) c.Assert(len(data), Equals, 0) - db, err := sql.Open("mysql", ts.getDSN()) - c.Assert(err, IsNil, Commentf("Error connecting")) - defer db.Close() - dbt := &DBTest{c, db} - dbt.mustExec("use tidb") dbt.mustExec("alter table test set tiflash replica 2 location labels 'a','b';") @@ -564,16 +585,30 @@ func (ts *HTTPHandlerTestSuite) TestTiFlashReplica(c *C) { // Should not take effect. dbt.mustExec("alter table test set tiflash replica 2 location labels 'a','b';") - resp, err = ts.fetchStatus("/tiflash/replica") - c.Assert(err, IsNil) - decoder = json.NewDecoder(resp.Body) - err = decoder.Decode(&data) - c.Assert(err, IsNil) - resp.Body.Close() - c.Assert(len(data), Equals, 1) - c.Assert(data[0].ReplicaCount, Equals, uint64(2)) - c.Assert(strings.Join(data[0].LocationLabels, ","), Equals, "a,b") - c.Assert(data[0].Available, Equals, true) // The status should be true now. + checkFunc := func() { + resp, err = ts.fetchStatus("/tiflash/replica") + c.Assert(err, IsNil) + decoder = json.NewDecoder(resp.Body) + err = decoder.Decode(&data) + c.Assert(err, IsNil) + resp.Body.Close() + c.Assert(len(data), Equals, 1) + c.Assert(data[0].ReplicaCount, Equals, uint64(2)) + c.Assert(strings.Join(data[0].LocationLabels, ","), Equals, "a,b") + c.Assert(data[0].Available, Equals, true) // The status should be true now. + } + + // Test for get dropped table tiflash replica info. + dbt.mustExec("drop table test") + checkFunc() + + // Test unique table id replica info. + dbt.mustExec("flashback table test") + checkFunc() + dbt.mustExec("drop table test") + checkFunc() + dbt.mustExec("flashback table test") + checkFunc() // Test for partition table. dbt.mustExec("alter table pt set tiflash replica 2 location labels 'a','b';") @@ -618,16 +653,23 @@ func (ts *HTTPHandlerTestSuite) TestTiFlashReplica(c *C) { resp, err = ts.postStatus("/tiflash/replica", "application/json", bytes.NewBuffer([]byte(req))) c.Assert(err, IsNil) resp.Body.Close() - resp, err = ts.fetchStatus("/tiflash/replica") - c.Assert(err, IsNil) - decoder = json.NewDecoder(resp.Body) - err = decoder.Decode(&data) - c.Assert(err, IsNil) - resp.Body.Close() - c.Assert(len(data), Equals, 3) - c.Assert(data[0].Available, Equals, true) - c.Assert(data[1].Available, Equals, true) - c.Assert(data[2].Available, Equals, true) + checkFunc = func() { + resp, err = ts.fetchStatus("/tiflash/replica") + c.Assert(err, IsNil) + decoder = json.NewDecoder(resp.Body) + err = decoder.Decode(&data) + c.Assert(err, IsNil) + resp.Body.Close() + c.Assert(len(data), Equals, 3) + c.Assert(data[0].Available, Equals, true) + c.Assert(data[1].Available, Equals, true) + c.Assert(data[2].Available, Equals, true) + } + + // Test for get truncated table tiflash replica info. + dbt.mustExec("truncate table pt") + dbt.mustExec("alter table pt set tiflash replica 0;") + checkFunc() } func (ts *HTTPHandlerTestSuite) TestDecodeColumnValue(c *C) { diff --git a/server/http_status.go b/server/http_status.go index 2326ecbc681f4..4b871a60495e8 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -46,6 +46,7 @@ import ( "github.com/soheilhy/cmux" "github.com/tiancaiamao/appdash/traceapp" "go.uber.org/zap" + "google.golang.org/grpc/channelz/service" static "sourcegraph.com/sourcegraph/appdash-data" ) @@ -305,6 +306,7 @@ func (s *Server) startStatusServerAndRPCServer(serverMux *http.ServeMux) { s.statusServer = &http.Server{Addr: s.statusAddr, Handler: CorsHandler{handler: serverMux, cfg: s.cfg}} s.grpcServer = NewRPCServer(s.cfg, s.dom, s) + service.RegisterChannelzServiceToServer(s.grpcServer) go util.WithRecovery(func() { err := s.grpcServer.Serve(grpcL) diff --git a/server/tidb_test.go b/server/tidb_test.go index bd83e123273ec..0b35f5821d982 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -729,7 +729,7 @@ func (ts *tidbTestSuite) TestCreateTableFlen(c *C) { // issue #4540 qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) c.Assert(err, IsNil) - _, err = qctx.Execute(context.Background(), "use test;") + _, err = Execute(context.Background(), qctx, "use test;") c.Assert(err, IsNil) ctx := context.Background() @@ -762,44 +762,55 @@ func (ts *tidbTestSuite) TestCreateTableFlen(c *C) { "`z` decimal(20, 4)," + "PRIMARY KEY (`a`)" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin" - _, err = qctx.Execute(ctx, testSQL) + _, err = Execute(ctx, qctx, testSQL) c.Assert(err, IsNil) - rs, err := qctx.Execute(ctx, "show create table t1") + rs, err := Execute(ctx, qctx, "show create table t1") c.Assert(err, IsNil) - req := rs[0].NewChunk() - err = rs[0].Next(ctx, req) + req := rs.NewChunk() + err = rs.Next(ctx, req) c.Assert(err, IsNil) - cols := rs[0].Columns() + cols := rs.Columns() c.Assert(err, IsNil) c.Assert(len(cols), Equals, 2) c.Assert(int(cols[0].ColumnLength), Equals, 5*tmysql.MaxBytesOfCharacter) c.Assert(int(cols[1].ColumnLength), Equals, len(req.GetRow(0).GetString(1))*tmysql.MaxBytesOfCharacter) // for issue#5246 - rs, err = qctx.Execute(ctx, "select y, z from t1") + rs, err = Execute(ctx, qctx, "select y, z from t1") c.Assert(err, IsNil) - cols = rs[0].Columns() + cols = rs.Columns() c.Assert(len(cols), Equals, 2) c.Assert(int(cols[0].ColumnLength), Equals, 21) c.Assert(int(cols[1].ColumnLength), Equals, 22) } +func Execute(ctx context.Context, qc QueryCtx, sql string) (ResultSet, error) { + stmts, err := qc.Parse(ctx, sql) + if err != nil { + return nil, err + } + if len(stmts) != 1 { + panic("wrong input for Execute: " + sql) + } + return qc.ExecuteStmt(ctx, stmts[0]) +} + func (ts *tidbTestSuite) TestShowTablesFlen(c *C) { qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) c.Assert(err, IsNil) - _, err = qctx.Execute(context.Background(), "use test;") + ctx := context.Background() + _, err = Execute(ctx, qctx, "use test;") c.Assert(err, IsNil) - ctx := context.Background() testSQL := "create table abcdefghijklmnopqrstuvwxyz (i int)" - _, err = qctx.Execute(ctx, testSQL) + _, err = Execute(ctx, qctx, testSQL) c.Assert(err, IsNil) - rs, err := qctx.Execute(ctx, "show tables") + rs, err := Execute(ctx, qctx, "show tables") c.Assert(err, IsNil) - req := rs[0].NewChunk() - err = rs[0].Next(ctx, req) + req := rs.NewChunk() + err = rs.Next(ctx, req) c.Assert(err, IsNil) - cols := rs[0].Columns() + cols := rs.Columns() c.Assert(err, IsNil) c.Assert(len(cols), Equals, 1) c.Assert(int(cols[0].ColumnLength), Equals, 26*tmysql.MaxBytesOfCharacter) @@ -815,7 +826,7 @@ func checkColNames(c *C, columns []*ColumnInfo, names ...string) { func (ts *tidbTestSuite) TestFieldList(c *C) { qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) c.Assert(err, IsNil) - _, err = qctx.Execute(context.Background(), "use test;") + _, err = Execute(context.Background(), qctx, "use test;") c.Assert(err, IsNil) ctx := context.Background() @@ -840,7 +851,7 @@ func (ts *tidbTestSuite) TestFieldList(c *C) { c_json JSON, c_year year )` - _, err = qctx.Execute(ctx, testSQL) + _, err = Execute(ctx, qctx, testSQL) c.Assert(err, IsNil) colInfos, err := qctx.FieldList("t") c.Assert(err, IsNil) @@ -878,15 +889,15 @@ func (ts *tidbTestSuite) TestFieldList(c *C) { tooLongColumnAsName := "COALESCE(0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0)" columnAsName := tooLongColumnAsName[:tmysql.MaxAliasIdentifierLen] - rs, err := qctx.Execute(ctx, "select "+tooLongColumnAsName) + rs, err := Execute(ctx, qctx, "select "+tooLongColumnAsName) c.Assert(err, IsNil) - cols := rs[0].Columns() + cols := rs.Columns() c.Assert(cols[0].OrgName, Equals, tooLongColumnAsName) c.Assert(cols[0].Name, Equals, columnAsName) - rs, err = qctx.Execute(ctx, "select c_bit as '"+tooLongColumnAsName+"' from t") + rs, err = Execute(ctx, qctx, "select c_bit as '"+tooLongColumnAsName+"' from t") c.Assert(err, IsNil) - cols = rs[0].Columns() + cols = rs.Columns() c.Assert(cols[0].OrgName, Equals, "c_bit") c.Assert(cols[0].Name, Equals, columnAsName) } @@ -902,9 +913,9 @@ func (ts *tidbTestSuite) TestNullFlag(c *C) { c.Assert(err, IsNil) ctx := context.Background() - rs, err := qctx.Execute(ctx, "select 1") + rs, err := Execute(ctx, qctx, "select 1") c.Assert(err, IsNil) - cols := rs[0].Columns() + cols := rs.Columns() c.Assert(len(cols), Equals, 1) expectFlag := uint16(tmysql.NotNullFlag | tmysql.BinaryFlag) c.Assert(dumpFlag(cols[0].Type, cols[0].Flag), Equals, expectFlag) diff --git a/session/bootstrap.go b/session/bootstrap.go index 9863730fd144b..67b1a285c07a7 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -77,6 +77,7 @@ const ( Shutdown_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Reload_priv ENUM('N','Y') NOT NULL DEFAULT 'N', FILE_priv ENUM('N','Y') NOT NULL DEFAULT 'N', + Config_priv ENUM('N','Y') NOT NULL DEFAULT 'N', PRIMARY KEY (Host, User));` // CreateGlobalPrivTable is the SQL statement creates Global scope privilege table in system db. CreateGlobalPrivTable = "CREATE TABLE if not exists mysql.global_priv (" + @@ -274,7 +275,9 @@ const ( // CreateExprPushdownBlacklist stores the expressions which are not allowed to be pushed down. CreateExprPushdownBlacklist = `CREATE TABLE IF NOT EXISTS mysql.expr_pushdown_blacklist ( - name char(100) NOT NULL + name char(100) NOT NULL, + store_type char(100) NOT NULL DEFAULT 'tikv,tiflash,tidb', + reason varchar(200) );` // CreateOptRuleBlacklist stores the list of disabled optimizing operations. @@ -373,6 +376,63 @@ const ( // see https://github.com/pingcap/tidb/pull/14574 for more details. version40 = 40 version41 = 41 + // version42 add storeType and reason column in expr_pushdown_blacklist + version42 = 42 + // version43 updates global variables related to statement summary. + version43 = 43 + // version44 delete tidb_isolation_read_engines from mysql.global_variables to avoid unexpected behavior after upgrade. + version44 = 44 + // version45 introduces CONFIG_PRIV for SET CONFIG statements. + version45 = 45 +) + +var ( + bootstrapVersion = []func(Session, int64){ + upgradeToVer2, + upgradeToVer3, + upgradeToVer4, + upgradeToVer5, + upgradeToVer6, + upgradeToVer7, + upgradeToVer8, + upgradeToVer9, + upgradeToVer10, + upgradeToVer11, + upgradeToVer12, + upgradeToVer13, + upgradeToVer14, + upgradeToVer15, + upgradeToVer16, + upgradeToVer17, + upgradeToVer18, + upgradeToVer19, + upgradeToVer20, + upgradeToVer21, + upgradeToVer22, + upgradeToVer23, + upgradeToVer24, + upgradeToVer25, + upgradeToVer26, + upgradeToVer27, + upgradeToVer28, + upgradeToVer29, + upgradeToVer30, + upgradeToVer31, + upgradeToVer32, + upgradeToVer33, + upgradeToVer34, + upgradeToVer35, + upgradeToVer36, + upgradeToVer37, + upgradeToVer38, + upgradeToVer39, + upgradeToVer40, + upgradeToVer41, + upgradeToVer42, + upgradeToVer43, + upgradeToVer44, + upgradeToVer45, + } ) func checkBootstrapped(s Session) (bool, error) { @@ -437,164 +497,8 @@ func upgrade(s Session) { return } // Do upgrade works then update bootstrap version. - if ver < version2 { - upgradeToVer2(s) - ver = version2 - } - if ver < version3 { - upgradeToVer3(s) - } - if ver < version4 { - upgradeToVer4(s) - } - - if ver < version5 { - upgradeToVer5(s) - } - - if ver < version6 { - upgradeToVer6(s) - } - - if ver < version7 { - upgradeToVer7(s) - } - - if ver < version8 { - upgradeToVer8(s) - } - - if ver < version9 { - upgradeToVer9(s) - } - - if ver < version10 { - upgradeToVer10(s) - } - - if ver < version11 { - upgradeToVer11(s) - } - - if ver < version12 { - upgradeToVer12(s) - } - - if ver < version13 { - upgradeToVer13(s) - } - - if ver < version14 { - upgradeToVer14(s) - } - - if ver < version15 { - upgradeToVer15(s) - } - - if ver < version16 { - upgradeToVer16(s) - } - - if ver < version17 { - upgradeToVer17(s) - } - - if ver < version18 { - upgradeToVer18(s) - } - - if ver < version19 { - upgradeToVer19(s) - } - - if ver < version20 { - upgradeToVer20(s) - } - - if ver < version21 { - upgradeToVer21(s) - } - - if ver < version22 { - upgradeToVer22(s) - } - - if ver < version23 { - upgradeToVer23(s) - } - - if ver < version24 { - upgradeToVer24(s) - } - - if ver < version25 { - upgradeToVer25(s) - } - - if ver < version26 { - upgradeToVer26(s) - } - - if ver < version27 { - upgradeToVer27(s) - } - - if ver < version28 { - upgradeToVer28(s) - } - - // upgradeToVer29 only need to be run when the current version is 28. - if ver == version28 { - upgradeToVer29(s) - } - - if ver < version30 { - upgradeToVer30(s) - } - - if ver < version31 { - upgradeToVer31(s) - } - - if ver < version32 { - upgradeToVer32(s) - } - - if ver < version33 { - upgradeToVer33(s) - } - - if ver < version34 { - upgradeToVer34(s) - } - - if ver < version35 { - upgradeToVer35(s) - } - - if ver < version36 { - upgradeToVer36(s) - } - - if ver < version37 { - upgradeToVer37(s) - } - - if ver < version38 { - upgradeToVer38(s) - } - - if ver < version39 { - upgradeToVer39(s) - } - - if ver < version40 { - upgradeToVer40(s) - } - - if ver < version41 { - upgradeToVer41(s) + for _, upgrade := range bootstrapVersion { + upgrade(s, ver) } updateBootstrapVer(s) @@ -622,7 +526,10 @@ func upgrade(s Session) { } // upgradeToVer2 updates to version 2. -func upgradeToVer2(s Session) { +func upgradeToVer2(s Session, ver int64) { + if ver >= version2 { + return + } // Version 2 add two system variable for DistSQL concurrency controlling. // Insert distsql related system variable. distSQLVars := []string{variable.TiDBDistSQLScanConcurrency} @@ -637,7 +544,10 @@ func upgradeToVer2(s Session) { } // upgradeToVer3 updates to version 3. -func upgradeToVer3(s Session) { +func upgradeToVer3(s Session, ver int64) { + if ver >= version3 { + return + } // Version 3 fix tx_read_only variable value. sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %s.%s set variable_value = '0' where variable_name = 'tx_read_only';", mysql.SystemDB, mysql.GlobalVariablesTable) @@ -645,37 +555,55 @@ func upgradeToVer3(s Session) { } // upgradeToVer4 updates to version 4. -func upgradeToVer4(s Session) { +func upgradeToVer4(s Session, ver int64) { + if ver >= version4 { + return + } sql := CreateStatsMetaTable mustExecute(s, sql) } -func upgradeToVer5(s Session) { +func upgradeToVer5(s Session, ver int64) { + if ver >= version5 { + return + } mustExecute(s, CreateStatsColsTable) mustExecute(s, CreateStatsBucketsTable) } -func upgradeToVer6(s Session) { +func upgradeToVer6(s Session, ver int64) { + if ver >= version6 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Super_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Show_db_priv`", infoschema.ErrColumnExists) // For reasons of compatibility, set the non-exists privilege column value to 'Y', as TiDB doesn't check them in older versions. mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Super_priv='Y'") } -func upgradeToVer7(s Session) { +func upgradeToVer7(s Session, ver int64) { + if ver >= version7 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Process_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Drop_priv`", infoschema.ErrColumnExists) // For reasons of compatibility, set the non-exists privilege column value to 'Y', as TiDB doesn't check them in older versions. mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Process_priv='Y'") } -func upgradeToVer8(s Session) { +func upgradeToVer8(s Session, ver int64) { + if ver >= version8 { + return + } // This is a dummy upgrade, it checks whether upgradeToVer7 success, if not, do it again. if _, err := s.Execute(context.Background(), "SELECT HIGH_PRIORITY `Process_priv` from mysql.user limit 0"); err == nil { return } - upgradeToVer7(s) + upgradeToVer7(s, ver) } -func upgradeToVer9(s Session) { +func upgradeToVer9(s Session, ver int64) { + if ver >= version9 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Trigger_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_user_priv`", infoschema.ErrColumnExists) // For reasons of compatibility, set the non-exists privilege column value to 'Y', as TiDB doesn't check them in older versions. mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Trigger_priv='Y'") @@ -693,7 +621,10 @@ func doReentrantDDL(s Session, sql string, ignorableErrs ...error) { } } -func upgradeToVer10(s Session) { +func upgradeToVer10(s Session, ver int64) { + if ver >= version10 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.stats_buckets CHANGE COLUMN `value` `upper_bound` BLOB NOT NULL", infoschema.ErrColumnNotExists, infoschema.ErrColumnExists) doReentrantDDL(s, "ALTER TABLE mysql.stats_buckets ADD COLUMN `lower_bound` BLOB", infoschema.ErrColumnExists) doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `null_count` bigint(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) @@ -701,7 +632,10 @@ func upgradeToVer10(s Session) { doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms DROP COLUMN use_count_to_estimate", ddl.ErrCantDropFieldOrKey) } -func upgradeToVer11(s Session) { +func upgradeToVer11(s Session, ver int64) { + if ver >= version11 { + return + } _, err := s.Execute(context.Background(), "ALTER TABLE mysql.user ADD COLUMN `References_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Grant_priv`") if err != nil { if terror.ErrorEqual(err, infoschema.ErrColumnExists) { @@ -712,7 +646,10 @@ func upgradeToVer11(s Session) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET References_priv='Y'") } -func upgradeToVer12(s Session) { +func upgradeToVer12(s Session, ver int64) { + if ver >= version12 { + return + } ctx := context.Background() _, err := s.Execute(ctx, "BEGIN") terror.MustNil(err) @@ -755,7 +692,10 @@ func upgradeToVer12(s Session) { mustExecute(s, "COMMIT") } -func upgradeToVer13(s Session) { +func upgradeToVer13(s Session, ver int64) { + if ver >= version13 { + return + } sqls := []string{ "ALTER TABLE mysql.user ADD COLUMN `Create_tmp_table_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Super_priv`", "ALTER TABLE mysql.user ADD COLUMN `Lock_tables_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Create_tmp_table_priv`", @@ -779,7 +719,10 @@ func upgradeToVer13(s Session) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Create_view_priv='Y',Show_view_priv='Y' WHERE Create_priv='Y'") } -func upgradeToVer14(s Session) { +func upgradeToVer14(s Session, ver int64) { + if ver >= version14 { + return + } sqls := []string{ "ALTER TABLE mysql.db ADD COLUMN `References_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Grant_priv`", "ALTER TABLE mysql.db ADD COLUMN `Create_tmp_table_priv` enum('N','Y') CHARACTER SET utf8 NOT NULL DEFAULT 'N' AFTER `Alter_priv`", @@ -803,7 +746,10 @@ func upgradeToVer14(s Session) { } } -func upgradeToVer15(s Session) { +func upgradeToVer15(s Session, ver int64) { + if ver >= version15 { + return + } var err error _, err = s.Execute(context.Background(), CreateGCDeleteRangeTable) if err != nil { @@ -811,29 +757,47 @@ func upgradeToVer15(s Session) { } } -func upgradeToVer16(s Session) { +func upgradeToVer16(s Session, ver int64) { + if ver >= version16 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `cm_sketch` blob", infoschema.ErrColumnExists) } -func upgradeToVer17(s Session) { +func upgradeToVer17(s Session, ver int64) { + if ver >= version17 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.user MODIFY User CHAR(32)") } -func upgradeToVer18(s Session) { +func upgradeToVer18(s Session, ver int64) { + if ver >= version18 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `tot_col_size` bigint(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } -func upgradeToVer19(s Session) { +func upgradeToVer19(s Session, ver int64) { + if ver >= version19 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.db MODIFY User CHAR(32)") doReentrantDDL(s, "ALTER TABLE mysql.tables_priv MODIFY User CHAR(32)") doReentrantDDL(s, "ALTER TABLE mysql.columns_priv MODIFY User CHAR(32)") } -func upgradeToVer20(s Session) { +func upgradeToVer20(s Session, ver int64) { + if ver >= version20 { + return + } doReentrantDDL(s, CreateStatsFeedbackTable) } -func upgradeToVer21(s Session) { +func upgradeToVer21(s Session, ver int64) { + if ver >= version21 { + return + } mustExecute(s, CreateGCDeleteRangeDoneTable) doReentrantDDL(s, "ALTER TABLE mysql.gc_delete_range DROP INDEX job_id", ddl.ErrCantDropFieldOrKey) @@ -841,11 +805,17 @@ func upgradeToVer21(s Session) { doReentrantDDL(s, "ALTER TABLE mysql.gc_delete_range DROP INDEX element_id", ddl.ErrCantDropFieldOrKey) } -func upgradeToVer22(s Session) { +func upgradeToVer22(s Session, ver int64) { + if ver >= version22 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `stats_ver` bigint(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } -func upgradeToVer23(s Session) { +func upgradeToVer23(s Session, ver int64) { + if ver >= version23 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `flag` bigint(64) NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } @@ -857,18 +827,27 @@ func writeSystemTZ(s Session) { } // upgradeToVer24 initializes `System` timezone according to docs/design/2018-09-10-adding-tz-env.md -func upgradeToVer24(s Session) { +func upgradeToVer24(s Session, ver int64) { + if ver >= version24 { + return + } writeSystemTZ(s) } // upgradeToVer25 updates tidb_max_chunk_size to new low bound value 32 if previous value is small than 32. -func upgradeToVer25(s Session) { +func upgradeToVer25(s Session, ver int64) { + if ver >= version25 { + return + } sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %[1]s.%[2]s SET VARIABLE_VALUE = '%[4]d' WHERE VARIABLE_NAME = '%[3]s' AND VARIABLE_VALUE < %[4]d", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBMaxChunkSize, variable.DefInitChunkSize) mustExecute(s, sql) } -func upgradeToVer26(s Session) { +func upgradeToVer26(s Session, ver int64) { + if ver >= version26 { + return + } mustExecute(s, CreateRoleEdgesTable) mustExecute(s, CreateDefaultRolesTable) doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Create_role_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) @@ -880,61 +859,98 @@ func upgradeToVer26(s Session) { mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Create_view_priv='Y',Show_view_priv='Y' WHERE Create_priv='Y'") } -func upgradeToVer27(s Session) { +func upgradeToVer27(s Session, ver int64) { + if ver >= version27 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `correlation` double NOT NULL DEFAULT 0", infoschema.ErrColumnExists) } -func upgradeToVer28(s Session) { +func upgradeToVer28(s Session, ver int64) { + if ver >= version28 { + return + } doReentrantDDL(s, CreateBindInfoTable) } -func upgradeToVer29(s Session) { +func upgradeToVer29(s Session, ver int64) { + // upgradeToVer29 only need to be run when the current version is 28. + if ver != version28 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.bind_info change create_time create_time timestamp(3)") doReentrantDDL(s, "ALTER TABLE mysql.bind_info change update_time update_time timestamp(3)") doReentrantDDL(s, "ALTER TABLE mysql.bind_info add index sql_index (original_sql(1024),default_db(1024))", ddl.ErrDupKeyName) } -func upgradeToVer30(s Session) { +func upgradeToVer30(s Session, ver int64) { + if ver >= version30 { + return + } mustExecute(s, CreateStatsTopNTable) } -func upgradeToVer31(s Session) { +func upgradeToVer31(s Session, ver int64) { + if ver >= version31 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.stats_histograms ADD COLUMN `last_analyze_pos` blob default null", infoschema.ErrColumnExists) } -func upgradeToVer32(s Session) { +func upgradeToVer32(s Session, ver int64) { + if ver >= version32 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.tables_priv MODIFY table_priv SET('Select','Insert','Update','Delete','Create','Drop','Grant', 'Index', 'Alter', 'Create View', 'Show View', 'Trigger', 'References')") } -func upgradeToVer33(s Session) { +func upgradeToVer33(s Session, ver int64) { + if ver >= version33 { + return + } doReentrantDDL(s, CreateExprPushdownBlacklist) } -func upgradeToVer34(s Session) { +func upgradeToVer34(s Session, ver int64) { + if ver >= version34 { + return + } doReentrantDDL(s, CreateOptRuleBlacklist) } -func upgradeToVer35(s Session) { +func upgradeToVer35(s Session, ver int64) { + if ver >= version35 { + return + } sql := fmt.Sprintf("UPDATE HIGH_PRIORITY %s.%s SET VARIABLE_NAME = '%s' WHERE VARIABLE_NAME = 'tidb_back_off_weight'", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBBackOffWeight) mustExecute(s, sql) } -func upgradeToVer36(s Session) { +func upgradeToVer36(s Session, ver int64) { + if ver >= version36 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Shutdown_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) // A root user will have those privileges after upgrading. mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Shutdown_priv='Y' where Super_priv='Y'") mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Create_tmp_table_priv='Y',Lock_tables_priv='Y',Create_routine_priv='Y',Alter_routine_priv='Y',Event_priv='Y' WHERE Super_priv='Y'") } -func upgradeToVer37(s Session) { +func upgradeToVer37(s Session, ver int64) { + if ver >= version37 { + return + } // when upgrade from old tidb and no 'tidb_enable_window_function' in GLOBAL_VARIABLES, init it with 0. sql := fmt.Sprintf("INSERT IGNORE INTO %s.%s (`VARIABLE_NAME`, `VARIABLE_VALUE`) VALUES ('%s', '%d')", mysql.SystemDB, mysql.GlobalVariablesTable, variable.TiDBEnableWindowFunction, 0) mustExecute(s, sql) } -func upgradeToVer38(s Session) { +func upgradeToVer38(s Session, ver int64) { + if ver >= version38 { + return + } var err error _, err = s.Execute(context.Background(), CreateGlobalPrivTable) if err != nil { @@ -942,7 +958,10 @@ func upgradeToVer38(s Session) { } } -func upgradeToVer39(s Session) { +func upgradeToVer39(s Session, ver int64) { + if ver >= version39 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Reload_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `File_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Reload_priv='Y' where Super_priv='Y'") @@ -960,16 +979,72 @@ func writeNewCollationParameter(s Session, flag bool) { mustExecute(s, sql) } -func upgradeToVer40(s Session) { +func upgradeToVer40(s Session, ver int64) { + if ver >= version40 { + return + } // There is no way to enable new collation for an existing TiDB cluster. writeNewCollationParameter(s, false) } -func upgradeToVer41(s Session) { +func upgradeToVer41(s Session, ver int64) { + if ver >= version41 { + return + } doReentrantDDL(s, "ALTER TABLE mysql.user CHANGE `password` `authentication_string` TEXT", infoschema.ErrColumnExists, infoschema.ErrColumnNotExists) doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `password` TEXT as (`authentication_string`)", infoschema.ErrColumnExists) } +// writeDefaultExprPushDownBlacklist writes default expr pushdown blacklist into mysql.expr_pushdown_blacklist +func writeDefaultExprPushDownBlacklist(s Session) { + mustExecute(s, "INSERT HIGH_PRIORITY INTO mysql.expr_pushdown_blacklist VALUES"+ + "('date_add','tiflash', 'DST(daylight saving time) does not take effect in TiFlash date_add'),"+ + "('cast','tiflash', 'Behavior of some corner cases(overflow, truncate etc) is different in TiFlash and TiDB')") +} + +func upgradeToVer42(s Session, ver int64) { + if ver >= version42 { + return + } + doReentrantDDL(s, "ALTER TABLE mysql.expr_pushdown_blacklist ADD COLUMN `store_type` char(100) NOT NULL DEFAULT 'tikv,tiflash,tidb'", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.expr_pushdown_blacklist ADD COLUMN `reason` varchar(200)", infoschema.ErrColumnExists) + writeDefaultExprPushDownBlacklist(s) +} + +// Convert statement summary global variables to non-empty values. +func writeStmtSummaryVars(s Session) { + sql := fmt.Sprintf("UPDATE %s.%s SET variable_value='%%s' WHERE variable_name='%%s' AND variable_value=''", mysql.SystemDB, mysql.GlobalVariablesTable) + stmtSummaryConfig := config.GetGlobalConfig().StmtSummary + mustExecute(s, fmt.Sprintf(sql, variable.BoolToIntStr(stmtSummaryConfig.Enable), variable.TiDBEnableStmtSummary)) + mustExecute(s, fmt.Sprintf(sql, variable.BoolToIntStr(stmtSummaryConfig.EnableInternalQuery), variable.TiDBStmtSummaryInternalQuery)) + mustExecute(s, fmt.Sprintf(sql, strconv.Itoa(stmtSummaryConfig.RefreshInterval), variable.TiDBStmtSummaryRefreshInterval)) + mustExecute(s, fmt.Sprintf(sql, strconv.Itoa(stmtSummaryConfig.HistorySize), variable.TiDBStmtSummaryHistorySize)) + mustExecute(s, fmt.Sprintf(sql, strconv.FormatUint(uint64(stmtSummaryConfig.MaxStmtCount), 10), variable.TiDBStmtSummaryMaxStmtCount)) + mustExecute(s, fmt.Sprintf(sql, strconv.FormatUint(uint64(stmtSummaryConfig.MaxSQLLength), 10), variable.TiDBStmtSummaryMaxSQLLength)) +} + +func upgradeToVer43(s Session, ver int64) { + if ver >= version43 { + return + } + writeStmtSummaryVars(s) +} + +func upgradeToVer44(s Session, ver int64) { + if ver >= version44 { + return + } + mustExecute(s, "DELETE FROM mysql.global_variables where variable_name = \"tidb_isolation_read_engines\"") +} + +func upgradeToVer45(s Session, ver int64) { + if ver >= version45 { + return + } + doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN `Config_priv` ENUM('N','Y') DEFAULT 'N'", infoschema.ErrColumnExists) + mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET Config_priv='Y' where Super_priv='Y'") +} + // updateBootstrapVer updates bootstrap version variable in mysql.TiDB table. func updateBootstrapVer(s Session) { // Update bootstrap version. @@ -1042,7 +1117,7 @@ func doDMLWorks(s Session) { // Insert a default user with empty password. mustExecute(s, `INSERT HIGH_PRIORITY INTO mysql.user VALUES - ("%", "root", "", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y")`) + ("%", "root", "", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y")`) // Init global system variables table. values := make([]string, 0, len(variable.SysVars)) @@ -1077,6 +1152,10 @@ func doDMLWorks(s Session) { writeNewCollationParameter(s, config.GetGlobalConfig().NewCollationsEnabledOnFirstBootstrap) + writeDefaultExprPushDownBlacklist(s) + + writeStmtSummaryVars(s) + _, err := s.Execute(context.Background(), "COMMIT") if err != nil { sleepTime := 1 * time.Second diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index 28eaf6b6091c3..f03e66300ffb8 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -54,7 +54,7 @@ func (s *testBootstrapSuite) TestBootstrap(c *C) { c.Assert(err, IsNil) c.Assert(req.NumRows() == 0, IsFalse) datums := statistics.RowToDatums(req.GetRow(0), r.Fields()) - match(c, datums, `%`, "root", []byte(""), "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y") + match(c, datums, `%`, "root", []byte(""), "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y") c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "anyhost"}, []byte(""), []byte("")), IsTrue) mustExecSQL(c, se, "USE test;") @@ -159,7 +159,7 @@ func (s *testBootstrapSuite) TestBootstrapWithError(c *C) { c.Assert(req.NumRows() == 0, IsFalse) row := req.GetRow(0) datums := statistics.RowToDatums(row, r.Fields()) - match(c, datums, `%`, "root", []byte(""), "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y") + match(c, datums, `%`, "root", []byte(""), "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y") c.Assert(r.Close(), IsNil) mustExecSQL(c, se, "USE test;") @@ -318,3 +318,21 @@ func (s *testBootstrapSuite) TestBootstrapInitExpensiveQueryHandle(c *C) { dom.InitExpensiveQueryHandle() c.Assert(dom.ExpensiveQueryHandle(), NotNil) } + +func (s *testBootstrapSuite) TestStmtSummary(c *C) { + defer testleak.AfterTest(c)() + ctx := context.Background() + store, dom := newStoreWithBootstrap(c, s.dbName) + defer store.Close() + defer dom.Close() + se := newSession(c, store, s.dbName) + mustExecSQL(c, se, `update mysql.global_variables set variable_value='' where variable_name='tidb_enable_stmt_summary'`) + writeStmtSummaryVars(se) + + r := mustExecSQL(c, se, "select variable_value from mysql.global_variables where variable_name='tidb_enable_stmt_summary'") + req := r.NewChunk() + c.Assert(r.Next(ctx, req), IsNil) + row := req.GetRow(0) + c.Assert(row.GetBytes(0), BytesEquals, []byte("1")) + c.Assert(r.Close(), IsNil) +} diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 8974ff51b04e9..439b3b9f0979f 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -196,11 +196,11 @@ func (s *testPessimisticSuite) TestSingleStatementRollback(c *C) { tk.MustExec("create table single_statement (id int primary key, v int)") tk.MustExec("insert into single_statement values (1, 1), (2, 1), (3, 1), (4, 1)") tblID := tk.GetTableID("single_statement") - s.cluster.SplitTable(s.mvccStore, tblID, 2) - region1Key := codec.EncodeBytes(nil, tablecodec.EncodeRowKeyWithHandle(tblID, 1)) + s.cluster.SplitTable(tblID, 2) + region1Key := codec.EncodeBytes(nil, tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(1))) region1, _ := s.cluster.GetRegionByKey(region1Key) region1ID := region1.Id - region2Key := codec.EncodeBytes(nil, tablecodec.EncodeRowKeyWithHandle(tblID, 3)) + region2Key := codec.EncodeBytes(nil, tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(3))) region2, _ := s.cluster.GetRegionByKey(region2Key) region2ID := region2.Id @@ -675,10 +675,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { defer wg.Done() // tk3 try lock c1 = 1 timeout 1sec tk3.MustExec("begin pessimistic") - start := time.Now() _, err := tk3.Exec("select * from tk where c1 = 1 for update") - c.Check(time.Since(start), GreaterEqual, time.Duration(1000*time.Millisecond)) - c.Check(time.Since(start), LessEqual, time.Duration(1100*time.Millisecond)) // unit test diff should not be too big c.Check(err.Error(), Equals, tikv.ErrLockWaitTimeout.Error()) tk3.MustExec("commit") }() @@ -689,10 +686,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { tk5 := testkit.NewTestKitWithInit(c, s.store) tk5.MustExec("set innodb_lock_wait_timeout = 2") tk5.MustExec("begin pessimistic") - start := time.Now() _, err := tk5.Exec("update tk set c2 = c2 - 1 where c1 = 1") - c.Check(time.Since(start), GreaterEqual, time.Duration(2000*time.Millisecond)) - c.Check(time.Since(start), LessEqual, time.Duration(2100*time.Millisecond)) // unit test diff should not be too big c.Check(err.Error(), Equals, tikv.ErrLockWaitTimeout.Error()) tk5.MustExec("rollback") }() @@ -706,8 +700,8 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { start := time.Now() _, err := tk2.Exec("delete from tk where c1 = 2") - c.Check(time.Since(start), GreaterEqual, time.Duration(1000*time.Millisecond)) - c.Check(time.Since(start), LessEqual, time.Duration(1100*time.Millisecond)) // unit test diff should not be too big + c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) + c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big c.Check(err.Error(), Equals, tikv.ErrLockWaitTimeout.Error()) tk4.MustExec("commit") @@ -724,8 +718,8 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { start = time.Now() _, err = tk2.Exec("delete from tk where c1 = 3") // tk2 tries to lock c1 = 3 fail, this delete should be rollback, but previous update should be keeped - c.Check(time.Since(start), GreaterEqual, time.Duration(1000*time.Millisecond)) - c.Check(time.Since(start), LessEqual, time.Duration(1100*time.Millisecond)) // unit test diff should not be too big + c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) + c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big c.Check(err.Error(), Equals, tikv.ErrLockWaitTimeout.Error()) tk2.MustExec("commit") @@ -796,8 +790,8 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeoutWaitStart(c *C) { waitErr := <-done c.Assert(waitErr, NotNil) c.Check(waitErr.Error(), Equals, tikv.ErrLockWaitTimeout.Error()) - c.Check(duration, GreaterEqual, time.Duration(1000*time.Millisecond)) - c.Check(duration, LessEqual, time.Duration(1100*time.Millisecond)) + c.Check(duration, GreaterEqual, 1000*time.Millisecond) + c.Check(duration, LessEqual, 3000*time.Millisecond) tk2.MustExec("rollback") tk3.MustExec("commit") } @@ -1255,3 +1249,143 @@ func (s *testPessimisticSuite) TestTxnWithExpiredPessimisticLocks(c *C) { tk.MustExec("update t1 set c2 = c2 + 1") tk.MustExec("rollback") } + +func (s *testPessimisticSuite) TestKillWaitLockTxn(c *C) { + // Test kill command works on waiting pessimistic lock. + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists test_kill") + tk.MustExec("create table test_kill (id int primary key, c int)") + tk.MustExec("insert test_kill values (1, 1)") + + tk.MustExec("begin pessimistic") + tk2.MustExec("begin pessimistic") + + tk.MustQuery("select * from test_kill where id = 1 for update") + errCh := make(chan error) + go func() { + var err error + defer func() { + errCh <- err + }() + time.Sleep(20 * time.Millisecond) + _, err = tk2.Exec("update test_kill set c = c + 1 where id = 1") + if err != nil { + return + } + }() + time.Sleep(100 * time.Millisecond) + sessVars := tk.Se.GetSessionVars() + // lock query in tk is killed, the ttl manager will stop + succ := atomic.CompareAndSwapUint32(&sessVars.Killed, 0, 1) + c.Assert(succ, IsTrue) + err := <-errCh + c.Assert(err, IsNil) + tk.Exec("rollback") + tk.MustExec("rollback") + tk2.MustExec("rollback") +} + +func (s *testPessimisticSuite) TestDupLockInconsistency(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int, index b (b))") + tk.MustExec("insert t (a) values (1), (1)") + tk.MustExec("begin pessimistic") + tk.MustExec("update t, (select a from t) s set t.b = s.a") + tk.MustExec("commit") + tk.MustExec("admin check table t") +} + +func (s *testPessimisticSuite) TestUseLockCacheInRCMode(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk3 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists test_kill") + tk.MustExec("CREATE TABLE SEQUENCE_VALUE_ITEM(SEQ_NAME varchar(60) NOT NULL, SEQ_ID decimal(18,0) DEFAULT NULL, " + + "PRIMARY KEY (SEQ_NAME))") + tk.MustExec("create table t1(c1 int, c2 int, unique key(c1))") + tk.MustExec(`insert into sequence_value_item values("OSCurrentStep", 0)`) + tk.MustExec("insert into t1 values(1, 1)") + tk.MustExec("insert into t1 values(2, 2)") + + // tk2 uses RC isolation level + tk2.MustExec("set @@tx_isolation='READ-COMMITTED'") + tk2.MustExec("set autocommit = 0") + + // test point get + tk2.MustExec("SELECT SEQ_ID FROM SEQUENCE_VALUE_ITEM WHERE SEQ_NAME='OSCurrentStep' FOR UPDATE") + tk2.MustExec("UPDATE SEQUENCE_VALUE_ITEM SET SEQ_ID=SEQ_ID+100 WHERE SEQ_NAME='OSCurrentStep'") + tk2.MustExec("rollback") + tk2.MustQuery("select * from t1 where c1 = 1 for update").Check(testkit.Rows("1 1")) + tk2.MustExec("update t1 set c2 = c2 + 10 where c1 = 1") + tk2.MustQuery("select * from t1 where c1 in (1, 2) for update").Check(testkit.Rows("1 11", "2 2")) + tk2.MustExec("update t1 set c2 = c2 + 10 where c1 in (2)") + tk2.MustQuery("select * from t1 where c1 in (1, 2) for update").Check(testkit.Rows("1 11", "2 12")) + tk2.MustExec("commit") + + // tk3 uses begin with RC isolation level + tk3.MustQuery("select * from SEQUENCE_VALUE_ITEM").Check(testkit.Rows("OSCurrentStep 0")) + tk3.MustExec("set @@tx_isolation='READ-COMMITTED'") + tk3.MustExec("begin") + tk3.MustExec("SELECT SEQ_ID FROM SEQUENCE_VALUE_ITEM WHERE SEQ_NAME='OSCurrentStep' FOR UPDATE") + tk3.MustExec("UPDATE SEQUENCE_VALUE_ITEM SET SEQ_ID=SEQ_ID+100 WHERE SEQ_NAME='OSCurrentStep'") + tk3.MustQuery("select * from t1 where c1 = 1 for update").Check(testkit.Rows("1 11")) + tk3.MustExec("update t1 set c2 = c2 + 10 where c1 = 1") + tk3.MustQuery("select * from t1 where c1 in (1, 2) for update").Check(testkit.Rows("1 21", "2 12")) + tk3.MustExec("update t1 set c2 = c2 + 10 where c1 in (2)") + tk3.MustQuery("select * from t1 where c1 in (1, 2) for update").Check(testkit.Rows("1 21", "2 22")) + tk3.MustExec("commit") + + // verify + tk.MustQuery("select * from SEQUENCE_VALUE_ITEM").Check(testkit.Rows("OSCurrentStep 100")) + tk.MustQuery("select * from SEQUENCE_VALUE_ITEM where SEQ_ID = 100").Check(testkit.Rows("OSCurrentStep 100")) + tk.MustQuery("select * from t1 where c1 = 2").Check(testkit.Rows("2 22")) + tk.MustQuery("select * from t1 where c1 in (1, 2, 3)").Check(testkit.Rows("1 21", "2 22")) + + // test batch point get + tk2.MustExec("set autocommit = 1") + tk2.MustExec("set autocommit = 0") + tk2.MustExec("SELECT SEQ_ID FROM SEQUENCE_VALUE_ITEM WHERE SEQ_NAME in ('OSCurrentStep') FOR UPDATE") + tk2.MustExec("UPDATE SEQUENCE_VALUE_ITEM SET SEQ_ID=SEQ_ID+100 WHERE SEQ_NAME in ('OSCurrentStep')") + tk2.MustQuery("select * from t1 where c1 in (1, 2, 3, 4, 5) for update").Check(testkit.Rows("1 21", "2 22")) + tk2.MustExec("update t1 set c2 = c2 + 10 where c1 in (1, 2, 3, 4, 5)") + tk2.MustQuery("select * from t1 where c1 in (1, 2, 3, 4, 5) for update").Check(testkit.Rows("1 31", "2 32")) + tk2.MustExec("commit") + tk2.MustExec("SELECT SEQ_ID FROM SEQUENCE_VALUE_ITEM WHERE SEQ_NAME in ('OSCurrentStep') FOR UPDATE") + tk2.MustExec("UPDATE SEQUENCE_VALUE_ITEM SET SEQ_ID=SEQ_ID+100 WHERE SEQ_NAME in ('OSCurrentStep')") + tk2.MustExec("rollback") + + tk.MustQuery("select * from SEQUENCE_VALUE_ITEM").Check(testkit.Rows("OSCurrentStep 200")) + tk.MustQuery("select * from SEQUENCE_VALUE_ITEM where SEQ_NAME in ('OSCurrentStep')").Check(testkit.Rows("OSCurrentStep 200")) + tk.MustQuery("select * from t1 where c1 in (1, 2, 3)").Check(testkit.Rows("1 31", "2 32")) + tk.MustExec("rollback") + tk2.MustExec("rollback") + tk3.MustExec("rollback") +} + +func (s *testPessimisticSuite) TestPointGetWithDeleteInMem(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists uk") + tk.MustExec("create table uk (c1 int primary key, c2 int, unique key uk(c2))") + tk.MustExec("insert uk values (1, 77), (2, 88), (3, 99)") + tk.MustExec("begin pessimistic") + tk.MustExec("delete from uk where c1 = 1") + tk.MustQuery("select * from uk where c2 = 77").Check(testkit.Rows()) + tk.MustQuery("select * from uk where c2 in(77, 88, 99)").Check(testkit.Rows("2 88", "3 99")) + tk.MustQuery("select * from uk").Check(testkit.Rows("2 88", "3 99")) + tk.MustQuery("select * from uk where c2 = 77 for update").Check(testkit.Rows()) + tk.MustQuery("select * from uk where c2 in(77, 88, 99) for update").Check(testkit.Rows("2 88", "3 99")) + tk.MustExec("rollback") + tk2.MustQuery("select * from uk where c1 = 1").Check(testkit.Rows("1 77")) + tk.MustExec("begin") + tk.MustExec("update uk set c1 = 10 where c1 = 1") + tk.MustQuery("select * from uk where c2 = 77").Check(testkit.Rows("10 77")) + tk.MustQuery("select * from uk where c2 in(77, 88, 99)").Check(testkit.Rows("10 77", "2 88", "3 99")) + tk.MustExec("commit") + tk2.MustQuery("select * from uk where c1 = 1").Check(testkit.Rows()) + tk2.MustQuery("select * from uk where c2 = 77").Check(testkit.Rows("10 77")) + tk2.MustQuery("select * from uk where c1 = 10").Check(testkit.Rows("10 77")) + tk.MustExec("drop table if exists uk") +} diff --git a/session/session.go b/session/session.go index 36e08975ec21f..dfb781f2cd256 100644 --- a/session/session.go +++ b/session/session.go @@ -83,9 +83,6 @@ var ( transactionDurationGeneralCommit = metrics.TransactionDuration.WithLabelValues(metrics.LblGeneral, metrics.LblCommit) transactionDurationGeneralAbort = metrics.TransactionDuration.WithLabelValues(metrics.LblGeneral, metrics.LblAbort) - sessionExecuteRunDurationInternal = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblInternal) - sessionExecuteRunDurationGeneral = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblGeneral) - sessionExecuteCompileDurationInternal = metrics.SessionExecuteCompileDuration.WithLabelValues(metrics.LblInternal) sessionExecuteCompileDurationGeneral = metrics.SessionExecuteCompileDuration.WithLabelValues(metrics.LblGeneral) sessionExecuteParseDurationInternal = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblInternal) @@ -95,13 +92,16 @@ var ( // Session context, it is consistent with the lifecycle of a client connection. type Session interface { sessionctx.Context - Status() uint16 // Flag of current status, such as autocommit. - LastInsertID() uint64 // LastInsertID is the last inserted auto_increment ID. - LastMessage() string // LastMessage is the info message that may be generated by last command - AffectedRows() uint64 // Affected rows by latest executed stmt. + Status() uint16 // Flag of current status, such as autocommit. + LastInsertID() uint64 // LastInsertID is the last inserted auto_increment ID. + LastMessage() string // LastMessage is the info message that may be generated by last command + AffectedRows() uint64 // Affected rows by latest executed stmt. + // Execute is deprecated, use ExecuteStmt() instead. Execute(context.Context, string) ([]sqlexec.RecordSet, error) // Execute a sql statement. ExecuteInternal(context.Context, string) ([]sqlexec.RecordSet, error) // Execute a internal sql statement. - String() string // String is used to debug. + ExecuteStmt(context.Context, ast.StmtNode) (sqlexec.RecordSet, error) + Parse(ctx context.Context, sql string) ([]ast.StmtNode, error) + String() string // String is used to debug. CommitTxn(context.Context) error RollbackTxn(context.Context) // PrepareStmt executes prepare statement in binary protocol. @@ -321,7 +321,11 @@ func (s *session) SetCollation(coID int) error { for _, v := range variable.SetNamesVariables { terror.Log(s.sessionVars.SetSystemVar(v, cs)) } - terror.Log(s.sessionVars.SetSystemVar(variable.CollationConnection, co)) + err = s.sessionVars.SetSystemVar(variable.CollationConnection, co) + if err != nil { + // Some clients may use the unsupported collations, such as utf8mb4_0900_ai_ci, We shouldn't return error or use the ERROR level log. + logutil.BgLogger().Warn(err.Error()) + } return nil } @@ -979,7 +983,7 @@ func (s *session) SetGlobalSysVar(name, value string) error { } var sVal string var err error - sVal, err = variable.ValidateSetSystemVar(s.sessionVars, name, value) + sVal, err = variable.ValidateSetSystemVar(s.sessionVars, name, value, variable.ScopeGlobal) if err != nil { return err } @@ -1033,7 +1037,6 @@ func (s *session) SetProcessInfo(sql string, t time.Time, command byte, maxExecu func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode ast.StmtNode, stmt sqlexec.Statement, recordSets []sqlexec.RecordSet, inMulitQuery bool) ([]sqlexec.RecordSet, error) { logStmt(stmtNode, s.sessionVars) - startTime := time.Now() recordSet, err := runStmt(ctx, s, stmt) if err != nil { if !kv.ErrKeyExists.Equal(err) { @@ -1044,11 +1047,6 @@ func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode } return nil, err } - if s.isInternal() { - sessionExecuteRunDurationInternal.Observe(time.Since(startTime).Seconds()) - } else { - sessionExecuteRunDurationGeneral.Observe(time.Since(startTime).Seconds()) - } if inMulitQuery && recordSet == nil { recordSet = &multiQueryNoDelayRecordSet{ @@ -1088,6 +1086,93 @@ func (s *session) Execute(ctx context.Context, sql string) (recordSets []sqlexec return } +// Parse parses a query string to raw ast.StmtNode. +func (s *session) Parse(ctx context.Context, sql string) ([]ast.StmtNode, error) { + charsetInfo, collation := s.sessionVars.GetCharsetInfo() + parseStartTime := time.Now() + stmts, warns, err := s.ParseSQL(ctx, sql, charsetInfo, collation) + if err != nil { + s.rollbackOnError(ctx) + + // Only print log message when this SQL is from the user. + // Mute the warning for internal SQLs. + if !s.sessionVars.InRestrictedSQL { + logutil.Logger(ctx).Warn("parse SQL failed", zap.Error(err), zap.String("SQL", sql)) + } + return nil, util.SyntaxError(err) + } + + durParse := time.Since(parseStartTime) + s.GetSessionVars().DurationParse = durParse + isInternal := s.isInternal() + if isInternal { + sessionExecuteParseDurationInternal.Observe(durParse.Seconds()) + } else { + sessionExecuteParseDurationGeneral.Observe(durParse.Seconds()) + } + for _, warn := range warns { + s.sessionVars.StmtCtx.AppendWarning(util.SyntaxWarn(warn)) + } + return stmts, nil +} + +func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlexec.RecordSet, error) { + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span1 := span.Tracer().StartSpan("session.ExecuteStmt", opentracing.ChildOf(span.Context())) + defer span1.Finish() + ctx = opentracing.ContextWithSpan(ctx, span1) + } + + s.PrepareTxnCtx(ctx) + err := s.loadCommonGlobalVariablesIfNeeded() + if err != nil { + return nil, err + } + + s.sessionVars.StartTime = time.Now() + + // Some executions are done in compile stage, so we reset them before compile. + if err := executor.ResetContextOfStmt(s, stmtNode); err != nil { + return nil, err + } + + // Transform abstract syntax tree to a physical plan(stored in executor.ExecStmt). + compiler := executor.Compiler{Ctx: s} + stmt, err := compiler.Compile(ctx, stmtNode) + if err != nil { + s.rollbackOnError(ctx) + + // Only print log message when this SQL is from the user. + // Mute the warning for internal SQLs. + if !s.sessionVars.InRestrictedSQL { + logutil.Logger(ctx).Warn("compile SQL failed", zap.Error(err), zap.String("SQL", stmtNode.Text())) + } + return nil, err + } + durCompile := time.Since(s.sessionVars.StartTime) + s.GetSessionVars().DurationCompile = durCompile + if s.isInternal() { + sessionExecuteCompileDurationInternal.Observe(durCompile.Seconds()) + } else { + sessionExecuteCompileDurationGeneral.Observe(durCompile.Seconds()) + } + s.currentPlan = stmt.Plan + + // Execute the physical plan. + logStmt(stmtNode, s.sessionVars) + recordSet, err := runStmt(ctx, s, stmt) + if err != nil { + if !kv.ErrKeyExists.Equal(err) { + logutil.Logger(ctx).Warn("run statement failed", + zap.Int64("schemaVersion", s.sessionVars.TxnCtx.SchemaVersion), + zap.Error(err), + zap.String("session", s.String())) + } + return nil, err + } + return recordSet, nil +} + func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec.RecordSet, err error) { s.PrepareTxnCtx(ctx) connID := s.sessionVars.ConnectionID @@ -1210,6 +1295,7 @@ func (s *session) CommonExec(ctx context.Context, if err != nil { return nil, err } + sessionExecuteCompileDurationGeneral.Observe(time.Since(s.sessionVars.StartTime).Seconds()) logQuery(st.OriginText(), s.sessionVars) return runStmt(ctx, s, st) } @@ -1239,7 +1325,10 @@ func (s *session) CachedPlanExec(ctx context.Context, OutputNames: execPlan.OutputNames(), PsStmt: prepareStmt, } - s.GetSessionVars().DurationCompile = time.Since(s.sessionVars.StartTime) + compileDuration := time.Since(s.sessionVars.StartTime) + sessionExecuteCompileDurationGeneral.Observe(compileDuration.Seconds()) + s.GetSessionVars().DurationCompile = compileDuration + stmt.Text = prepared.Stmt.Text() stmtCtx.OriginalSQL = stmt.Text stmtCtx.InitSQLDigest(prepareStmt.NormalizedSQL, prepareStmt.SQLDigest) @@ -1828,7 +1917,7 @@ func CreateSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er const ( notBootstrapped = 0 - currentBootstrapVersion = version41 + currentBootstrapVersion = version45 ) func getStoreBootstrapVersion(store kv.Storage) int64 { @@ -1995,18 +2084,16 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { // Use GlobalVariableCache if TiDB just loaded global variables within 2 second ago. // When a lot of connections connect to TiDB simultaneously, it can protect TiKV meta region from overload. gvc := domain.GetDomain(s).GetGlobalVarsCache() - succ, rows, fields := gvc.Get() - if !succ { - // Set the variable to true to prevent cyclic recursive call. - vars.CommonGlobalLoaded = true - rows, fields, err = s.ExecRestrictedSQL(loadCommonGlobalVarsSQL) - if err != nil { - vars.CommonGlobalLoaded = false - logutil.BgLogger().Error("failed to load common global variables.") - return err - } - gvc.Update(rows, fields) + loadFunc := func() ([]chunk.Row, []*ast.ResultField, error) { + return s.ExecRestrictedSQL(loadCommonGlobalVarsSQL) } + rows, fields, err := gvc.LoadGlobalVariables(loadFunc) + if err != nil { + logutil.BgLogger().Warn("failed to load global variables", + zap.Uint64("conn", s.sessionVars.ConnectionID), zap.Error(err)) + return err + } + vars.CommonGlobalLoaded = true for _, row := range rows { varName := row.GetString(0) @@ -2145,6 +2232,8 @@ func logQuery(query string, vars *variable.SessionVars) { zap.Stringer("user", vars.User), zap.Int64("schemaVersion", vars.TxnCtx.SchemaVersion), zap.Uint64("txnStartTS", vars.TxnCtx.StartTS), + zap.Uint64("forUpdateTS", vars.TxnCtx.GetForUpdateTS()), + zap.Bool("isReadConsistency", vars.IsReadConsistencyTxn()), zap.String("current_db", vars.CurrentDB), zap.String("txn_mode", vars.GetReadableTxnMode()), zap.String("sql", query+vars.PreparedParams.String())) diff --git a/session/session_test.go b/session/session_test.go index 76123957b6b5f..0cc56f7f9d4ad 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/table/tables" @@ -71,11 +72,10 @@ var _ = SerialSuites(&testSchemaSerialSuite{}) var _ = SerialSuites(&testSessionSerialSuite{}) type testSessionSuiteBase struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - dom *domain.Domain - pdAddr string + cluster cluster.Cluster + store kv.Storage + dom *domain.Domain + pdAddr string } type testSessionSuite struct { @@ -158,7 +158,6 @@ func initPdAddrs() { func (s *testSessionSuiteBase) SetUpSuite(c *C) { testleak.BeforeTest() - s.cluster = mocktikv.NewCluster() if *withTiKV { initPdAddrs() @@ -174,11 +173,15 @@ func (s *testSessionSuiteBase) SetUpSuite(c *C) { session.ResetStoreForWithTiKVTest(store) s.store = store } else { - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) s.store = store @@ -1914,10 +1917,9 @@ func (s *testSessionSuite2) TestInformationSchemaCreateTime(c *C) { } type testSchemaSuiteBase struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - dom *domain.Domain + cluster cluster.Cluster + store kv.Storage + dom *domain.Domain } type testSchemaSuite struct { @@ -1939,12 +1941,15 @@ func (s *testSchemaSuiteBase) TearDownTest(c *C) { func (s *testSchemaSuiteBase) SetUpSuite(c *C) { testleak.BeforeTest() - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) s.store = store @@ -2178,7 +2183,7 @@ func (s *testSchemaSuite) TestTableReaderChunk(c *C) { } tbl, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("chk")) c.Assert(err, IsNil) - s.cluster.SplitTable(s.mvccStore, tbl.Meta().ID, 10) + s.cluster.SplitTable(tbl.Meta().ID, 10) tk.Se.GetSessionVars().DistSQLScanConcurrency = 1 tk.MustExec("set tidb_init_chunk_size = 2") @@ -2363,7 +2368,7 @@ func (s *testSchemaSuite) TestIndexLookUpReaderChunk(c *C) { } tbl, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("chk")) c.Assert(err, IsNil) - s.cluster.SplitIndex(s.mvccStore, tbl.Meta().ID, tbl.Indices()[0].Meta().ID, 10) + s.cluster.SplitIndex(tbl.Meta().ID, tbl.Indices()[0].Meta().ID, 10) tk.Se.GetSessionVars().IndexLookupSize = 10 rs, err := tk.Exec("select * from chk order by k") diff --git a/session/tidb.go b/session/tidb.go index 62615e406362f..46c1582a53db0 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" @@ -266,11 +265,7 @@ func runStmt(ctx context.Context, sctx sessionctx.Context, s sqlexec.Statement) // If it is not a select statement, we record its slow log here, // then it could include the transaction commit time. if rs == nil { - // `LowSlowQuery` and `SummaryStmt` must be called before recording `PrevStmt`. - s.(*executor.ExecStmt).LogSlowQuery(origTxnCtx.StartTS, err == nil, false) - s.(*executor.ExecStmt).SummaryStmt() - pps := types.CloneRow(sessVars.PreparedParams) - sessVars.PrevStmt = executor.FormatSQL(s.OriginText(), pps) + s.(*executor.ExecStmt).FinishExecuteStmt(origTxnCtx.StartTS, err == nil, false) } }() diff --git a/session/tidb_test.go b/session/tidb_test.go index 06ead8d2166f8..7a77003d11e26 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -174,7 +174,7 @@ func match(c *C, row []types.Datum, expected ...interface{}) { } func (s *testMainSuite) TestKeysNeedLock(c *C) { - rowKey := tablecodec.EncodeRowKeyWithHandle(1, 1) + rowKey := tablecodec.EncodeRowKeyWithHandle(1, kv.IntHandle(1)) indexKey := tablecodec.EncodeIndexSeekKey(1, 1, []byte{1}) uniqueValue := make([]byte, 8) uniqueUntouched := append(uniqueValue, '1') diff --git a/session/txn.go b/session/txn.go old mode 100755 new mode 100644 index fb95dfe61929f..9f0c76a7fe1e2 --- a/session/txn.go +++ b/session/txn.go @@ -207,7 +207,7 @@ func (st *TxnState) changeToInvalid() { type dirtyTableOperation struct { kind int tid int64 - handle int64 + handle kv.Handle } var hasMockAutoIncIDRetry = int64(0) @@ -307,6 +307,14 @@ func (st *TxnState) Get(ctx context.Context, k kv.Key) ([]byte, error) { return val, nil } +// GetMemBuffer overrides the Transaction interface. +func (st *TxnState) GetMemBuffer() kv.MemBuffer { + if st.stmtBuf == nil || st.stmtBuf.Size() == 0 { + return st.Transaction.GetMemBuffer() + } + return kv.NewBufferStoreFrom(st.Transaction.GetMemBuffer(), st.stmtBuf) +} + // BatchGet overrides the Transaction interface. func (st *TxnState) BatchGet(ctx context.Context, keys []kv.Key) (map[string][]byte, error) { bufferValues := make([][]byte, len(keys)) @@ -591,6 +599,6 @@ func (s *session) StmtGetMutation(tableID int64) *binlog.TableMutation { return st.mutations[tableID] } -func (s *session) StmtAddDirtyTableOP(op int, tid int64, handle int64) { +func (s *session) StmtAddDirtyTableOP(op int, tid int64, handle kv.Handle) { s.txn.dirtyTableOP = append(s.txn.dirtyTableOP, dirtyTableOperation{op, tid, handle}) } diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 608abe4ecdc3c..8f745a1a99570 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -314,6 +314,7 @@ func AddSpecialComment(ddlQuery string) string { } // addSpecialCommentByRegexps uses to add special comment for the worlds in the ddlQuery with match the regexps. +// addSpecialCommentByRegexps will merge multi pattern regs to one special comment. func addSpecialCommentByRegexps(ddlQuery string, prefix string, regs ...*regexp.Regexp) string { upperQuery := strings.ToUpper(ddlQuery) var specialComments []string diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 9ed7bb0c7d04a..001009c378949 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -559,6 +559,34 @@ func (s *testBinlogSuite) TestAddSpecialComment(c *C) { "create table t1 (id int auto_random ( 4 ) primary key);", "create table t1 (id int /*T![auto_rand] auto_random ( 4 ) */ primary key);", }, + { + "create table t1 (id int auto_random ( 3 ) primary key) auto_random_base = 100;", + "create table t1 (id int /*T![auto_rand] auto_random ( 3 ) */ primary key) /*T![auto_rand_base] auto_random_base = 100 */ ;", + }, + { + "create table t1 (id int auto_random primary key) auto_random_base = 50;", + "create table t1 (id int /*T![auto_rand] auto_random */ primary key) /*T![auto_rand_base] auto_random_base = 50 */ ;", + }, + { + "create table t1 (id int auto_increment key) auto_id_cache 100;", + "create table t1 (id int auto_increment key) /*T![auto_id_cache] auto_id_cache 100 */ ;", + }, + { + "create table t1 (id int auto_increment unique) auto_id_cache 10;", + "create table t1 (id int auto_increment unique) /*T![auto_id_cache] auto_id_cache 10 */ ;", + }, + { + "create table t1 (id int) auto_id_cache = 5;", + "create table t1 (id int) /*T![auto_id_cache] auto_id_cache = 5 */ ;", + }, + { + "create table t1 (id int) auto_id_cache=5;", + "create table t1 (id int) /*T![auto_id_cache] auto_id_cache=5 */ ;", + }, + { + "create table t1 (id int) /*T![auto_id_cache] auto_id_cache=5 */ ;", + "create table t1 (id int) /*T![auto_id_cache] auto_id_cache=5 */ ;", + }, } for _, ca := range testCase { re := binloginfo.AddSpecialComment(ca.input) diff --git a/sessionctx/context.go b/sessionctx/context.go index 167fc3bd79803..4112d0e9046aa 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -84,7 +84,7 @@ type Context interface { // StmtGetMutation gets the binlog mutation for current statement. StmtGetMutation(int64) *binlog.TableMutation // StmtAddDirtyTableOP adds the dirty table operation for current statement. - StmtAddDirtyTableOP(op int, physicalID int64, handle int64) + StmtAddDirtyTableOP(op int, physicalID int64, handle kv.Handle) // DDLOwnerChecker returns owner.DDLOwnerChecker. DDLOwnerChecker() owner.DDLOwnerChecker // AddTableLock adds table lock to the session lock map. diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index fc9f2aac61a5b..98879a6401d17 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -458,6 +458,7 @@ func (sc *StatementContext) ResetForRetry() { func (sc *StatementContext) MergeExecDetails(details *execdetails.ExecDetails, commitDetails *execdetails.CommitDetails) { sc.mu.Lock() if details != nil { + sc.mu.execDetails.CopTime += details.CopTime sc.mu.execDetails.ProcessTime += details.ProcessTime sc.mu.execDetails.WaitTime += details.WaitTime sc.mu.execDetails.BackoffTime += details.BackoffTime diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 90c01484a26fd..4ece801ddcfe0 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -596,6 +596,11 @@ type SessionVars struct { // WindowingUseHighPrecision determines whether to compute window operations without loss of precision. // see https://dev.mysql.com/doc/refman/8.0/en/window-function-optimization.html for more details. WindowingUseHighPrecision bool + + // FoundInPlanCache indicates whether this statement was found in plan cache. + FoundInPlanCache bool + // PrevFoundInPlanCache indicates whether the last statement was found in plan cache. + PrevFoundInPlanCache bool } // PreparedParams contains the parameters of the current prepared statement when executing it. @@ -682,6 +687,8 @@ func NewSessionVars() *SessionVars { MetricSchemaRangeDuration: DefTiDBMetricSchemaRangeDuration, SequenceState: NewSequenceState(), WindowingUseHighPrecision: true, + PrevFoundInPlanCache: DefTiDBFoundInPlanCache, + FoundInPlanCache: DefTiDBFoundInPlanCache, } vars.KVVars = kv.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ @@ -1044,7 +1051,7 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.MaxExecutionTime = uint64(timeoutMS) case InnodbLockWaitTimeout: lockWaitSec := tidbOptInt64(val, DefInnodbLockWaitTimeout) - s.LockWaitTimeout = int64(lockWaitSec * 1000) + s.LockWaitTimeout = lockWaitSec * 1000 case WindowingUseHighPrecision: s.WindowingUseHighPrecision = TiDBOptOn(val) case TiDBSkipUTF8Check: @@ -1243,40 +1250,19 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { return errors.Trace(err) } case TiDBSlowLogThreshold: - conf := config.GetGlobalConfig() - if !conf.EnableDynamicConfig { - atomic.StoreUint64(&config.GetGlobalConfig().Log.SlowThreshold, uint64(tidbOptInt64(val, logutil.DefaultSlowThreshold))) - } else { - s.StmtCtx.AppendWarning(errors.Errorf("cannot update %s when enabling dynamic configs", TiDBSlowLogThreshold)) - } + atomic.StoreUint64(&config.GetGlobalConfig().Log.SlowThreshold, uint64(tidbOptInt64(val, logutil.DefaultSlowThreshold))) case TiDBRecordPlanInSlowLog: - conf := config.GetGlobalConfig() - if !conf.EnableDynamicConfig { - atomic.StoreUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog, uint32(tidbOptInt64(val, logutil.DefaultRecordPlanInSlowLog))) - } else { - s.StmtCtx.AppendWarning(errors.Errorf("cannot update %s when enabling dynamic configs", TiDBRecordPlanInSlowLog)) - } + atomic.StoreUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog, uint32(tidbOptInt64(val, logutil.DefaultRecordPlanInSlowLog))) case TiDBEnableSlowLog: - conf := config.GetGlobalConfig() - if !conf.EnableDynamicConfig { - config.GetGlobalConfig().Log.EnableSlowLog = TiDBOptOn(val) - } else { - s.StmtCtx.AppendWarning(errors.Errorf("cannot update %s when enabling dynamic configs", TiDBEnableSlowLog)) - } + config.GetGlobalConfig().Log.EnableSlowLog = TiDBOptOn(val) case TiDBQueryLogMaxLen: - conf := config.GetGlobalConfig() - if !conf.EnableDynamicConfig { - atomic.StoreUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen, uint64(tidbOptInt64(val, logutil.DefaultQueryLogMaxLen))) - } else { - s.StmtCtx.AppendWarning(errors.Errorf("cannot update %s when enabling dynamic configs", TiDBQueryLogMaxLen)) - } + atomic.StoreUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen, uint64(tidbOptInt64(val, logutil.DefaultQueryLogMaxLen))) case TiDBCheckMb4ValueInUTF8: - conf := config.GetGlobalConfig() - if !conf.EnableDynamicConfig { - config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) - } else { - s.StmtCtx.AppendWarning(errors.Errorf("cannot update %s when enabling dynamic configs", TiDBCheckMb4ValueInUTF8)) - } + config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) + case TiDBFoundInPlanCache: + s.FoundInPlanCache = TiDBOptOn(val) + case TiDBEnableCollectExecutionInfo: + config.GetGlobalConfig().EnableCollectExecutionInfo = TiDBOptOn(val) } s.systems[name] = val return nil diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 4c64bd681bcce..bf9bf549f84d1 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -611,7 +611,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBSnapshot, ""}, {ScopeSession, TiDBOptAggPushDown, BoolToIntStr(DefOptAggPushDown)}, {ScopeGlobal | ScopeSession, TiDBOptBCJ, BoolToIntStr(DefOptBCJ)}, - {ScopeSession, TiDBOptDistinctAggPushDown, BoolToIntStr(DefOptDistinctAggPushDown)}, + {ScopeSession, TiDBOptDistinctAggPushDown, BoolToIntStr(config.GetGlobalConfig().Performance.DistinctAggPushDown)}, {ScopeSession, TiDBOptWriteRowID, BoolToIntStr(DefOptWriteRowID)}, {ScopeGlobal | ScopeSession, TiDBBuildStatsConcurrency, strconv.Itoa(DefBuildStatsConcurrency)}, {ScopeGlobal, TiDBAutoAnalyzeRatio, strconv.FormatFloat(DefAutoAnalyzeRatio, 'f', -1, 64)}, @@ -658,7 +658,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBEnableStreaming, "0"}, {ScopeSession, TiDBEnableChunkRPC, "1"}, {ScopeSession, TxnIsolationOneShot, ""}, - {ScopeGlobal | ScopeSession, TiDBEnableTablePartition, "auto"}, + {ScopeGlobal | ScopeSession, TiDBEnableTablePartition, "on"}, {ScopeGlobal | ScopeSession, TiDBHashJoinConcurrency, strconv.Itoa(DefTiDBHashJoinConcurrency)}, {ScopeGlobal | ScopeSession, TiDBProjectionConcurrency, strconv.Itoa(DefTiDBProjectionConcurrency)}, {ScopeGlobal | ScopeSession, TiDBHashAggPartialConcurrency, strconv.Itoa(DefTiDBHashAggPartialConcurrency)}, @@ -698,12 +698,12 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal | ScopeSession, TiDBEnableNoopFuncs, BoolToIntStr(DefTiDBEnableNoopFuncs)}, {ScopeSession, TiDBReplicaRead, "leader"}, {ScopeSession, TiDBAllowRemoveAutoInc, BoolToIntStr(DefTiDBAllowRemoveAutoInc)}, - {ScopeGlobal | ScopeSession, TiDBEnableStmtSummary, ""}, - {ScopeGlobal | ScopeSession, TiDBStmtSummaryInternalQuery, ""}, - {ScopeGlobal | ScopeSession, TiDBStmtSummaryRefreshInterval, ""}, - {ScopeGlobal | ScopeSession, TiDBStmtSummaryHistorySize, ""}, - {ScopeGlobal | ScopeSession, TiDBStmtSummaryMaxStmtCount, ""}, - {ScopeGlobal | ScopeSession, TiDBStmtSummaryMaxSQLLength, ""}, + {ScopeGlobal | ScopeSession, TiDBEnableStmtSummary, BoolToIntStr(config.GetGlobalConfig().StmtSummary.Enable)}, + {ScopeGlobal | ScopeSession, TiDBStmtSummaryInternalQuery, BoolToIntStr(config.GetGlobalConfig().StmtSummary.EnableInternalQuery)}, + {ScopeGlobal | ScopeSession, TiDBStmtSummaryRefreshInterval, strconv.Itoa(config.GetGlobalConfig().StmtSummary.RefreshInterval)}, + {ScopeGlobal | ScopeSession, TiDBStmtSummaryHistorySize, strconv.Itoa(config.GetGlobalConfig().StmtSummary.HistorySize)}, + {ScopeGlobal | ScopeSession, TiDBStmtSummaryMaxStmtCount, strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxStmtCount), 10)}, + {ScopeGlobal | ScopeSession, TiDBStmtSummaryMaxSQLLength, strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxSQLLength), 10)}, {ScopeGlobal | ScopeSession, TiDBCapturePlanBaseline, "off"}, {ScopeGlobal | ScopeSession, TiDBUsePlanBaselines, boolToOnOff(DefTiDBUsePlanBaselines)}, {ScopeGlobal | ScopeSession, TiDBEvolvePlanBaselines, boolToOnOff(DefTiDBEvolvePlanBaselines)}, @@ -719,6 +719,8 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBEnableSlowLog, BoolToIntStr(logutil.DefaultTiDBEnableSlowLog)}, {ScopeSession, TiDBQueryLogMaxLen, strconv.Itoa(logutil.DefaultQueryLogMaxLen)}, {ScopeSession, TiDBCheckMb4ValueInUTF8, BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8)}, + {ScopeSession, TiDBFoundInPlanCache, BoolToIntStr(DefTiDBFoundInPlanCache)}, + {ScopeSession, TiDBEnableCollectExecutionInfo, BoolToIntStr(logutil.DefaultTiDBEnableSlowLog)}, } // SynonymsSysVariables is synonyms of system variables. diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index 7a7ed4ae5a3bd..eaf309db70f52 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -51,6 +51,9 @@ func (*testSysVarSuite) TestSysVar(c *C) { f = GetSysVar("tidb_replica_read") c.Assert(f.Value, Equals, "leader") + + f = GetSysVar("tidb_enable_table_partition") + c.Assert(f.Value, Equals, "on") } func (*testSysVarSuite) TestTxnMode(c *C) { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index a8b6bb2335149..1b4a67f563b4c 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -130,8 +130,7 @@ const ( // tidb_enable_table_partition is used to control table partition feature. // The valid value include auto/on/off: - // auto: enable table partition when that feature is implemented. - // on: always enable table partition. + // on or auto: enable table partition if the partition type is implemented. // off: always disable table partition. TiDBEnableTablePartition = "tidb_enable_table_partition" @@ -170,6 +169,9 @@ const ( // TiDBCheckMb4ValueInUTF8 is used to control whether to enable the check wrong utf8 value. TiDBCheckMb4ValueInUTF8 = "tidb_check_mb4_value_in_utf8" + + // TiDBFoundInPlanCache indicates whether the last statement was found in plan cache + TiDBFoundInPlanCache = "last_plan_from_cache" ) // TiDB system variable names that both in session and global scope. @@ -244,8 +246,9 @@ const ( // TiDBMaxChunkSize is used to control the max chunk size during query execution. TiDBMaxChunkSize = "tidb_max_chunk_size" - // TiDBAllowBatchCop means if we should send batch coprocessor to TiFlash. Default value is 1, means to use batch cop in case of aggregation and join. - // If value is set to 2 , which means to force to send batch cop for any query. Value is set to 0 means never use batch cop. + // TiDBAllowBatchCop means if we should send batch coprocessor to TiFlash. It can be set to 0, 1 and 2. + // 0 means never use batch cop, 1 means use batch cop in case of aggregation and join, 2, means to force to send batch cop for any query. + // The default value is 0 TiDBAllowBatchCop = "tidb_allow_batch_cop" // TiDBInitChunkSize is used to control the init chunk size during query execution. @@ -388,6 +391,9 @@ const ( // TiDBMetricSchemaRangeDuration indicates the range duration when query metric schema. TiDBMetricSchemaRangeDuration = "tidb_metric_query_range_duration" + + // TiDBEnableCollectExecutionInfo indicates that whether execution info is collected. + TiDBEnableCollectExecutionInfo = "tidb_enable_collect_execution_info" ) // Default TiDB system variable values. @@ -409,7 +415,6 @@ const ( DefSkipUTF8Check = false DefOptAggPushDown = false DefOptBCJ = false - DefOptDistinctAggPushDown = false DefOptWriteRowID = false DefOptCorrelationThreshold = 0.9 DefOptCorrelationExpFactor = 1 @@ -482,6 +487,8 @@ const ( DefTiDBStoreLimit = 0 DefTiDBMetricSchemaStep = 60 // 60s DefTiDBMetricSchemaRangeDuration = 60 // 60s + DefTiDBFoundInPlanCache = false + DefTidbEnableCollectExecutionInfo = false ) // Process global variables. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 6f53bc490c27a..4a690486a9777 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -152,6 +152,10 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { return BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8), true, nil case TiDBCapturePlanBaseline: return CapturePlanBaseline.GetVal(), true, nil + case TiDBFoundInPlanCache: + return BoolToIntStr(s.PrevFoundInPlanCache), true, nil + case TiDBEnableCollectExecutionInfo: + return BoolToIntStr(config.GetGlobalConfig().EnableCollectExecutionInfo), true, nil } sVal, ok := s.GetSystemVar(key) if ok { @@ -209,7 +213,7 @@ func SetSessionSystemVar(vars *SessionVars, name string, value types.Datum) erro if err != nil { return err } - sVal, err = ValidateSetSystemVar(vars, name, sVal) + sVal, err = ValidateSetSystemVar(vars, name, sVal, ScopeSession) if err != nil { return err } @@ -286,7 +290,7 @@ const ( ) // ValidateSetSystemVar checks if system variable satisfies specific restriction. -func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, error) { +func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope ScopeFlag) (string, error) { if strings.EqualFold(value, "DEFAULT") { if val := GetSysVar(name); val != nil { return val.Value, nil @@ -417,7 +421,8 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction, TiDBPProfSQLCPU, TiDBLowResolutionTSO, TiDBEnableIndexMerge, TiDBEnableNoopFuncs, TiDBCheckMb4ValueInUTF8, TiDBEnableSlowLog, TiDBRecordPlanInSlowLog, - TiDBScatterRegion, TiDBGeneralLog, TiDBConstraintCheckInPlace, TiDBEnableVectorizedExpression: + TiDBScatterRegion, TiDBGeneralLog, TiDBConstraintCheckInPlace, + TiDBEnableVectorizedExpression, TiDBFoundInPlanCache, TiDBEnableCollectExecutionInfo: fallthrough case GeneralLog, AvoidTemporalUpgrade, BigTables, CheckProxyUsers, LogBin, CoreFile, EndMakersInJSON, SQLLogBin, OfflineMode, PseudoSlaveMode, LowPriorityUpdates, @@ -682,26 +687,28 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, case strings.EqualFold(value, "OFF") || value == "0": return "0", nil case value == "": - return "", nil + if scope == ScopeSession { + return "", nil + } } return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) case TiDBStmtSummaryRefreshInterval: - if value == "" { + if value == "" && scope == ScopeSession { return "", nil } return checkUInt64SystemVar(name, value, 1, math.MaxInt32, vars) case TiDBStmtSummaryHistorySize: - if value == "" { + if value == "" && scope == ScopeSession { return "", nil } return checkUInt64SystemVar(name, value, 0, math.MaxUint8, vars) case TiDBStmtSummaryMaxStmtCount: - if value == "" { + if value == "" && scope == ScopeSession { return "", nil } return checkInt64SystemVar(name, value, 1, math.MaxInt16, vars) case TiDBStmtSummaryMaxSQLLength: - if value == "" { + if value == "" && scope == ScopeSession { return "", nil } return checkInt64SystemVar(name, value, 0, math.MaxInt32, vars) diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 6ecd775febbe6..22dfb3d69a1ab 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -84,6 +84,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.AllowWriteRowID, Equals, DefOptWriteRowID) c.Assert(vars.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) c.Assert(vars.EnableFastAnalyze, Equals, DefTiDBUseFastAnalyze) + c.Assert(vars.FoundInPlanCache, Equals, DefTiDBFoundInPlanCache) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.Concurrency)) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.MemQuota)) @@ -434,6 +435,13 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(val, Equals, "0") err = SetSessionSystemVar(v, TiDBStmtSummaryMaxSQLLength, types.NewStringDatum("a")) c.Assert(err, ErrorMatches, ".*Incorrect argument type to variable 'tidb_stmt_summary_max_sql_length'") + + err = SetSessionSystemVar(v, TiDBFoundInPlanCache, types.NewStringDatum("1")) + c.Assert(err, IsNil) + val, err = GetSessionSystemVar(v, TiDBFoundInPlanCache) + c.Assert(err, IsNil) + c.Assert(val, Equals, "0") + c.Assert(v.systems[TiDBFoundInPlanCache], Equals, "1") } func (s *testVarsutilSuite) TestSetOverflowBehave(c *C) { @@ -529,21 +537,10 @@ func (s *testVarsutilSuite) TestValidate(c *C) { {TiDBIsolationReadEngines, "tikv", false}, {TiDBIsolationReadEngines, "TiKV,tiflash", false}, {TiDBIsolationReadEngines, " tikv, tiflash ", false}, - {TiDBEnableStmtSummary, "a", true}, - {TiDBEnableStmtSummary, "-1", true}, - {TiDBEnableStmtSummary, "", false}, - {TiDBStmtSummaryRefreshInterval, "a", true}, - {TiDBStmtSummaryRefreshInterval, "", false}, - {TiDBStmtSummaryHistorySize, "a", true}, - {TiDBStmtSummaryHistorySize, "", false}, - {TiDBStmtSummaryMaxStmtCount, "a", true}, - {TiDBStmtSummaryMaxStmtCount, "", false}, - {TiDBStmtSummaryMaxSQLLength, "a", true}, - {TiDBStmtSummaryMaxSQLLength, "", false}, } for _, t := range tests { - _, err := ValidateSetSystemVar(v, t.key, t.value) + _, err := ValidateSetSystemVar(v, t.key, t.value, ScopeGlobal) if t.error { c.Assert(err, NotNil, Commentf("%v got err=%v", t, err)) } else { @@ -552,3 +549,46 @@ func (s *testVarsutilSuite) TestValidate(c *C) { } } + +func (s *testVarsutilSuite) TestValidateStmtSummary(c *C) { + v := NewSessionVars() + v.GlobalVarsAccessor = NewMockGlobalAccessor() + v.TimeZone = time.UTC + + tests := []struct { + key string + value string + error bool + scope ScopeFlag + }{ + {TiDBEnableStmtSummary, "a", true, ScopeSession}, + {TiDBEnableStmtSummary, "-1", true, ScopeSession}, + {TiDBEnableStmtSummary, "", false, ScopeSession}, + {TiDBEnableStmtSummary, "", true, ScopeGlobal}, + {TiDBStmtSummaryInternalQuery, "a", true, ScopeSession}, + {TiDBStmtSummaryInternalQuery, "-1", true, ScopeSession}, + {TiDBStmtSummaryInternalQuery, "", false, ScopeSession}, + {TiDBStmtSummaryInternalQuery, "", true, ScopeGlobal}, + {TiDBStmtSummaryRefreshInterval, "a", true, ScopeSession}, + {TiDBStmtSummaryRefreshInterval, "", false, ScopeSession}, + {TiDBStmtSummaryRefreshInterval, "", true, ScopeGlobal}, + {TiDBStmtSummaryHistorySize, "a", true, ScopeSession}, + {TiDBStmtSummaryHistorySize, "", false, ScopeSession}, + {TiDBStmtSummaryHistorySize, "", true, ScopeGlobal}, + {TiDBStmtSummaryMaxStmtCount, "a", true, ScopeSession}, + {TiDBStmtSummaryMaxStmtCount, "", false, ScopeSession}, + {TiDBStmtSummaryMaxStmtCount, "", true, ScopeGlobal}, + {TiDBStmtSummaryMaxSQLLength, "a", true, ScopeSession}, + {TiDBStmtSummaryMaxSQLLength, "", false, ScopeSession}, + {TiDBStmtSummaryMaxSQLLength, "", true, ScopeGlobal}, + } + + for _, t := range tests { + _, err := ValidateSetSystemVar(v, t.key, t.value, t.scope) + if t.error { + c.Assert(err, NotNil, Commentf("%v got err=%v", t, err)) + } else { + c.Assert(err, IsNil, Commentf("%v got err=%v", t, err)) + } + } +} diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 33f0dc6860b98..ca2c09320fbd9 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -52,6 +52,11 @@ type TopNMeta struct { Count uint64 } +// GetH2 get the the second part of `murmur3.Sum128()`, just for test. +func (t *TopNMeta) GetH2() uint64 { + return t.h2 +} + // NewCMSketch returns a new CM sketch. func NewCMSketch(d, w int32) *CMSketch { tbl := make([][]uint32, d) @@ -150,11 +155,11 @@ func buildCMSWithTopN(helper *topNHelper, d, w int32, scaleRatio uint64, default func calculateDefaultVal(helper *topNHelper, estimateNDV, scaleRatio, rowCount uint64) uint64 { sampleNDV := uint64(len(helper.sorted)) - if rowCount <= (helper.sampleSize-uint64(helper.onlyOnceItems))*scaleRatio { + if rowCount <= (helper.sampleSize-helper.onlyOnceItems)*scaleRatio { return 1 } - estimateRemainingCount := rowCount - (helper.sampleSize-uint64(helper.onlyOnceItems))*scaleRatio - return estimateRemainingCount / mathutil.MaxUint64(1, estimateNDV-uint64(sampleNDV)+helper.onlyOnceItems) + estimateRemainingCount := rowCount - (helper.sampleSize-helper.onlyOnceItems)*scaleRatio + return estimateRemainingCount / mathutil.MaxUint64(1, estimateNDV-sampleNDV+helper.onlyOnceItems) } func (c *CMSketch) findTopNMeta(h1, h2 uint64, d []byte) *TopNMeta { @@ -180,7 +185,8 @@ func (c *CMSketch) updateTopNWithDelta(h1, h2 uint64, d []byte, delta uint64) bo return false } -func (c *CMSketch) queryTopN(h1, h2 uint64, d []byte) (uint64, bool) { +// QueryTopN returns the results for (h1, h2) in murmur3.Sum128(), if not exists, return (0, false). +func (c *CMSketch) QueryTopN(h1, h2 uint64, d []byte) (uint64, bool) { if c.topN == nil { return 0, false } @@ -216,7 +222,7 @@ func (c *CMSketch) considerDefVal(cnt uint64) bool { // updateValueBytes updates value of d to count. func (c *CMSketch) updateValueBytes(d []byte, count uint64) { h1, h2 := murmur3.Sum128(d) - if oriCount, ok := c.queryTopN(h1, h2, d); ok { + if oriCount, ok := c.QueryTopN(h1, h2, d); ok { deltaCount := count - oriCount c.updateTopNWithDelta(h1, h2, d, deltaCount) } @@ -264,7 +270,7 @@ func (c *CMSketch) queryValue(sc *stmtctx.StatementContext, val types.Datum) (ui // QueryBytes is used to query the count of specified bytes. func (c *CMSketch) QueryBytes(d []byte) uint64 { h1, h2 := murmur3.Sum128(d) - if count, ok := c.queryTopN(h1, h2, d); ok { + if count, ok := c.QueryTopN(h1, h2, d); ok { return count } return c.queryHashValue(h1, h2) @@ -505,6 +511,11 @@ func (c *CMSketch) TopN() []*TopNMeta { return topN } +// TopNMap gets the origin topN map. +func (c *CMSketch) TopNMap() map[uint64][]*TopNMeta { + return c.topN +} + // AppendTopN appends a topn into the cm sketch. func (c *CMSketch) AppendTopN(data []byte, count uint64) { if c.topN == nil { diff --git a/statistics/cmsketch_test.go b/statistics/cmsketch_test.go index 5d1ad66ba6852..b90f91873dd16 100644 --- a/statistics/cmsketch_test.go +++ b/statistics/cmsketch_test.go @@ -100,7 +100,7 @@ func averageAbsoluteError(cms *CMSketch, mp map[int64]uint32) (uint64, error) { } else { diff = estimate - uint64(count) } - total += uint64(diff) + total += diff } return total / uint64(len(mp)), nil } diff --git a/statistics/sample.go b/statistics/sample.go index 0f69a4c3644e2..5e94227606fd4 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -16,12 +16,14 @@ package statistics import ( "context" "sort" + "time" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/fastrand" @@ -262,9 +264,9 @@ func RowToDatums(row chunk.Row, fields []*ast.ResultField) []types.Datum { } // ExtractTopN extracts the topn from the CM Sketch. -func (c *SampleCollector) ExtractTopN(numTop uint32) { +func (c *SampleCollector) ExtractTopN(numTop uint32, sc *stmtctx.StatementContext, tp *types.FieldType, timeZone *time.Location) error { if numTop == 0 { - return + return nil } values := make([][]byte, 0, len(c.Samples)) for _, sample := range c.Samples { @@ -276,10 +278,20 @@ func (c *SampleCollector) ExtractTopN(numTop uint32) { // Process them decreasingly so we can handle most frequent values first and reduce the probability of hash collision // by small values. for i := uint32(0); i < helper.actualNumTop; i++ { - data := helper.sorted[i].data - h1, h2 := murmur3.Sum128(data) + h1, h2 := murmur3.Sum128(helper.sorted[i].data) realCnt := cms.queryHashValue(h1, h2) + // Because the encode of topn is the new encode type. But analyze proto returns the old encode type for a sample datum, + // we should decode it and re-encode it to get the correct bytes. + d, err := tablecodec.DecodeColumnValue(helper.sorted[i].data, tp, timeZone) + if err != nil { + return err + } + data, err := tablecodec.EncodeValue(sc, nil, d) + if err != nil { + return err + } cms.subValue(h1, h2, realCnt) cms.topN[h1] = append(cms.topN[h1], &TopNMeta{h2, data, realCnt}) } + return nil } diff --git a/statistics/sample_test.go b/statistics/sample_test.go index cf07ec799a957..8809e3b51ac5a 100644 --- a/statistics/sample_test.go +++ b/statistics/sample_test.go @@ -72,7 +72,7 @@ func (s *testSampleSuite) TestCollectColumnStats(c *C) { c.Assert(collectors[0].NullCount+collectors[0].Count, Equals, int64(s.count)) c.Assert(collectors[0].FMSketch.NDV(), Equals, int64(6232)) c.Assert(collectors[0].CMSketch.TotalCount(), Equals, uint64(collectors[0].Count)) - c.Assert(int64(pkBuilder.Count), Equals, int64(s.count)) + c.Assert(pkBuilder.Count, Equals, int64(s.count)) c.Assert(pkBuilder.Hist().NDV, Equals, int64(s.count)) } diff --git a/statistics/scalar_test.go b/statistics/scalar_test.go index 1dc12b4f695cc..4b5b5a7c52ea3 100644 --- a/statistics/scalar_test.go +++ b/statistics/scalar_test.go @@ -35,12 +35,12 @@ func getDuration(value string) types.Duration { } func getTime(year, month, day int, timeType byte) types.Time { - ret := types.NewTime(types.FromDate(year, int(month), day, 0, 0, 0, 0), timeType, types.DefaultFsp) + ret := types.NewTime(types.FromDate(year, month, day, 0, 0, 0, 0), timeType, types.DefaultFsp) return ret } func getTimeStamp(hour, min, sec int, timeType byte) types.Time { - ret := types.NewTime(types.FromDate(2017, int(1), 1, hour, min, sec, 0), timeType, 0) + ret := types.NewTime(types.FromDate(2017, 1, 1, hour, min, sec, 0), timeType, 0) return ret } diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index 009ae8064da92..41c943d7b2ec5 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -287,7 +287,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { checkRepeats(c, col) c.Assert(col.Len(), Equals, 250) - tblCount, col, _, err := buildIndex(ctx, bucketCount, 1, sqlexec.RecordSet(s.rc)) + tblCount, col, _, err := buildIndex(ctx, bucketCount, 1, s.rc) c.Check(err, IsNil) checkRepeats(c, col) col.PreCalculateScalar() @@ -304,7 +304,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { c.Check(int(count), Equals, 0) s.pk.(*recordSet).cursor = 0 - tblCount, col, err = buildPK(ctx, bucketCount, 4, sqlexec.RecordSet(s.pk)) + tblCount, col, err = buildPK(ctx, bucketCount, 4, s.pk) c.Check(err, IsNil) checkRepeats(c, col) col.PreCalculateScalar() @@ -338,7 +338,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { func (s *testStatisticsSuite) TestHistogramProtoConversion(c *C) { ctx := mock.NewContext() c.Assert(s.rc.Close(), IsNil) - tblCount, col, _, err := buildIndex(ctx, 256, 1, sqlexec.RecordSet(s.rc)) + tblCount, col, _, err := buildIndex(ctx, 256, 1, s.rc) c.Check(err, IsNil) c.Check(int(tblCount), Equals, 100000) diff --git a/store/helper/helper.go b/store/helper/helper.go index 8033ee4421091..22c944b8b4703 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -309,7 +309,11 @@ func NewFrameItemFromRegionKey(key []byte) (frame *FrameItem, err error) { frame.TableID, frame.IndexID, frame.IsRecord, err = tablecodec.DecodeKeyHead(key) if err == nil { if frame.IsRecord { - _, frame.RecordID, err = tablecodec.DecodeRecordKey(key) + var handle kv.Handle + _, handle, err = tablecodec.DecodeRecordKey(key) + if err == nil { + frame.RecordID = handle.IntValue() + } } else { _, _, frame.IndexValues, err = tablecodec.DecodeIndexKey(key) } @@ -429,6 +433,8 @@ type RegionInfo struct { ReadBytes int64 `json:"read_bytes"` ApproximateSize int64 `json:"approximate_size"` ApproximateKeys int64 `json:"approximate_keys"` + + ReplicationStatus *ReplicationStatus `json:"replication_status,omitempty"` } // RegionsInfo stores the information of regions. @@ -437,6 +443,12 @@ type RegionsInfo struct { Regions []RegionInfo `json:"regions"` } +// ReplicationStatus represents the replication mode status of the region. +type ReplicationStatus struct { + State string `json:"state"` + StateID int64 `json:"state_id"` +} + // TableInfo stores the information of a table or an index type TableInfo struct { DB *model.DBInfo diff --git a/store/mockstore/cluster/cluster.go b/store/mockstore/cluster/cluster.go new file mode 100644 index 0000000000000..ee7f2fd7ac83e --- /dev/null +++ b/store/mockstore/cluster/cluster.go @@ -0,0 +1,49 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cluster + +import ( + "time" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/kv" +) + +// Cluster simulates a TiKV cluster. +// It can be used to change cluster states in tests. +type Cluster interface { + // AllocID creates an unique ID in cluster. The ID could be used as either + // StoreID, RegionID, or PeerID. + AllocID() uint64 + // GetRegionByKey returns the Region and its leader whose range contains the key. + GetRegionByKey(key []byte) (*metapb.Region, *metapb.Peer) + // GetAllStores returns all Stores' meta. + GetAllStores() []*metapb.Store + // ScheduleDelay schedules a delay event for a transaction on a region. + ScheduleDelay(startTS, regionID uint64, dur time.Duration) + // SetMvccStore sets the the mvccStore used by SplitTable, SplitIndex and SplitKeys. + Split(regionID, newRegionID uint64, key []byte, peerIDs []uint64, leaderPeerID uint64) + // SplitRaw splits a Region at the key (not encoded) and creates new Region. + SplitRaw(regionID, newRegionID uint64, rawKey []byte, peerIDs []uint64, leaderPeerID uint64) *metapb.Region + // SplitTable evenly splits the data in table into count regions. + SplitTable(tableID int64, count int) + // SplitIndex evenly splits the data in index into count regions. + SplitIndex(tableID, indexID int64, count int) + // SplitKeys evenly splits the start, end key into "count" regions. + SplitKeys(start, end kv.Key, count int) + // AddStore adds a new Store to the cluster. + AddStore(storeID uint64, addr string) + // RemoveStore removes a Store from the cluster. + RemoveStore(storeID uint64) +} diff --git a/store/mockstore/mocktikv/analyze.go b/store/mockstore/mocktikv/analyze.go index 5c00500dc98f3..ccda51e2bda4c 100644 --- a/store/mockstore/mocktikv/analyze.go +++ b/store/mockstore/mocktikv/analyze.go @@ -163,7 +163,7 @@ func (h *rpcHandler) handleAnalyzeColumnsReq(req *coprocessor.Request, analyzeRe } return col.DefaultVal, nil } - rd := rowcodec.NewByteDecoder(colInfos, -1, defVal, nil) + rd := rowcodec.NewByteDecoder(colInfos, []int64{-1}, defVal, nil) e := &analyzeColumnsExec{ tblExec: &tableScanExec{ TableScan: &tipb.TableScan{Columns: columns}, diff --git a/store/mockstore/mocktikv/cluster.go b/store/mockstore/mocktikv/cluster.go index 1f53e577ae96f..4664941eee1ca 100644 --- a/store/mockstore/mocktikv/cluster.go +++ b/store/mockstore/mocktikv/cluster.go @@ -45,6 +45,8 @@ type Cluster struct { stores map[uint64]*Store regions map[uint64]*Region + mvccStore MVCCStore + // delayEvents is used to control the execution sequence of rpc requests for test. delayEvents map[delayKey]time.Duration delayMu sync.Mutex @@ -65,6 +67,11 @@ func NewCluster() *Cluster { } } +// SetMvccStore sets the the mvccStore used by SplitTable, SplitIndex and SplitKeys. +func (c *Cluster) SetMvccStore(s MVCCStore) { + c.mvccStore = s +} + // AllocID creates an unique ID in cluster. The ID could be used as either // StoreID, RegionID, or PeerID. func (c *Cluster) AllocID() uint64 { @@ -393,24 +400,24 @@ func (c *Cluster) Merge(regionID1, regionID2 uint64) { // SplitTable evenly splits the data in table into count regions. // Only works for single store. -func (c *Cluster) SplitTable(mvccStore MVCCStore, tableID int64, count int) { +func (c *Cluster) SplitTable(tableID int64, count int) { tableStart := tablecodec.GenTableRecordPrefix(tableID) tableEnd := tableStart.PrefixNext() - c.splitRange(mvccStore, NewMvccKey(tableStart), NewMvccKey(tableEnd), count) + c.splitRange(c.mvccStore, NewMvccKey(tableStart), NewMvccKey(tableEnd), count) } // SplitIndex evenly splits the data in index into count regions. // Only works for single store. -func (c *Cluster) SplitIndex(mvccStore MVCCStore, tableID, indexID int64, count int) { +func (c *Cluster) SplitIndex(tableID, indexID int64, count int) { indexStart := tablecodec.EncodeTableIndexPrefix(tableID, indexID) indexEnd := indexStart.PrefixNext() - c.splitRange(mvccStore, NewMvccKey(indexStart), NewMvccKey(indexEnd), count) + c.splitRange(c.mvccStore, NewMvccKey(indexStart), NewMvccKey(indexEnd), count) } // SplitKeys evenly splits the start, end key into "count" regions. // Only works for single store. -func (c *Cluster) SplitKeys(mvccStore MVCCStore, start, end kv.Key, count int) { - c.splitRange(mvccStore, NewMvccKey(start), NewMvccKey(end), count) +func (c *Cluster) SplitKeys(start, end kv.Key, count int) { + c.splitRange(c.mvccStore, NewMvccKey(start), NewMvccKey(end), count) } // ScheduleDelay schedules a delay event for a transaction on a region. @@ -444,7 +451,7 @@ func (c *Cluster) splitRange(mvccStore MVCCStore, start, end MvccKey, count int) // getEntriesGroupByRegions groups the key value pairs into splitted regions. func (c *Cluster) getEntriesGroupByRegions(mvccStore MVCCStore, start, end MvccKey, count int) [][]Pair { startTS := uint64(math.MaxUint64) - limit := int(math.MaxInt32) + limit := math.MaxInt32 pairs := mvccStore.Scan(start.Raw(), end.Raw(), limit, startTS, kvrpcpb.IsolationLevel_SI, nil) regionEntriesSlice := make([][]Pair, 0, count) quotient := len(pairs) / count diff --git a/store/mockstore/mocktikv/cluster_test.go b/store/mockstore/mocktikv/cluster_test.go index 0247b6120b6e7..f1a8489662564 100644 --- a/store/mockstore/mocktikv/cluster_test.go +++ b/store/mockstore/mocktikv/cluster_test.go @@ -42,6 +42,7 @@ func (s *testClusterSuite) TestClusterSplit(c *C) { cluster := mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(cluster) mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) store, err := mockstore.NewMockTikvStore( mockstore.WithCluster(cluster), mockstore.WithMVCCStore(mvccStore), @@ -58,7 +59,7 @@ func (s *testClusterSuite) TestClusterSplit(c *C) { handle := int64(1) sc := &stmtctx.StatementContext{TimeZone: time.UTC} for i := 0; i < 1000; i++ { - rowKey := tablecodec.EncodeRowKeyWithHandle(tblID, handle) + rowKey := tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(handle)) colValue := types.NewStringDatum(strconv.Itoa(int(handle))) // TODO: Should use session's TimeZone instead of UTC. rd := rowcodec.Encoder{Enable: true} @@ -76,7 +77,7 @@ func (s *testClusterSuite) TestClusterSplit(c *C) { c.Assert(err, IsNil) // Split Table into 10 regions. - cluster.SplitTable(mvccStore, tblID, 10) + cluster.SplitTable(tblID, 10) // 10 table regions and first region and last region. regions := cluster.GetAllRegions() @@ -100,7 +101,7 @@ func (s *testClusterSuite) TestClusterSplit(c *C) { } c.Assert(allKeysMap, HasLen, 1000) - cluster.SplitIndex(mvccStore, tblID, idxID, 10) + cluster.SplitIndex(tblID, idxID, 10) allIndexMap := make(map[string]bool) indexPrefix := tablecodec.EncodeTableIndexPrefix(tblID, idxID) diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index c418d5b99194b..0ecbee01d52cf 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -219,7 +219,7 @@ func (h *rpcHandler) buildTableScan(ctx *dagContext, executor *tipb.Executor) (* } return col.DefaultVal, nil } - rd := rowcodec.NewByteDecoder(colInfos, -1, defVal, nil) + rd := rowcodec.NewByteDecoder(colInfos, []int64{-1}, defVal, nil) e := &tableScanExec{ TableScan: executor.TblScan, kvRanges: ranges, @@ -787,8 +787,8 @@ func (h *rpcHandler) extractKVRanges(keyRanges []*coprocessor.KeyRange, descScan break } var kvr kv.KeyRange - kvr.StartKey = kv.Key(maxStartKey(lowerKey, h.rawStartKey)) - kvr.EndKey = kv.Key(minEndKey(upperKey, h.rawEndKey)) + kvr.StartKey = maxStartKey(lowerKey, h.rawStartKey) + kvr.EndKey = minEndKey(upperKey, h.rawEndKey) kvRanges = append(kvRanges, kvr) } if descScan { @@ -816,15 +816,15 @@ func appendRow(chunks []tipb.Chunk, data []byte, rowCnt int) []tipb.Chunk { } func maxStartKey(rangeStartKey kv.Key, regionStartKey []byte) []byte { - if bytes.Compare([]byte(rangeStartKey), regionStartKey) > 0 { - return []byte(rangeStartKey) + if bytes.Compare(rangeStartKey, regionStartKey) > 0 { + return rangeStartKey } return regionStartKey } func minEndKey(rangeEndKey kv.Key, regionEndKey []byte) []byte { - if len(regionEndKey) == 0 || bytes.Compare([]byte(rangeEndKey), regionEndKey) < 0 { - return []byte(rangeEndKey) + if len(regionEndKey) == 0 || bytes.Compare(rangeEndKey, regionEndKey) < 0 { + return rangeEndKey } return regionEndKey } diff --git a/store/mockstore/mocktikv/executor.go b/store/mockstore/mocktikv/executor.go index c812d435fbaa6..ae1cb1a8fbf47 100644 --- a/store/mockstore/mocktikv/executor.go +++ b/store/mockstore/mocktikv/executor.go @@ -193,7 +193,7 @@ func (e *tableScanExec) getRowFromPoint(ran kv.KeyRange) ([][]byte, error) { if err != nil { return nil, errors.Trace(err) } - row, err := getRowData(e.Columns, e.colIDs, handle, val, e.rd) + row, err := getRowData(e.Columns, e.colIDs, handle.IntValue(), val, e.rd) if err != nil { return nil, errors.Trace(err) } @@ -241,7 +241,7 @@ func (e *tableScanExec) getRowFromRange(ran kv.KeyRange) ([][]byte, error) { if err != nil { return nil, errors.Trace(err) } - row, err := getRowData(e.Columns, e.colIDs, handle, pair.Value, e.rd) + row, err := getRowData(e.Columns, e.colIDs, handle.IntValue(), pair.Value, e.rd) if err != nil { return nil, errors.Trace(err) } @@ -408,7 +408,7 @@ func (e *indexScanExec) getRowFromRange(ran kv.KeyRange) ([][]byte, error) { if bytes.Compare(pair.Key, ran.EndKey) >= 0 { return nil, nil } - e.seekKey = []byte(kv.Key(pair.Key).PrefixNext()) + e.seekKey = kv.Key(pair.Key).PrefixNext() } return tablecodec.DecodeIndexKV(pair.Key, pair.Value, e.colsLen, e.hdStatus, e.colInfos) } @@ -672,7 +672,7 @@ func hasColVal(data [][]byte, colIDs map[int64]int, id int64) bool { // getRowData decodes raw byte slice to row data. func getRowData(columns []*tipb.ColumnInfo, colIDs map[int64]int, handle int64, value []byte, rd *rowcodec.BytesDecoder) ([][]byte, error) { if rowcodec.IsNewFormat(value) { - return rd.DecodeToBytes(colIDs, handle, value, nil) + return rd.DecodeToBytes(colIDs, kv.IntHandle(handle), value, nil) } values, err := tablecodec.CutRowNew(value, colIDs) if err != nil { diff --git a/store/mockstore/mocktikv/executor_test.go b/store/mockstore/mocktikv/executor_test.go index 0e9d55b8d0a38..76d784c6450d0 100644 --- a/store/mockstore/mocktikv/executor_test.go +++ b/store/mockstore/mocktikv/executor_test.go @@ -77,7 +77,7 @@ func (s *testExecutorSuite) TestResolvedLargeTxnLocks(c *C) { tso, err := oracle.GetTimestamp(context.Background()) c.Assert(err, IsNil) - key := tablecodec.EncodeRowKeyWithHandle(tbl.Meta().ID, 1) + key := tablecodec.EncodeRowKeyWithHandle(tbl.Meta().ID, kv.IntHandle(1)) pairs := s.mvccStore.Scan(key, nil, 1, tso, kvrpcpb.IsolationLevel_SI, nil) c.Assert(pairs, HasLen, 1) c.Assert(pairs[0].Err, IsNil) diff --git a/store/mockstore/mocktikv/mock_tikv_test.go b/store/mockstore/mocktikv/mock_tikv_test.go index 49ef737e99dd9..5444c8ddcc828 100644 --- a/store/mockstore/mocktikv/mock_tikv_test.go +++ b/store/mockstore/mocktikv/mock_tikv_test.go @@ -52,7 +52,7 @@ func (s *testMockTiKVSuite) SetUpTest(c *C) { } // PutMutations is exported for testing. -var PutMutations func(kvpairs ...string) []*kvrpcpb.Mutation = putMutations +var PutMutations = putMutations func putMutations(kvpairs ...string) []*kvrpcpb.Mutation { var mutations []*kvrpcpb.Mutation @@ -695,12 +695,18 @@ func (s *testMVCCLevelDB) TestCheckTxnStatus(c *C) { c.Assert(commitTS, Equals, uint64(0)) c.Assert(action, Equals, kvrpcpb.Action_MinCommitTSPushed) + // MaxUint64 as callerStartTS shouldn't update minCommitTS but return Action_MinCommitTSPushed. + ttl, commitTS, action, err = s.store.CheckTxnStatus([]byte("pk"), startTS, math.MaxUint64, 666, false) + c.Assert(err, IsNil) + c.Assert(ttl, Equals, uint64(666)) + c.Assert(commitTS, Equals, uint64(0)) + c.Assert(action, Equals, kvrpcpb.Action_MinCommitTSPushed) s.mustCommitOK(c, [][]byte{[]byte("pk")}, startTS, startTS+101) ttl, commitTS, _, err = s.store.CheckTxnStatus([]byte("pk"), startTS, 0, 666, false) c.Assert(err, IsNil) c.Assert(ttl, Equals, uint64(0)) - c.Assert(commitTS, Equals, uint64(startTS+101)) + c.Assert(commitTS, Equals, startTS+101) s.mustPrewriteWithTTLOK(c, putMutations("pk1", "val"), "pk1", startTS, 666) s.mustRollbackOK(c, [][]byte{[]byte("pk1")}, startTS) diff --git a/store/mockstore/mocktikv/mvcc_leveldb.go b/store/mockstore/mocktikv/mvcc_leveldb.go index ffe30f9906153..e1f191009f806 100644 --- a/store/mockstore/mocktikv/mvcc_leveldb.go +++ b/store/mockstore/mocktikv/mvcc_leveldb.go @@ -1162,9 +1162,15 @@ func (mvcc *MVCCLevelDB) CheckTxnStatus(primaryKey []byte, lockTS, callerStartTS return 0, 0, kvrpcpb.Action_TTLExpireRollback, nil } - // If this is a large transaction and the lock is active, push forward the minCommitTS. - // lock.minCommitTS == 0 may be a secondary lock, or not a large transaction (old version TiDB). - if lock.minCommitTS > 0 { + // If the caller_start_ts is MaxUint64, it's a point get in the autocommit transaction. + // Even though the MinCommitTs is not pushed, the point get can ingore the lock + // next time because it's not committed. So we pretend it has been pushed. + if callerStartTS == math.MaxUint64 { + action = kvrpcpb.Action_MinCommitTSPushed + + // If this is a large transaction and the lock is active, push forward the minCommitTS. + // lock.minCommitTS == 0 may be a secondary lock, or not a large transaction (old version TiDB). + } else if lock.minCommitTS > 0 { action = kvrpcpb.Action_MinCommitTSPushed // We *must* guarantee the invariance lock.minCommitTS >= callerStartTS + 1 if lock.minCommitTS < callerStartTS+1 { diff --git a/store/mockstore/mocktikv/pd.go b/store/mockstore/mocktikv/pd.go index 5497a38aedece..fcc93be991847 100644 --- a/store/mockstore/mocktikv/pd.go +++ b/store/mockstore/mocktikv/pd.go @@ -15,6 +15,7 @@ package mocktikv import ( "context" + "math" "sync" "time" @@ -34,15 +35,19 @@ var tsMu = struct { type pdClient struct { cluster *Cluster // SafePoint set by `UpdateGCSafePoint`. Not to be confused with SafePointKV. - gcSafePoint uint64 - gcSafePointMu sync.Mutex + gcSafePoint uint64 + // Represents the current safePoint of all services including TiDB, representing how much data they want to retain + // in GC. + serviceSafePoints map[string]uint64 + gcSafePointMu sync.Mutex } // NewPDClient creates a mock pd.Client that uses local timestamp and meta data // from a Cluster. func NewPDClient(cluster *Cluster) pd.Client { return &pdClient{ - cluster: cluster, + cluster: cluster, + serviceSafePoints: make(map[string]uint64), } } @@ -130,6 +135,35 @@ func (c *pdClient) UpdateGCSafePoint(ctx context.Context, safePoint uint64) (uin return c.gcSafePoint, nil } +func (c *pdClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { + c.gcSafePointMu.Lock() + defer c.gcSafePointMu.Unlock() + + if ttl == 0 { + delete(c.serviceSafePoints, serviceID) + } else { + var minSafePoint uint64 = math.MaxUint64 + for _, ssp := range c.serviceSafePoints { + if ssp < minSafePoint { + minSafePoint = ssp + } + } + + if len(c.serviceSafePoints) == 0 || minSafePoint <= safePoint { + c.serviceSafePoints[serviceID] = safePoint + } + } + + // The minSafePoint may have changed. Reload it. + var minSafePoint uint64 = math.MaxUint64 + for _, ssp := range c.serviceSafePoints { + if ssp < minSafePoint { + minSafePoint = ssp + } + } + return minSafePoint, nil +} + func (c *pdClient) Close() { } diff --git a/store/mockstore/mocktikv/rpc.go b/store/mockstore/mocktikv/rpc.go index 324e838d6b574..36eb3b14facba 100644 --- a/store/mockstore/mocktikv/rpc.go +++ b/store/mockstore/mocktikv/rpc.go @@ -252,7 +252,7 @@ func (h *rpcHandler) checkRequest(ctx *kvrpcpb.Context, size int) *errorpb.Error } func (h *rpcHandler) checkKeyInRegion(key []byte) bool { - return regionContains(h.startKey, h.endKey, []byte(NewMvccKey(key))) + return regionContains(h.startKey, h.endKey, NewMvccKey(key)) } func (h *rpcHandler) handleKvGet(req *kvrpcpb.GetRequest) *kvrpcpb.GetResponse { @@ -766,6 +766,10 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R } }) + // increase coverage for mock tikv + _ = req.Type.String() + _ = req.ToBatchCommandsRequest() + reqCtx := &req.Context resp := &tikvrpc.Response{} // When the store type is TiDB, the request should handle over to TiDB rpc server to handle. diff --git a/store/mockstore/tikv.go b/store/mockstore/tikv.go index 32ec9e8132aca..c9bd9c776522f 100644 --- a/store/mockstore/tikv.go +++ b/store/mockstore/tikv.go @@ -18,7 +18,7 @@ import ( "strings" "github.com/pingcap/errors" - "github.com/pingcap/pd/v4/client" + pd "github.com/pingcap/pd/v4/client" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore/mocktikv" diff --git a/store/store_test.go b/store/store_test.go index b5f004eacc7df..13bad3ce2172f 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -297,7 +297,7 @@ func (s *testKVSuite) TestDelete2(c *C) { it, err := txn.Iter([]byte("DATA_test_tbl_department_record__0000000001_0003"), nil) c.Assert(err, IsNil) for it.Valid() { - err = txn.Delete([]byte(it.Key())) + err = txn.Delete(it.Key()) c.Assert(err, IsNil) err = it.Next() c.Assert(err, IsNil) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 2049b2fbc2c4f..fdc5bcc5c827b 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx/binloginfo" @@ -49,7 +50,7 @@ type twoPhaseCommitAction interface { } type actionPrewrite struct{} -type actionCommit struct{} +type actionCommit struct{ retry bool } type actionCleanup struct{} type actionPessimisticLock struct { *kv.LockCtx @@ -422,6 +423,9 @@ func txnLockTTL(startTime time.Time, txnSize int) uint64 { return lockTTL + uint64(elapsed) } +var preSplitDetectThreshold uint32 = 100000 +var preSplitSizeThreshold uint32 = 32 << 20 + // doActionOnMutations groups keys into primary batch and secondary batches, if primary batch exists in the key, // it does action on primary batch first, then on secondary batches. If action is commit, secondary batches // is done in background goroutine. @@ -434,6 +438,68 @@ func (c *twoPhaseCommitter) doActionOnMutations(bo *Backoffer, action twoPhaseCo return errors.Trace(err) } + // Pre-split regions to avoid too much write workload into a single region. + // In the large transaction case, this operation is important to avoid TiKV 'server is busy' error. + var preSplited bool + preSplitDetectThresholdVal := atomic.LoadUint32(&preSplitDetectThreshold) + for _, group := range groups { + if uint32(group.mutations.len()) >= preSplitDetectThresholdVal { + logutil.BgLogger().Info("2PC detect large amount of mutations on a single region", + zap.Uint64("region", group.region.GetID()), + zap.Int("mutations count", group.mutations.len())) + // Use context.Background, this time should not add up to Backoffer. + if preSplitAndScatterIn2PC(context.Background(), c.store, group) { + preSplited = true + } + } + } + // Reload region cache again. + if preSplited { + groups, err = c.store.regionCache.GroupSortedMutationsByRegion(bo, mutations) + if err != nil { + return errors.Trace(err) + } + } + + return c.doActionOnGroupMutations(bo, action, groups) +} + +func preSplitAndScatterIn2PC(ctx context.Context, store *tikvStore, group groupedMutations) bool { + length := group.mutations.len() + splitKeys := make([][]byte, 0, 4) + + preSplitSizeThresholdVal := atomic.LoadUint32(&preSplitSizeThreshold) + regionSize := 0 + for i := 0; i < length; i++ { + regionSize = regionSize + len(group.mutations.keys[i]) + len(group.mutations.values[i]) + // The second condition is used for testing. + if regionSize >= int(preSplitSizeThresholdVal) { + regionSize = 0 + splitKeys = append(splitKeys, group.mutations.keys[i]) + } + } + if len(splitKeys) == 0 { + return false + } + + regionIDs, err := store.SplitRegions(ctx, splitKeys, true) + if err != nil { + logutil.BgLogger().Warn("2PC split regions failed", zap.Uint64("regionID", group.region.id), zap.Int("keys count", length), zap.Error(err)) + return false + } + + for _, regionID := range regionIDs { + err := store.WaitScatterRegionFinish(regionID, 0) + if err != nil { + logutil.BgLogger().Warn("2PC wait scatter region failed", zap.Uint64("regionID", regionID), zap.Error(err)) + } + } + // Invalidate the old region cache information. + store.regionCache.InvalidateCachedRegion(group.region) + return true +} + +func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPhaseCommitAction, groups []groupedMutations) error { action.tiKVTxnRegionsNumHistogram().Observe(float64(len(groups))) var batches []batchMutations @@ -462,10 +528,11 @@ func (c *twoPhaseCommitter) doActionOnMutations(bo *Backoffer, action twoPhaseCo firstIsPrimary = true } - _, actionIsCommit := action.(actionCommit) + actionCommit, actionIsCommit := action.(actionCommit) _, actionIsCleanup := action.(actionCleanup) _, actionIsPessimiticLock := action.(actionPessimisticLock) + var err error failpoint.Inject("skipKeyReturnOK", func(val failpoint.Value) { valStr, ok := val.(string) if ok && c.connID > 0 { @@ -498,7 +565,7 @@ func (c *twoPhaseCommitter) doActionOnMutations(bo *Backoffer, action twoPhaseCo } batches = batches[1:] } - if actionIsCommit { + if actionIsCommit && !actionCommit.retry { // Commit secondary batches in background goroutine to reduce latency. // The backoffer instance is created outside of the goroutine to avoid // potential data race in unit test since `CommitMaxBackoff` will be updated @@ -526,24 +593,33 @@ func (c *twoPhaseCommitter) doActionOnBatches(bo *Backoffer, action twoPhaseComm return nil } - if len(batches) == 1 { - e := action.handleSingleBatch(c, bo, batches[0]) - if e != nil { - logutil.BgLogger().Debug("2PC doActionOnBatches failed", - zap.Uint64("conn", c.connID), - zap.Stringer("action type", action), - zap.Error(e), - zap.Uint64("txnStartTS", c.startTS)) + noNeedFork := len(batches) == 1 + if !noNeedFork { + if ac, ok := action.(actionCommit); ok && ac.retry { + noNeedFork = true } - return errors.Trace(e) + } + if noNeedFork { + for _, b := range batches { + e := action.handleSingleBatch(c, bo, b) + if e != nil { + logutil.BgLogger().Debug("2PC doActionOnBatches failed", + zap.Uint64("conn", c.connID), + zap.Stringer("action type", action), + zap.Error(e), + zap.Uint64("txnStartTS", c.startTS)) + return errors.Trace(e) + } + } + return nil } rateLim := len(batches) // Set rateLim here for the large transaction. // If the rate limit is too high, tikv will report service is busy. // If the rate limit is too low, we can't full utilize the tikv's throughput. // TODO: Find a self-adaptive way to control the rate limit here. - if rateLim > 16 { - rateLim = 16 + if rateLim > config.GetGlobalConfig().Performance.CommitterConcurrency { + rateLim = config.GetGlobalConfig().Performance.CommitterConcurrency } batchExecutor := newBatchExecutor(rateLim, c, action, bo) err := batchExecutor.process(batches) @@ -729,12 +805,13 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { } uptime := uint64(oracle.ExtractPhysical(now) - oracle.ExtractPhysical(c.startTS)) - const c10min = 10 * 60 * 1000 - if uptime > c10min { - // Set a 10min maximum lifetime for the ttlManager, so when something goes wrong + if uptime > config.GetGlobalConfig().Performance.MaxTxnTTL { + // Checks maximum lifetime for the ttlManager, so when something goes wrong // the key will not be locked forever. logutil.Logger(bo.ctx).Info("ttlManager live up to its lifetime", - zap.Uint64("txnStartTS", c.startTS)) + zap.Uint64("txnStartTS", c.startTS), + zap.Uint64("uptime", uptime), + zap.Uint64("maxTxnTTL", config.GetGlobalConfig().Performance.MaxTxnTTL)) metrics.TiKVTTLLifeTimeReachCounter.Inc() // the pessimistic locks may expire if the ttl manager has timed out, set `LockExpired` flag // so that this transaction could only commit or rollback with no more statement executions @@ -996,7 +1073,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch return errors.Trace(err) } // re-split keys and commit again. - err = c.commitMutations(bo, batch.mutations) + err = c.doActionOnMutations(bo, actionCommit{retry: true}, batch.mutations) return errors.Trace(err) } if resp.Resp == nil { @@ -1408,7 +1485,7 @@ func (c *twoPhaseCommitter) appendBatchMutationsBySize(b []batchMutations, regio func newBatchExecutor(rateLimit int, committer *twoPhaseCommitter, action twoPhaseCommitAction, backoffer *Backoffer) *batchExecutor { return &batchExecutor{rateLimit, nil, committer, - action, backoffer, time.Duration(1 * time.Millisecond)} + action, backoffer, 1 * time.Millisecond} } // initUtils do initialize batchExecutor related policies like rateLimit util diff --git a/store/tikv/2pc_slow_test.go b/store/tikv/2pc_slow_test.go index bf96a24485ac3..c7a70a188306e 100644 --- a/store/tikv/2pc_slow_test.go +++ b/store/tikv/2pc_slow_test.go @@ -15,7 +15,12 @@ package tikv -import . "github.com/pingcap/check" +import ( + "context" + "sync/atomic" + + . "github.com/pingcap/check" +) // TestCommitMultipleRegions tests commit multiple regions. // The test takes too long under the race detector. @@ -35,3 +40,38 @@ func (s *testCommitterSuite) TestCommitMultipleRegions(c *C) { } s.mustCommit(c, m) } + +func (s *testTiclientSuite) TestSplitRegionIn2PC(c *C) { + const preSplitThresholdInTest = 500 + old := atomic.LoadUint32(&preSplitDetectThreshold) + defer atomic.StoreUint32(&preSplitDetectThreshold, old) + atomic.StoreUint32(&preSplitDetectThreshold, preSplitThresholdInTest) + + old = atomic.LoadUint32(&preSplitSizeThreshold) + defer atomic.StoreUint32(&preSplitSizeThreshold, old) + atomic.StoreUint32(&preSplitSizeThreshold, 5000) + + bo := NewBackoffer(context.Background(), 1) + startKey := encodeKey(s.prefix, s08d("key", 0)) + endKey := encodeKey(s.prefix, s08d("key", preSplitThresholdInTest)) + checkKeyRegion := func(bo *Backoffer, start, end []byte, checker Checker) { + // Check regions after split. + loc1, err := s.store.regionCache.LocateKey(bo, start) + c.Assert(err, IsNil) + loc2, err := s.store.regionCache.LocateKey(bo, end) + c.Assert(err, IsNil) + c.Assert(loc1.Region.id, checker, loc2.Region.id) + } + + // Check before test. + checkKeyRegion(bo, startKey, endKey, Equals) + txn := s.beginTxn(c) + for i := 0; i < preSplitThresholdInTest; i++ { + err := txn.Set(encodeKey(s.prefix, s08d("key", i)), valueBytes(i)) + c.Assert(err, IsNil) + } + err := txn.Commit(context.Background()) + c.Assert(err, IsNil) + // Check region split after test. + checkKeyRegion(bo, startKey, endKey, Not(Equals)) +} diff --git a/store/tikv/2pc_test.go b/store/tikv/2pc_test.go index 5009ca8b9b55f..b045805dae33b 100644 --- a/store/tikv/2pc_test.go +++ b/store/tikv/2pc_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -36,9 +37,8 @@ import ( type testCommitterSuite struct { OneByOneSuite - cluster *mocktikv.Cluster - store *tikvStore - mvccStore mocktikv.MVCCStore + cluster cluster.Cluster + store *tikvStore } var _ = SerialSuites(&testCommitterSuite{}) @@ -49,13 +49,14 @@ func (s *testCommitterSuite) SetUpSuite(c *C) { } func (s *testCommitterSuite) SetUpTest(c *C) { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithMultiRegions(s.cluster, []byte("a"), []byte("b"), []byte("c")) + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithMultiRegions(cluster, []byte("a"), []byte("b"), []byte("c")) + s.cluster = cluster mvccStore, err := mocktikv.NewMVCCLevelDB("") c.Assert(err, IsNil) - s.mvccStore = mvccStore - client := mocktikv.NewRPCClient(s.cluster, mvccStore) - pdCli := &codecPDClient{mocktikv.NewPDClient(s.cluster)} + cluster.SetMvccStore(mvccStore) + client := mocktikv.NewRPCClient(cluster, mvccStore) + pdCli := &codecPDClient{mocktikv.NewPDClient(cluster)} spkv := NewMockSafePointKV() store, err := newTikvStore("mocktikv-store", pdCli, spkv, client, false, nil) c.Assert(err, IsNil) @@ -845,7 +846,7 @@ func (c *twoPhaseCommitter) mutationsOfKeys(keys [][]byte) committerMutations { func (s *testCommitterSuite) TestCommitDeadLock(c *C) { // Split into two region and let k1 k2 in different regions. - s.cluster.SplitKeys(s.mvccStore, kv.Key("z"), kv.Key("a"), 2) + s.cluster.SplitKeys(kv.Key("z"), kv.Key("a"), 2) k1 := kv.Key("a_deadlock_k1") k2 := kv.Key("y_deadlock_k2") diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index 55c9165d7e59a..48c993bdc58b0 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -51,6 +51,7 @@ var ( tikvBackoffHistogramRegionMiss = metrics.TiKVBackoffHistogram.WithLabelValues("regionMiss") tikvBackoffHistogramUpdateLeader = metrics.TiKVBackoffHistogram.WithLabelValues("updateLeader") tikvBackoffHistogramServerBusy = metrics.TiKVBackoffHistogram.WithLabelValues("serverBusy") + tikvBackoffHistogramStaleCmd = metrics.TiKVBackoffHistogram.WithLabelValues("staleCommand") tikvBackoffHistogramEmpty = metrics.TiKVBackoffHistogram.WithLabelValues("") ) @@ -70,6 +71,8 @@ func (t backoffType) metric() prometheus.Observer { return tikvBackoffHistogramUpdateLeader case boServerBusy: return tikvBackoffHistogramServerBusy + case boStaleCmd: + return tikvBackoffHistogramStaleCmd } return tikvBackoffHistogramEmpty } @@ -134,6 +137,7 @@ const ( BoUpdateLeader boServerBusy boTxnNotFound + boStaleCmd ) func (t backoffType) createFn(vars *kv.Variables) func(context.Context, int) int { @@ -158,6 +162,8 @@ func (t backoffType) createFn(vars *kv.Variables) func(context.Context, int) int return NewBackoffFn(1, 10, NoJitter) case boServerBusy: return NewBackoffFn(2000, 10000, EqualJitter) + case boStaleCmd: + return NewBackoffFn(2, 1000, NoJitter) } return nil } @@ -178,6 +184,8 @@ func (t backoffType) String() string { return "updateLeader" case boServerBusy: return "serverBusy" + case boStaleCmd: + return "staleCommand" case boTxnNotFound: return "txnNotFound" } @@ -196,6 +204,8 @@ func (t backoffType) TError() error { return ErrRegionUnavailable case boServerBusy: return ErrTiKVServerBusy + case boStaleCmd: + return ErrTiKVStaleCommand } return ErrUnknown } diff --git a/store/tikv/backoff_test.go b/store/tikv/backoff_test.go index ddf7d1fcf86f6..a406f53b5244c 100644 --- a/store/tikv/backoff_test.go +++ b/store/tikv/backoff_test.go @@ -16,6 +16,7 @@ package tikv import ( "context" "errors" + . "github.com/pingcap/check" ) diff --git a/store/tikv/client.go b/store/tikv/client.go index 5af5aacf2b031..d83d0add1cba8 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -85,20 +85,20 @@ type connArray struct { done chan struct{} } -func newConnArray(maxSize uint, addr string, security config.Security, dieNotify *uint32, enableBatch bool) (*connArray, error) { +func newConnArray(maxSize uint, addr string, security config.Security, idleNotify *uint32, enableBatch bool) (*connArray, error) { a := &connArray{ index: 0, v: make([]*grpc.ClientConn, maxSize), streamTimeout: make(chan *tikvrpc.Lease, 1024), done: make(chan struct{}), } - if err := a.Init(addr, security, dieNotify, enableBatch); err != nil { + if err := a.Init(addr, security, idleNotify, enableBatch); err != nil { return nil, err } return a, nil } -func (a *connArray) Init(addr string, security config.Security, dieNotify *uint32, enableBatch bool) error { +func (a *connArray) Init(addr string, security config.Security, idleNotify *uint32, enableBatch bool) error { a.target = addr opt := grpc.WithInsecure() @@ -122,7 +122,7 @@ func (a *connArray) Init(addr string, security config.Security, dieNotify *uint3 allowBatch := (cfg.TiKVClient.MaxBatchSize > 0) && enableBatch if allowBatch { - a.batchConn = newBatchConn(uint(len(a.v)), cfg.TiKVClient.MaxBatchSize, dieNotify) + a.batchConn = newBatchConn(uint(len(a.v)), cfg.TiKVClient.MaxBatchSize, idleNotify) a.pendingRequests = metrics.TiKVPendingBatchRequests.WithLabelValues(a.target) } keepAlive := cfg.TiKVClient.GrpcKeepAliveTime @@ -170,8 +170,6 @@ func (a *connArray) Init(addr string, security config.Security, dieNotify *uint3 closed: 0, tikvClientCfg: cfg.TiKVClient, tikvLoad: &a.tikvTransportLayerLoad, - dieNotify: a.dieNotify, - dieFlag: &a.die, } a.batchCommandsClients = append(a.batchCommandsClients, batchClient) } @@ -215,11 +213,10 @@ type rpcClient struct { conns map[string]*connArray security config.Security - dieNotify uint32 - // Periodically check whether there is any connection that was die and then close and remove these connections. + idleNotify uint32 + // Periodically check whether there is any connection that is idle and then close and remove these connections. // Implement background cleanup. - isClosed bool - dieEventListener func(addr []string) + isClosed bool } func newRPCClient(security config.Security) *rpcClient { @@ -259,7 +256,7 @@ func (c *rpcClient) createConnArray(addr string, enableBatch bool) (*connArray, if !ok { var err error connCount := config.GetGlobalConfig().TiKVClient.GrpcConnectionCount - array, err = newConnArray(connCount, addr, c.security, &c.dieNotify, enableBatch) + array, err = newConnArray(connCount, addr, c.security, &c.idleNotify, enableBatch) if err != nil { return nil, err } @@ -315,8 +312,8 @@ func (c *rpcClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R start := time.Now() defer c.updateTiKVSendReqHistogram(req, start) - if atomic.CompareAndSwapUint32(&c.dieNotify, 1, 0) { - c.recycleDieConnArray() + if atomic.CompareAndSwapUint32(&c.idleNotify, 1, 0) { + c.recycleIdleConnArray() } enableBatch := req.StoreTp != kv.TiDB diff --git a/store/tikv/client_batch.go b/store/tikv/client_batch.go index b30650ebaf4d9..08520afc51219 100644 --- a/store/tikv/client_batch.go +++ b/store/tikv/client_batch.go @@ -36,9 +36,9 @@ import ( ) type batchConn struct { - // An atomic flag indicates whether the batch is die or not. - // 0 for busy, others for die. - die uint32 + // An atomic flag indicates whether the batch is idle or not. + // 0 for busy, others for idle. + idle uint32 // batchCommandsCh used for batch commands. batchCommandsCh chan *batchCommandsEntry @@ -46,8 +46,8 @@ type batchConn struct { tikvTransportLayerLoad uint64 closed chan struct{} - // Notify rpcClient to check the die flag - dieNotify *uint32 + // Notify rpcClient to check the idle flag + idleNotify *uint32 idleDetect *time.Timer pendingRequests prometheus.Gauge @@ -55,20 +55,20 @@ type batchConn struct { index uint32 } -func newBatchConn(connCount, maxBatchSize uint, dieNotify *uint32) *batchConn { +func newBatchConn(connCount, maxBatchSize uint, idleNotify *uint32) *batchConn { return &batchConn{ batchCommandsCh: make(chan *batchCommandsEntry, maxBatchSize), batchCommandsClients: make([]*batchCommandsClient, 0, connCount), tikvTransportLayerLoad: 0, closed: make(chan struct{}), - dieNotify: dieNotify, + idleNotify: idleNotify, idleDetect: time.NewTimer(idleTimeout), } } -func (a *batchConn) isDie() bool { - return atomic.LoadUint32(&a.die) != 0 +func (a *batchConn) isIdle() bool { + return atomic.LoadUint32(&a.idle) != 0 } // fetchAllPendingRequests fetches all pending requests from the channel. @@ -87,17 +87,9 @@ func (a *batchConn) fetchAllPendingRequests( a.idleDetect.Reset(idleTimeout) case <-a.idleDetect.C: a.idleDetect.Reset(idleTimeout) - // send store heartbeat - heartBeatEntry := &batchCommandsEntry{ - ctx: context.Background(), - req: tikvrpc.NewRequest(tikvrpc.CmdEmpty, &tikvpb.BatchCommandsEmptyRequest{}).ToBatchCommandsRequest(), - res: make(chan *tikvpb.BatchCommandsResponse_Response, 1), - canceled: 0, - err: nil, - heartbeat: true, - } - *entries = append(*entries, heartBeatEntry) - *requests = append(*requests, heartBeatEntry.req) + atomic.AddUint32(&a.idle, 1) + atomic.CompareAndSwapUint32(a.idleNotify, 0, 1) + // This batchConn to be recycled return case <-a.closed: return @@ -216,9 +208,6 @@ type batchCommandsClient struct { closed int32 // tryLock protects client when re-create the streaming. tryLock - - dieNotify *uint32 - dieFlag *uint32 } func (c *batchCommandsClient) isStopped() bool { @@ -239,12 +228,6 @@ func (c *batchCommandsClient) send(request *tikvpb.BatchCommandsRequest, entries c.failPendingRequests(err) return } - if len(entries) == 1 { - failpoint.InjectContext(entries[0].ctx, "failBeforeSend", func() { - c.failPendingRequests(errors.New("test err")) - failpoint.Return() - }) - } if err := c.client.Send(request); err != nil { logutil.BgLogger().Info( "sending batch commands meets error", @@ -252,9 +235,7 @@ func (c *batchCommandsClient) send(request *tikvpb.BatchCommandsRequest, entries zap.Error(err), ) c.failPendingRequests(err) - return } - return } func (c *batchCommandsClient) recv() (*tikvpb.BatchCommandsResponse, error) { @@ -268,26 +249,24 @@ func (c *batchCommandsClient) recv() (*tikvpb.BatchCommandsResponse, error) { // `failPendingRequests` must be called in locked contexts in order to avoid double closing channels. func (c *batchCommandsClient) failPendingRequests(err error) { failpoint.Inject("panicInFailPendingRequests", nil) - var heartbeatFail bool c.batched.Range(func(key, value interface{}) bool { id, _ := key.(uint64) entry, _ := value.(*batchCommandsEntry) entry.err = err c.batched.Delete(id) close(entry.res) - if entry.heartbeat { - heartbeatFail = true - } return true }) - if heartbeatFail { - atomic.AddUint32(c.dieFlag, 1) - atomic.CompareAndSwapUint32(c.dieNotify, 0, 1) - } - return } func (c *batchCommandsClient) waitConnReady() (err error) { + if c.conn.GetState() == connectivity.Ready { + return + } + start := time.Now() + defer func() { + metrics.TiKVBatchClientWaitEstablish.Observe(time.Since(start).Seconds()) + }() dialCtx, cancel := context.WithTimeout(context.Background(), dialTimeout) for { s := c.conn.GetState() @@ -373,12 +352,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient, tikvTransport } entry := value.(*batchCommandsEntry) logutil.Eventf(entry.ctx, "receive %T response with other %d batched requests from %s", responses[i].GetCmd(), len(responses), c.target) - failpoint.InjectContext(entry.ctx, "forceReturnIdleHeartbeatResp", func() { - if entry.heartbeat { - entry.res <- responses[i] - } - }) - if atomic.LoadInt32(&entry.canceled) == 0 && !entry.heartbeat { + if atomic.LoadInt32(&entry.canceled) == 0 { // Put the response only if the request is not canceled. entry.res <- responses[i] } @@ -422,16 +396,15 @@ type batchCommandsEntry struct { res chan *tikvpb.BatchCommandsResponse_Response // canceled indicated the request is canceled or not. - canceled int32 - err error - heartbeat bool + canceled int32 + err error } func (b *batchCommandsEntry) isCanceled() bool { return atomic.LoadInt32(&b.canceled) == 1 } -const idleTimeout = 1 * time.Minute +const idleTimeout = 3 * time.Minute func resetEntries(entries []*batchCommandsEntry) []*batchCommandsEntry { for i := 0; i < len(entries); i++ { @@ -522,25 +495,14 @@ func (a *batchConn) getClientAndSend(entries []*batchCommandsEntry, requests []* break } } - if len(entries) == 1 { - failpoint.InjectContext(entries[0].ctx, "noAvConn", func() { - cli = nil - }) - } if cli == nil { logutil.BgLogger().Warn("no available connections", zap.String("target", target)) - var heartbeatFail bool + metrics.TiKVNoAvailableConnectionCounter.Inc() + for _, entry := range entries { // Please ensure the error is handled in region cache correctly. entry.err = errors.New("no available connections") close(entry.res) - if entry.heartbeat { - heartbeatFail = true - } - } - if heartbeatFail { - atomic.AddUint32(&a.die, 1) - atomic.CompareAndSwapUint32(a.dieNotify, 0, 1) } return } @@ -619,9 +581,6 @@ func sendBatchRequest( canceled: 0, err: nil, } - failpoint.InjectContext(ctx, "sendIdleHeartbeatReq", func() { - entry.heartbeat = true - }) timer := time.NewTimer(timeout) defer timer.Stop() @@ -651,26 +610,22 @@ func sendBatchRequest( } } -func (c *rpcClient) recycleDieConnArray() { +func (c *rpcClient) recycleIdleConnArray() { var addrs []string c.RLock() for _, conn := range c.conns { - if conn.isDie() { + if conn.batchConn != nil && conn.isIdle() { addrs = append(addrs, conn.target) } } c.RUnlock() - if c.dieEventListener != nil { - c.dieEventListener(addrs) - } - for _, addr := range addrs { c.Lock() conn, ok := c.conns[addr] if ok { delete(c.conns, addr) - logutil.BgLogger().Info("recycle die connection", + logutil.BgLogger().Info("recycle idle connection", zap.String("target", addr)) } c.Unlock() diff --git a/store/tikv/client_collapse.go b/store/tikv/client_collapse.go new file mode 100644 index 0000000000000..5b0386ef70bfa --- /dev/null +++ b/store/tikv/client_collapse.go @@ -0,0 +1,92 @@ +// Copyright 2020 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package tikv provides tcp connection to kvserver. +package tikv + +import ( + "context" + "strconv" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/store/tikv/tikvrpc" + "golang.org/x/sync/singleflight" +) + +var _ Client = reqCollapse{} + +var resolveRegionSf singleflight.Group + +type reqCollapse struct { + Client +} + +func (r reqCollapse) Close() error { + if r.Client == nil { + panic("client should not be nil") + } + return r.Client.Close() +} + +func (r reqCollapse) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { + if r.Client == nil { + panic("client should not be nil") + } + if canCollapse, resp, err := r.tryCollapseRequest(ctx, addr, req, timeout); canCollapse { + return resp, err + } + return r.Client.SendRequest(ctx, addr, req, timeout) +} + +func (r reqCollapse) tryCollapseRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (canCollapse bool, resp *tikvrpc.Response, err error) { + switch req.Type { + case tikvrpc.CmdResolveLock: + resolveLock := req.ResolveLock() + if len(resolveLock.Keys) > 0 { + // can not collapse resolve lock lite + return + } + canCollapse = true + key := strconv.FormatUint(resolveLock.Context.RegionId, 10) + "-" + strconv.FormatUint(resolveLock.StartVersion, 10) + resp, err = r.collapse(ctx, key, &resolveRegionSf, addr, req, timeout) + return + default: + // now we only support collapse resolve lock. + return + } +} + +func (r reqCollapse) collapse(ctx context.Context, key string, sf *singleflight.Group, + addr string, req *tikvrpc.Request, timeout time.Duration) (resp *tikvrpc.Response, err error) { + rsC := sf.DoChan(key, func() (interface{}, error) { + return r.Client.SendRequest(context.Background(), addr, req, readTimeoutShort) // use resolveLock timeout. + }) + timer := time.NewTimer(timeout) + defer timer.Stop() + select { + case <-ctx.Done(): + err = errors.Trace(ctx.Err()) + return + case <-timer.C: + err = errors.Trace(context.DeadlineExceeded) + return + case rs := <-rsC: + if rs.Err != nil { + err = errors.Trace(rs.Err) + return + } + resp = rs.Val.(*tikvrpc.Response) + return + } +} diff --git a/store/tikv/client_test.go b/store/tikv/client_test.go index f720f8a5d47f9..6e1184c5c3624 100644 --- a/store/tikv/client_test.go +++ b/store/tikv/client_test.go @@ -16,13 +16,11 @@ package tikv import ( "context" "fmt" - "sync/atomic" "testing" "time" . "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/tikvpb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -129,84 +127,3 @@ func (s *testClientSuite) TestSendWhenReconnect(c *C) { conn.Close() server.Stop() } - -func (s *testClientSuite) TestIdleHeartbeat(c *C) { - server, port := startMockTikvService() - c.Assert(port > 0, IsTrue) - defer server.Stop() - - rpcClient := newRPCClient(config.Security{}) - addr := fmt.Sprintf("%s:%d", "127.0.0.1", port) - conn, err := rpcClient.getConnArray(addr, true) - c.Assert(err, IsNil) - - sendIdleReq := "github.com/pingcap/tidb/store/tikv/sendIdleHeartbeatReq" - noStripResp := "github.com/pingcap/tidb/store/tikv/forceReturnIdleHeartbeatResp" - noAvConn := "github.com/pingcap/tidb/store/tikv/noAvConn" - failBeforeSend := "github.com/pingcap/tidb/store/tikv/failBeforeSend" - - c.Assert(failpoint.Enable(sendIdleReq, `return()`), IsNil) - c.Assert(failpoint.Enable(noStripResp, `return()`), IsNil) - c.Assert(failpoint.Enable(noAvConn, `return()`), IsNil) - c.Assert(failpoint.Enable(failBeforeSend, `return()`), IsNil) - defer func() { - c.Assert(failpoint.Disable(sendIdleReq), IsNil) - c.Assert(failpoint.Disable(noStripResp), IsNil) - c.Assert(failpoint.Disable(noAvConn), IsNil) - c.Assert(failpoint.Disable(failBeforeSend), IsNil) - }() - - // 1. test trigger idle heartbeat and return success by a live store. - ctx := failpoint.WithHook(context.TODO(), func(ctx context.Context, fpname string) bool { - if fpname == sendIdleReq || fpname == noStripResp { - return true - } - return false - }) - req := tikvrpc.NewRequest(tikvrpc.CmdEmpty, &tikvpb.BatchCommandsEmptyRequest{}).ToBatchCommandsRequest() - _, err = sendBatchRequest(ctx, addr, conn.batchConn, req, 100*time.Second) - c.Assert(err, IsNil) - - // 2. test trigger idle heartbeat and cannot found any conn. - ctx = failpoint.WithHook(context.TODO(), func(ctx context.Context, fpname string) bool { - if fpname == sendIdleReq || fpname == noStripResp || fpname == noAvConn { - return true - } - return false - }) - var dieNode []string - rpcClient.dieEventListener = func(addr []string) { - dieNode = append(dieNode, addr...) - } - _, err = sendBatchRequest(ctx, addr, conn.batchConn, req, 100*time.Second) - c.Assert(err, NotNil) // no available connections - c.Assert(conn.batchConn.isDie(), IsTrue) - c.Assert(atomic.LoadUint32(conn.batchConn.dieNotify), Equals, uint32(1)) - rpcClient.recycleDieConnArray() - c.Assert(len(dieNode), Equals, 1) - c.Assert(dieNode[0], Equals, addr) - - // 3. test trigger idle heartbeat and send fail before send. - conn, err = rpcClient.getConnArray(addr, true) - c.Assert(err, IsNil) - ctx = failpoint.WithHook(context.TODO(), func(ctx context.Context, fpname string) bool { - if fpname == sendIdleReq || fpname == noStripResp || fpname == failBeforeSend { - return true - } - return false - }) - dieNode = dieNode[:0] - rpcClient.dieEventListener = func(addr []string) { - dieNode = append(dieNode, addr...) - } - _, err = sendBatchRequest(ctx, addr, conn.batchConn, req, 100*time.Second) - c.Assert(err, NotNil) // no available connections - c.Assert(conn.batchConn.isDie(), IsTrue) - c.Assert(atomic.LoadUint32(conn.batchConn.dieNotify), Equals, uint32(1)) - rpcClient.recycleDieConnArray() - c.Assert(len(dieNode), Greater, 0) - c.Assert(dieNode[0], Equals, addr) - rpcClient.recycleDieConnArray() - c.Assert(len(dieNode), Equals, 1) - c.Assert(dieNode[0], Equals, addr) -} diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 7034cb8dc1541..941646b99526a 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -580,7 +580,7 @@ func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan *copRes select { case resp, ok = <-respCh: if it.memTracker != nil && resp != nil { - it.memTracker.Consume(-int64(resp.MemSize())) + it.memTracker.Consume(-resp.MemSize()) } case <-it.finishCh: exit = true @@ -605,7 +605,7 @@ func (sender *copIteratorTaskSender) sendToTaskCh(t *copTask) (exit bool) { func (worker *copIteratorWorker) sendToRespCh(resp *copResponse, respCh chan<- *copResponse, checkOOM bool) (exit bool) { if worker.memTracker != nil && checkOOM { - worker.memTracker.Consume(int64(resp.MemSize())) + worker.memTracker.Consume(resp.MemSize()) } select { case respCh <- resp: @@ -720,12 +720,14 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch if err == nil { cacheKey = cKey cValue := worker.store.coprCache.Get(cKey) + copReq.IsCacheEnabled = true if cValue != nil && cValue.RegionID == task.region.id && cValue.TimeStamp <= worker.req.StartTs { // Append cache version to the request to skip Coprocessor computation if possible // when request result is cached - copReq.IsCacheEnabled = true copReq.CacheIfMatchVersion = cValue.RegionDataVersion cacheValue = cValue + } else { + copReq.CacheIfMatchVersion = 0 } } else { logutil.BgLogger().Warn("Failed to build copr cache key", zap.Error(err)) @@ -1009,7 +1011,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *RPCCon resp.pbResp.Data = data } else { // Cache not hit or cache hit but not valid: update the cache if the response can be cached. - if cacheKey != nil && resp.pbResp.CacheLastVersion > 0 { + if cacheKey != nil && resp.pbResp.CanBeCached && resp.pbResp.CacheLastVersion > 0 { if worker.store.coprCache.CheckAdmission(resp.pbResp.Data.Size(), resp.detail.ProcessTime) { data := make([]byte, len(resp.pbResp.Data)) copy(data, resp.pbResp.Data) diff --git a/store/tikv/delete_range.go b/store/tikv/delete_range.go index dc61009cbd125..009c38b1ad8b6 100644 --- a/store/tikv/delete_range.go +++ b/store/tikv/delete_range.go @@ -72,7 +72,7 @@ func (t *DeleteRangeTask) Execute(ctx context.Context) error { runner := NewRangeTaskRunner(runnerName, t.store, t.concurrency, t.sendReqOnRange) err := runner.RunOnRange(ctx, t.startKey, t.endKey) - t.completedRegions = int(runner.CompletedRegions()) + t.completedRegions = runner.CompletedRegions() return err } diff --git a/store/tikv/delete_range_test.go b/store/tikv/delete_range_test.go index cbb9206917b9e..508413c75910f 100644 --- a/store/tikv/delete_range_test.go +++ b/store/tikv/delete_range_test.go @@ -20,21 +20,23 @@ import ( "sort" . "github.com/pingcap/check" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" ) type testDeleteRangeSuite struct { OneByOneSuite - cluster *mocktikv.Cluster + cluster cluster.Cluster store *tikvStore } var _ = Suite(&testDeleteRangeSuite{}) func (s *testDeleteRangeSuite) SetUpTest(c *C) { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithMultiRegions(s.cluster, []byte("b"), []byte("c"), []byte("d")) - client, pdClient, err := mocktikv.NewTiKVAndPDClient(s.cluster, nil, "") + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithMultiRegions(cluster, []byte("b"), []byte("c"), []byte("d")) + s.cluster = cluster + client, pdClient, err := mocktikv.NewTiKVAndPDClient(cluster, nil, "") c.Assert(err, IsNil) store, err := NewTestTiKVStore(client, pdClient, nil, nil, 0) @@ -118,7 +120,7 @@ func (s *testDeleteRangeSuite) TestDeleteRange(c *C) { // Generate a sequence of keys and random values for _, i := range []byte("abcd") { for j := byte('0'); j <= byte('9'); j++ { - key := []byte{byte(i), byte(j)} + key := []byte{i, j} value := []byte{byte(rand.Intn(256)), byte(rand.Intn(256))} testData[string(key)] = string(value) err := txn.Set(key, value) diff --git a/store/tikv/error.go b/store/tikv/error.go index cc5c87acc0c35..b135614321801 100644 --- a/store/tikv/error.go +++ b/store/tikv/error.go @@ -37,6 +37,7 @@ var ( ErrPDServerTimeout = terror.ClassTiKV.New(mysql.ErrPDServerTimeout, mysql.MySQLErrName[mysql.ErrPDServerTimeout]) ErrRegionUnavailable = terror.ClassTiKV.New(mysql.ErrRegionUnavailable, mysql.MySQLErrName[mysql.ErrRegionUnavailable]) ErrTiKVServerBusy = terror.ClassTiKV.New(mysql.ErrTiKVServerBusy, mysql.MySQLErrName[mysql.ErrTiKVServerBusy]) + ErrTiKVStaleCommand = terror.ClassTiKV.New(mysql.ErrTiKVStaleCommand, mysql.MySQLErrName[mysql.ErrTiKVStaleCommand]) ErrGCTooEarly = terror.ClassTiKV.New(mysql.ErrGCTooEarly, mysql.MySQLErrName[mysql.ErrGCTooEarly]) ErrQueryInterrupted = terror.ClassTiKV.New(mysql.ErrQueryInterrupted, mysql.MySQLErrName[mysql.ErrQueryInterrupted]) ErrLockAcquireFailAndNoWaitSet = terror.ClassTiKV.New(mysql.ErrLockAcquireFailAndNoWaitSet, mysql.MySQLErrName[mysql.ErrLockAcquireFailAndNoWaitSet]) diff --git a/store/tikv/gcworker/gc_worker.go b/store/tikv/gcworker/gc_worker.go index 5a9c43cd7cfee..0386f39885df2 100644 --- a/store/tikv/gcworker/gc_worker.go +++ b/store/tikv/gcworker/gc_worker.go @@ -18,6 +18,7 @@ import ( "container/heap" "context" "fmt" + "math" "os" "sort" "strconv" @@ -134,10 +135,12 @@ const ( gcScanLockModeKey = "tikv_gc_scan_lock_mode" gcScanLockModeLegacy = "legacy" gcScanLockModePhysical = "physical" - gcScanLockModeDefault = gcScanLockModeLegacy + gcScanLockModeDefault = gcScanLockModePhysical gcAutoConcurrencyKey = "tikv_gc_auto_concurrency" gcDefaultAutoConcurrency = true + + gcWorkerServiceSafePointID = "gc_worker" ) var gcSafePointCacheInterval = tikv.GcSafePointCacheInterval @@ -183,7 +186,6 @@ func (w *GCWorker) start(ctx context.Context, wg *sync.WaitGroup) { w.lastFinish = time.Now() if err != nil { logutil.Logger(ctx).Error("[gc worker] runGCJob", zap.Error(err)) - return } case <-ctx.Done(): logutil.Logger(ctx).Info("[gc worker] quit", zap.String("uuid", w.uuid)) @@ -312,7 +314,7 @@ func (w *GCWorker) checkPrepare(ctx context.Context) (bool, uint64, error) { if err != nil || !ok { return false, 0, errors.Trace(err) } - newSafePoint, err := w.calculateNewSafePoint(now) + newSafePoint, newSafePointValue, err := w.calculateNewSafePoint(ctx, now) if err != nil || newSafePoint == nil { return false, 0, errors.Trace(err) } @@ -324,7 +326,7 @@ func (w *GCWorker) checkPrepare(ctx context.Context) (bool, uint64, error) { if err != nil { return false, 0, errors.Trace(err) } - return true, oracle.ComposeTS(oracle.GetPhysical(*newSafePoint), 0), nil + return true, newSafePointValue, nil } // calculateNewSafePoint uses the current global transaction min start timestamp to calculate the new safe point. @@ -462,21 +464,29 @@ func (w *GCWorker) validateGCLiftTime(lifeTime time.Duration) (time.Duration, er return gcMinLifeTime, err } -func (w *GCWorker) calculateNewSafePoint(now time.Time) (*time.Time, error) { +func (w *GCWorker) calculateNewSafePoint(ctx context.Context, now time.Time) (*time.Time, uint64, error) { lifeTime, err := w.loadDurationWithDefault(gcLifeTimeKey, gcDefaultLifeTime) if err != nil { - return nil, errors.Trace(err) + return nil, 0, errors.Trace(err) } *lifeTime, err = w.validateGCLiftTime(*lifeTime) if err != nil { - return nil, err + return nil, 0, err } metrics.GCConfigGauge.WithLabelValues(gcLifeTimeKey).Set(lifeTime.Seconds()) lastSafePoint, err := w.loadTime(gcSafePointKey) if err != nil { - return nil, errors.Trace(err) + return nil, 0, errors.Trace(err) } safePoint := w.calSafePointByMinStartTS(now.Add(-*lifeTime)) + + safePointValue := oracle.ComposeTS(oracle.GetPhysical(safePoint), 0) + safePointValue, err = w.setGCWorkerServiceSafePoint(ctx, safePointValue) + safePoint = oracle.GetTimeFromTS(safePointValue) + + if err != nil { + return nil, 0, errors.Trace(err) + } // We should never decrease safePoint. if lastSafePoint != nil && safePoint.Before(*lastSafePoint) { logutil.BgLogger().Info("[gc worker] last safe point is later than current one."+ @@ -485,9 +495,33 @@ func (w *GCWorker) calculateNewSafePoint(now time.Time) (*time.Time, error) { zap.String("leaderTick on", w.uuid), zap.Time("last safe point", *lastSafePoint), zap.Time("current safe point", safePoint)) - return nil, nil + return nil, 0, nil } - return &safePoint, nil + return &safePoint, safePointValue, nil +} + +// setGCWorkerServiceSafePoint sets the given safePoint as TiDB's service safePoint to PD, and returns the current minimal +// service safePoint among all services. +func (w *GCWorker) setGCWorkerServiceSafePoint(ctx context.Context, safePoint uint64) (uint64, error) { + // Sets TTL to MAX to make it permanently valid. + minSafePoint, err := w.pdClient.UpdateServiceGCSafePoint(ctx, gcWorkerServiceSafePointID, math.MaxInt64, safePoint) + if err != nil { + logutil.Logger(ctx).Error("[gc worker] failed to update service safe point", + zap.String("uuid", w.uuid), + zap.Error(err)) + metrics.GCJobFailureCounter.WithLabelValues("update_service_safe_point").Inc() + return 0, errors.Trace(err) + } + if minSafePoint < safePoint { + logutil.Logger(ctx).Info("[gc worker] there's another service in the cluster requires an earlier safe point. "+ + "gc will continue with the earlier one", + zap.String("uuid", w.uuid), + zap.Uint64("ourSafePoint", safePoint), + zap.Uint64("minSafePoint", minSafePoint), + ) + safePoint = minSafePoint + } + return safePoint, nil } func (w *GCWorker) runGCJob(ctx context.Context, safePoint uint64, concurrency int) error { @@ -891,7 +925,6 @@ func (w *GCWorker) resolveLocks(ctx context.Context, safePoint uint64, concurren // First try resolve locks with physical scan err := w.resolveLocksPhysical(ctx, safePoint) - if err == nil { return nil } @@ -1037,27 +1070,31 @@ func (w *GCWorker) resolveLocksPhysical(ctx context.Context, safePoint uint64) e zap.Uint64("safePoint", safePoint)) startTime := time.Now() - stores, err := w.getUpStoresMapForGC(ctx) - if err != nil { - return errors.Trace(err) - } - - defer func() { - w.removeLockObservers(ctx, safePoint, stores) - }() + registeredStores := make(map[uint64]*metapb.Store) + defer w.removeLockObservers(ctx, safePoint, registeredStores) - err = w.registerLockObservers(ctx, safePoint, stores) + dirtyStores, err := w.getUpStoresMapForGC(ctx) if err != nil { return errors.Trace(err) } for retry := 0; retry < 3; retry++ { - resolvedStores, err := w.physicalScanAndResolveLocks(ctx, safePoint, stores) + err = w.registerLockObservers(ctx, safePoint, dirtyStores) if err != nil { return errors.Trace(err) } + for id, store := range dirtyStores { + registeredStores[id] = store + } - stores, err = w.getUpStoresMapForGC(ctx) + resolvedStores, err := w.physicalScanAndResolveLocks(ctx, safePoint, dirtyStores) + if err != nil { + return errors.Trace(err) + } + + failpoint.Inject("beforeCheckLockObservers", func() {}) + + stores, err := w.getUpStoresMapForGC(ctx) if err != nil { return errors.Trace(err) } @@ -1067,22 +1104,38 @@ func (w *GCWorker) resolveLocksPhysical(ctx context.Context, safePoint uint64) e return errors.Trace(err) } - // Remove clean stores from the set - for resolvedStore := range resolvedStores { - // Only stores that are both resolved and checked is clean. - // For each clean store, remove it from the stores set. - if _, ok := checkedStores[resolvedStore]; ok { - delete(stores, resolvedStore) + for store := range stores { + if _, ok := checkedStores[store]; ok { + // The store is resolved and checked. + if _, ok := resolvedStores[store]; ok { + delete(stores, store) + } + // The store is checked and has been resolved before. + if _, ok := dirtyStores[store]; !ok { + delete(stores, store) + } + // If the store is checked and not resolved, we can retry to resolve it again, so leave it in dirtyStores. + } else if _, ok := registeredStores[store]; ok { + // The store has been registered and it's dirty due to too many collected locks. Fall back to legacy mode. + // We can't remove the lock observer from the store and retry the whole procedure because if the store + // receives duplicated remove and register requests during resolving locks, the store will be cleaned + // when checking but the lock observer drops some locks. It may results in missing locks. + return errors.Errorf("store %v is dirty", store) } } + dirtyStores = stores // If there are still dirty stores, continue the loop to clean them again. // Only dirty stores will be scanned in the next loop. - if len(stores) == 0 { + if len(dirtyStores) == 0 { break } } + if len(dirtyStores) != 0 { + return errors.Errorf("still has %d dirty stores after physical resolve locks", len(dirtyStores)) + } + logutil.Logger(ctx).Info("[gc worker] finish resolve locks with physical scan locks", zap.String("uuid", w.uuid), zap.Uint64("safePoint", safePoint), @@ -1107,7 +1160,9 @@ func (w *GCWorker) registerLockObservers(ctx context.Context, safePoint uint64, if err != nil { return errors.Trace(err) } - + if resp.Resp == nil { + return errors.Trace(tikv.ErrBodyMissing) + } errStr := resp.Resp.(*kvrpcpb.RegisterLockObserverResponse).Error if len(errStr) > 0 { return errors.Errorf("register lock observer on store %v returns error: %v", store.Id, errStr) @@ -1127,31 +1182,41 @@ func (w *GCWorker) checkLockObservers(ctx context.Context, safePoint uint64, sto req := tikvrpc.NewRequest(tikvrpc.CmdCheckLockObserver, &kvrpcpb.CheckLockObserverRequest{ MaxTs: safePoint, }) - cleanStores := make(map[uint64]interface{}, len(stores)) + logError := func(store *metapb.Store, err error) { + logutil.Logger(ctx).Error("[gc worker] failed to check lock observer for store", + zap.String("uuid", w.uuid), + zap.Any("store", store), + zap.Error(err)) + } + // When error occurs, this function doesn't fail immediately, but continues without adding the failed store to // cleanStores set. - for _, store := range stores { address := store.Address resp, err := w.store.GetTiKVClient().SendRequest(ctx, address, req, tikv.AccessLockObserverTimeout) if err != nil { - logutil.Logger(ctx).Error("[gc worker] failed to check lock observer for store", - zap.String("uuid", w.uuid), - zap.Any("store", store), - zap.Error(err)) + logError(store, err) + continue + } + if resp.Resp == nil { + logError(store, tikv.ErrBodyMissing) continue } - respInner := resp.Resp.(*kvrpcpb.CheckLockObserverResponse) if len(respInner.Error) > 0 { err = errors.Errorf("check lock observer on store %v returns error: %v", store.Id, respInner.Error) - logutil.Logger(ctx).Error("[gc worker] failed to check lock observer for store", + logError(store, err) + continue + } + + // No need to resolve observed locks on uncleaned stores. + if !respInner.IsClean { + logutil.Logger(ctx).Warn("[gc worker] check lock observer: store is not clean", zap.String("uuid", w.uuid), - zap.Any("store", store), - zap.Error(err)) + zap.Any("store", store)) continue } @@ -1168,21 +1233,11 @@ func (w *GCWorker) checkLockObservers(ctx context.Context, safePoint uint64, sto if err != nil { err = errors.Errorf("check lock observer on store %v returns error: %v", store.Id, respInner.Error) - logutil.Logger(ctx).Error("[gc worker] failed to check lock observer for store", - zap.String("uuid", w.uuid), - zap.Any("store", store), - zap.Error(err)) + logError(store, err) continue } } - - if respInner.IsClean { - cleanStores[store.Id] = nil - } else { - logutil.Logger(ctx).Warn("[gc worker] check lock observer: store is not clean", - zap.String("uuid", w.uuid), - zap.Any("store", store)) - } + cleanStores[store.Id] = nil } return cleanStores, nil @@ -1197,86 +1252,89 @@ func (w *GCWorker) removeLockObservers(ctx context.Context, safePoint uint64, st MaxTs: safePoint, }) + logError := func(store *metapb.Store, err error) { + logutil.Logger(ctx).Warn("[gc worker] failed to remove lock observer from store", + zap.String("uuid", w.uuid), + zap.Any("store", store), + zap.Error(err)) + } + for _, store := range stores { address := store.Address resp, err := w.store.GetTiKVClient().SendRequest(ctx, address, req, tikv.AccessLockObserverTimeout) if err != nil { - logutil.Logger(ctx).Warn("[gc worker] failed to remove lock observer from store", - zap.String("uuid", w.uuid), - zap.Any("store", store), - zap.Error(err)) + logError(store, err) + continue + } + if resp.Resp == nil { + logError(store, tikv.ErrBodyMissing) continue } - errStr := resp.Resp.(*kvrpcpb.RemoveLockObserverResponse).Error if len(errStr) > 0 { err = errors.Errorf("remove lock observer on store %v returns error: %v", store.Id, errStr) - logutil.Logger(ctx).Error("[gc worker] failed to remove lock observer from store", - zap.String("uuid", w.uuid), - zap.Any("store", store), - zap.Error(err)) + logError(store, err) } } } // physicalScanAndResolveLocks performs physical scan lock and resolves these locks. Returns successful stores func (w *GCWorker) physicalScanAndResolveLocks(ctx context.Context, safePoint uint64, stores map[uint64]*metapb.Store) (map[uint64]interface{}, error) { + ctx, cancel := context.WithCancel(ctx) + // Cancel all spawned goroutines for lock scanning and resolving. + defer cancel() + scanner := newMergeLockScanner(safePoint, w.store.GetTiKVClient(), stores) err := scanner.Start(ctx) if err != nil { return nil, errors.Trace(err) } - innerCtx, cancel := context.WithCancel(ctx) - wg := &sync.WaitGroup{} - defer func() { - cancel() - wg.Wait() - }() - taskCh := make(chan []*tikv.Lock, len(stores)) errCh := make(chan error, len(stores)) + wg := &sync.WaitGroup{} for range stores { wg.Add(1) go func() { defer wg.Done() for { select { - case <-innerCtx.Done(): - return case locks, ok := <-taskCh: if !ok { - // Closed + // All locks have been resolved. return } - err := w.resolveLocksAcrossRegions(innerCtx, locks) + err := w.resolveLocksAcrossRegions(ctx, locks) if err != nil { - logutil.Logger(innerCtx).Error("resolve locks failed", zap.Error(err)) + logutil.Logger(ctx).Error("resolve locks failed", zap.Error(err)) errCh <- err } + case <-ctx.Done(): + return } } }() } for { - select { - case err := <-errCh: - return nil, errors.Trace(err) - default: - } - locks := scanner.NextBatch(128) if len(locks) == 0 { break } - taskCh <- locks + select { + case taskCh <- locks: + case err := <-errCh: + return nil, errors.Trace(err) + case <-ctx.Done(): + return nil, ctx.Err() + } } close(taskCh) + // Wait for all locks resolved. wg.Wait() select { @@ -1289,6 +1347,12 @@ func (w *GCWorker) physicalScanAndResolveLocks(ctx context.Context, safePoint ui } func (w *GCWorker) resolveLocksAcrossRegions(ctx context.Context, locks []*tikv.Lock) error { + failpoint.Inject("resolveLocksAcrossRegionsErr", func(v failpoint.Value) { + ms := v.(int) + time.Sleep(time.Duration(ms) * time.Millisecond) + failpoint.Return(errors.New("injectedError")) + }) + bo := tikv.NewBackoffer(ctx, tikv.GcResolveLockMaxBackoff) for { @@ -1660,19 +1724,25 @@ func (w *GCWorker) saveValueToSysTable(key, value string) error { } // RunGCJob sends GC command to KV. It is exported for kv api, do not use it with GCWorker at the same time. -func RunGCJob(ctx context.Context, s tikv.Storage, safePoint uint64, identifier string, concurrency int) error { +func RunGCJob(ctx context.Context, s tikv.Storage, pd pd.Client, safePoint uint64, identifier string, concurrency int) error { gcWorker := &GCWorker{ - store: s, - uuid: identifier, + store: s, + uuid: identifier, + pdClient: pd, } - err := gcWorker.resolveLocks(ctx, safePoint, concurrency, false) + if concurrency <= 0 { + return errors.Errorf("[gc worker] gc concurrency should greater than 0, current concurrency: %v", concurrency) + } + + safePoint, err := gcWorker.setGCWorkerServiceSafePoint(ctx, safePoint) if err != nil { return errors.Trace(err) } - if concurrency <= 0 { - return errors.Errorf("[gc worker] gc concurrency should greater than 0, current concurrency: %v", concurrency) + err = gcWorker.resolveLocks(ctx, safePoint, concurrency, false) + if err != nil { + return errors.Trace(err) } err = gcWorker.saveSafePoint(gcWorker.store.GetSafePointKV(), safePoint) @@ -1681,7 +1751,6 @@ func RunGCJob(ctx context.Context, s tikv.Storage, safePoint uint64, identifier } // Sleep to wait for all other tidb instances update their safepoint cache. time.Sleep(gcSafePointCacheInterval) - err = gcWorker.doGC(ctx, safePoint, concurrency) if err != nil { return errors.Trace(err) @@ -1699,7 +1768,12 @@ func RunDistributedGCJob(ctx context.Context, s tikv.Storage, pd pd.Client, safe pdClient: pd, } - err := gcWorker.resolveLocks(ctx, safePoint, concurrency, false) + safePoint, err := gcWorker.setGCWorkerServiceSafePoint(ctx, safePoint) + if err != nil { + return errors.Trace(err) + } + + err = gcWorker.resolveLocks(ctx, safePoint, concurrency, false) if err != nil { return errors.Trace(err) } @@ -1756,12 +1830,12 @@ const scanLockResultBufferSize = 128 // mergeLockScanner is used to scan specified stores by using PhysicalScanLock. For multiple stores, the scanner will // merge the scan results of each store, and remove the duplicating items from different stores. type mergeLockScanner struct { - safePoint uint64 - client tikv.Client - stores map[uint64]*metapb.Store - receivers mergeReceiver - currentLockKey []byte - scanLockLimit uint32 + safePoint uint64 + client tikv.Client + stores map[uint64]*metapb.Store + receivers mergeReceiver + currentLock *tikv.Lock + scanLockLimit uint32 } type receiver struct { @@ -1809,8 +1883,8 @@ func (r mergeReceiver) Less(i, j int) bool { // lhs != nil, so lhs < rhs return true } - - return bytes.Compare(lhs.Key, rhs.Key) < 0 + ord := bytes.Compare(lhs.Key, rhs.Key) + return ord < 0 || (ord == 0 && lhs.TxnID < rhs.TxnID) } func (r mergeReceiver) Swap(i, j int) { @@ -1858,7 +1932,11 @@ func (s *mergeLockScanner) Start(ctx context.Context) error { zap.Uint64("safePoint", s.safePoint), zap.Any("store", store1), zap.Error(err)) - ch <- scanLockResult{Err: err} + + select { + case ch <- scanLockResult{Err: err}: + case <-ctx.Done(): + } } }() receivers = append(receivers, &receiver{Ch: ch, StoreID: storeID}) @@ -1876,15 +1954,15 @@ func (s *mergeLockScanner) startWithReceivers(receivers []*receiver) { func (s *mergeLockScanner) Next() *tikv.Lock { for { - nextReceiver := heap.Pop(&s.receivers).(*receiver) + nextReceiver := s.receivers[0] nextLock := nextReceiver.TakeNextLock() - heap.Push(&s.receivers, nextReceiver) + heap.Fix(&s.receivers, 0) if nextLock == nil { return nil } - if s.currentLockKey == nil || !bytes.Equal(s.currentLockKey, nextLock.Key) { - s.currentLockKey = nextLock.Key + if s.currentLock == nil || !bytes.Equal(s.currentLock.Key, nextLock.Key) || s.currentLock.TxnID != nextLock.TxnID { + s.currentLock = nextLock return nextLock } } @@ -1895,7 +1973,7 @@ func (s *mergeLockScanner) NextBatch(batchSize int) []*tikv.Lock { for len(result) < batchSize { lock := s.Next() if lock == nil { - return result + break } result = append(result, lock) } @@ -1929,12 +2007,10 @@ func (s *mergeLockScanner) physicalScanLocksForStore(ctx context.Context, safePo if err != nil { return errors.Trace(err) } - - resp := response.Resp.(*kvrpcpb.PhysicalScanLockResponse) - if resp == nil { - return errors.Errorf("physical scan lock response is nil") + if response.Resp == nil { + return errors.Trace(tikv.ErrBodyMissing) } - + resp := response.Resp.(*kvrpcpb.PhysicalScanLockResponse) if len(resp.Error) > 0 { return errors.Errorf("physical scan lock received error from store: %v", resp.Error) } @@ -1947,7 +2023,11 @@ func (s *mergeLockScanner) physicalScanLocksForStore(ctx context.Context, safePo nextKey = append(nextKey, 0) for _, lockInfo := range resp.Locks { - lockCh <- scanLockResult{Lock: tikv.NewLock(lockInfo)} + select { + case lockCh <- scanLockResult{Lock: tikv.NewLock(lockInfo)}: + case <-ctx.Done(): + return ctx.Err() + } } if len(resp.Locks) < int(s.scanLockLimit) { diff --git a/store/tikv/gcworker/gc_worker_test.go b/store/tikv/gcworker/gc_worker_test.go index 5901a7ae9ae99..1a5cb78bf20fc 100644 --- a/store/tikv/gcworker/gc_worker_test.go +++ b/store/tikv/gcworker/gc_worker_test.go @@ -20,6 +20,8 @@ import ( "math" "sort" "strconv" + "sync" + "sync/atomic" "testing" "time" @@ -38,6 +40,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockoracle" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" @@ -50,7 +53,7 @@ func TestT(t *testing.T) { type testGCWorkerSuite struct { store tikv.Storage - cluster *mocktikv.Cluster + cluster cluster.Cluster oracle *mockoracle.MockOracle gcWorker *GCWorker dom *domain.Domain @@ -71,10 +74,11 @@ func (s *testGCWorkerSuite) SetUpTest(c *C) { return client } - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithMultiStores(s.cluster, 3) + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithMultiStores(cluster, 3) + s.cluster = cluster store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), + mockstore.WithCluster(cluster), mockstore.WithHijackClient(hijackClient)) s.store = store.(tikv.Storage) @@ -84,7 +88,7 @@ func (s *testGCWorkerSuite) SetUpTest(c *C) { s.dom, err = session.BootstrapSession(s.store) c.Assert(err, IsNil) - s.pdClient = mocktikv.NewPDClient(s.cluster) + s.pdClient = mocktikv.NewPDClient(cluster) gcWorker, err := NewGCWorker(s.store, s.pdClient) c.Assert(err, IsNil) gcWorker.Start() @@ -140,6 +144,34 @@ func (s *testGCWorkerSuite) mustGetSafePointFromPd(c *C) uint64 { return safePoint } +func (s *testGCWorkerSuite) mustGetMinServiceSafePointFromPd(c *C) uint64 { + // UpdateServiceGCSafePoint returns the minimal service safePoint. If trying to update it with a value less than the + // current minimal safePoint, nothing will be updated and the current minimal one will be returned. So we can use + // this API to check the current safePoint. + // This function shouldn't be invoked when there's no service safePoint set. + minSafePoint, err := s.pdClient.UpdateServiceGCSafePoint(context.Background(), "test", 0, 0) + c.Assert(err, IsNil) + return minSafePoint +} + +func (s *testGCWorkerSuite) mustUpdateServiceGCSafePoint(c *C, serviceID string, safePoint, expectedMinSafePoint uint64) { + minSafePoint, err := s.pdClient.UpdateServiceGCSafePoint(context.Background(), serviceID, math.MaxInt64, safePoint) + c.Assert(err, IsNil) + c.Assert(minSafePoint, Equals, expectedMinSafePoint) +} + +func (s *testGCWorkerSuite) mustRemoveServiceGCSafePoint(c *C, serviceID string, safePoint, expectedMinSafePoint uint64) { + minSafePoint, err := s.pdClient.UpdateServiceGCSafePoint(context.Background(), serviceID, 0, safePoint) + c.Assert(err, IsNil) + c.Assert(minSafePoint, Equals, expectedMinSafePoint) +} + +func (s *testGCWorkerSuite) mustSetTiDBServiceSafePoint(c *C, safePoint, expectedMinSafePoint uint64) { + minSafePoint, err := s.gcWorker.setGCWorkerServiceSafePoint(context.Background(), safePoint) + c.Assert(err, IsNil) + c.Assert(minSafePoint, Equals, expectedMinSafePoint) +} + // gcProbe represents a key that contains multiple versions, one of which should be collected. Execution of GC with // greater ts will be detected, but it may not work properly if there are newer versions of the key. // This is not used to check the correctness of GC algorithm, but only for checking whether GC has been executed on the @@ -440,7 +472,7 @@ func (s *testGCWorkerSuite) TestCheckGCMode(c *C) { func (s *testGCWorkerSuite) TestCheckScanLockMode(c *C) { usePhysical, err := s.gcWorker.checkUsePhysicalScanLock() c.Assert(err, IsNil) - c.Assert(usePhysical, Equals, false) + c.Assert(usePhysical, Equals, true) // This is a hidden config, so default value will not be inserted to table. str, err := s.gcWorker.loadValueFromSysTable(gcScanLockModeKey) c.Assert(err, IsNil) @@ -666,8 +698,11 @@ Loop: type testGCWorkerClient struct { tikv.Client - unsafeDestroyRangeHandler handler - physicalScanLockHandler handler + unsafeDestroyRangeHandler handler + physicalScanLockHandler handler + registerLockObserverHandler handler + checkLockObserverHandler handler + removeLockObserverHandler handler } type handler = func(addr string, req *tikvrpc.Request) (*tikvrpc.Response, error) @@ -679,6 +714,15 @@ func (c *testGCWorkerClient) SendRequest(ctx context.Context, addr string, req * if req.Type == tikvrpc.CmdPhysicalScanLock && c.physicalScanLockHandler != nil { return c.physicalScanLockHandler(addr, req) } + if req.Type == tikvrpc.CmdRegisterLockObserver && c.registerLockObserverHandler != nil { + return c.registerLockObserverHandler(addr, req) + } + if req.Type == tikvrpc.CmdCheckLockObserver && c.checkLockObserverHandler != nil { + return c.checkLockObserverHandler(addr, req) + } + if req.Type == tikvrpc.CmdRemoveLockObserver && c.removeLockObserverHandler != nil { + return c.removeLockObserverHandler(addr, req) + } return c.Client.SendRequest(ctx, addr, req, timeout) } @@ -821,12 +865,50 @@ func (s *testGCWorkerSuite) TestRunGCJob(c *C) { c.Assert(etcdSafePoint, Equals, safePoint) } +func (s *testGCWorkerSuite) TestSetServiceSafePoint(c *C) { + // SafePoint calculations are based on time rather than ts value. + safePoint := s.mustAllocTs(c) + s.mustSetTiDBServiceSafePoint(c, safePoint, safePoint) + c.Assert(s.mustGetMinServiceSafePointFromPd(c), Equals, safePoint) + + // Advance the service safe point + safePoint += 100 + s.mustSetTiDBServiceSafePoint(c, safePoint, safePoint) + c.Assert(s.mustGetMinServiceSafePointFromPd(c), Equals, safePoint) + + // It doesn't matter if there is a greater safePoint from other services. + safePoint += 100 + // Returns the last service safePoint that were uploaded. + s.mustUpdateServiceGCSafePoint(c, "svc1", safePoint+10, safePoint-100) + s.mustSetTiDBServiceSafePoint(c, safePoint, safePoint) + c.Assert(s.mustGetMinServiceSafePointFromPd(c), Equals, safePoint) + + // Test the case when there is a smaller safePoint from other services. + safePoint += 100 + // Returns the last service safePoint that were uploaded. + s.mustUpdateServiceGCSafePoint(c, "svc1", safePoint-10, safePoint-100) + s.mustSetTiDBServiceSafePoint(c, safePoint, safePoint-10) + c.Assert(s.mustGetMinServiceSafePointFromPd(c), Equals, safePoint-10) + + // Test removing the minimum service safe point. + s.mustRemoveServiceGCSafePoint(c, "svc1", safePoint-10, safePoint) + c.Assert(s.mustGetMinServiceSafePointFromPd(c), Equals, safePoint) + + // Test the case when there are many safePoints. + safePoint += 100 + for i := 0; i < 10; i++ { + svcName := fmt.Sprintf("svc%d", i) + s.mustUpdateServiceGCSafePoint(c, svcName, safePoint+uint64(i)*10, safePoint-100) + } + s.mustSetTiDBServiceSafePoint(c, safePoint+50, safePoint) +} + func (s *testGCWorkerSuite) TestRunGCJobAPI(c *C) { gcSafePointCacheInterval = 0 p := s.createGCProbe(c, "k1") safePoint := s.mustAllocTs(c) - err := RunGCJob(context.Background(), s.store, safePoint, "mock", 1) + err := RunGCJob(context.Background(), s.store, s.pdClient, safePoint, "mock", 1) c.Assert(err, IsNil) s.checkCollected(c, p) etcdSafePoint := s.loadEtcdSafePoint(c) @@ -859,6 +941,7 @@ func makeMergedChannel(c *C, count int) (*mergeLockScanner, []chan scanLockResul scanner := &mergeLockScanner{} channels := make([]chan scanLockResult, 0, count) receivers := make([]*receiver, 0, count) + storeIDs := make([]uint64, 0, count) for i := 0; i < count; i++ { ch := make(chan scanLockResult, 10) @@ -869,6 +952,7 @@ func makeMergedChannel(c *C, count int) (*mergeLockScanner, []chan scanLockResul channels = append(channels, ch) receivers = append(receivers, receiver) + storeIDs = append(storeIDs, uint64(i)) } resultCh := make(chan []*tikv.Lock) @@ -881,11 +965,6 @@ func makeMergedChannel(c *C, count int) (*mergeLockScanner, []chan scanLockResul resultCh <- result }() - storeIDs := make([]uint64, count) - for i := 0; i < count; i++ { - storeIDs[i] = uint64(i) - } - return scanner, channels, storeIDs, resultCh } @@ -926,7 +1005,7 @@ func (s *testGCWorkerSuite) makeMergedMockClient(c *C, count int) (*mergeLockSca locks = nil break } - lockInfo := &kvrpcpb.LockInfo{Key: res.Lock.Key} + lockInfo := &kvrpcpb.LockInfo{Key: res.Lock.Key, LockVersion: res.Lock.TxnID} locks = append(locks, lockInfo) } @@ -967,53 +1046,86 @@ func (s *testGCWorkerSuite) TestMergeLockScanner(c *C) { return res } - makeLock := func(key string) *tikv.Lock { - return &tikv.Lock{Key: []byte(key)} + makeLock := func(key string, ts uint64) *tikv.Lock { + return &tikv.Lock{Key: []byte(key), TxnID: ts} } - makeLockList := func(keys ...string) []*tikv.Lock { + makeLockList := func(locks ...*tikv.Lock) []*tikv.Lock { + res := make([]*tikv.Lock, 0, len(locks)) + for _, lock := range locks { + res = append(res, lock) + } + return res + } + + makeLockListByKey := func(keys ...string) []*tikv.Lock { res := make([]*tikv.Lock, 0, len(keys)) - for _, k := range keys { - res = append(res, makeLock(k)) + for _, key := range keys { + res = append(res, makeLock(key, 0)) } return res } - sendLocks := func(ch chan<- scanLockResult, keys ...string) { - for _, k := range keys { - ch <- scanLockResult{Lock: makeLock(k)} + sendLocks := func(ch chan<- scanLockResult, locks ...*tikv.Lock) { + for _, lock := range locks { + ch <- scanLockResult{Lock: lock} } } + sendLocksByKey := func(ch chan<- scanLockResult, keys ...string) []*tikv.Lock { + locks := make([]*tikv.Lock, 0, len(keys)) + for _, key := range keys { + locks = append(locks, makeLock(key, 0)) + } + sendLocks(ch, locks...) + return locks + } + sendErr := func(ch chan<- scanLockResult) { ch <- scanLockResult{Err: errors.New("error")} } + // No lock. scanner, sendCh, storeIDs, resCh := makeMergedChannel(c, 1) close(sendCh[0]) c.Assert(len(<-resCh), Equals, 0) c.Assert(scanner.GetSucceededStores(), DeepEquals, makeIDSet(storeIDs, 0)) scanner, sendCh, storeIDs, resCh = makeMergedChannel(c, 1) - sendLocks(sendCh[0], "a", "b", "c") + locks := sendLocksByKey(sendCh[0], "a", "b", "c") close(sendCh[0]) - c.Assert(<-resCh, DeepEquals, makeLockList("a", "b", "c")) + c.Assert(<-resCh, DeepEquals, locks) c.Assert(scanner.GetSucceededStores(), DeepEquals, makeIDSet(storeIDs, 0)) + // Send locks with error scanner, sendCh, storeIDs, resCh = makeMergedChannel(c, 1) - sendLocks(sendCh[0], "a", "b", "c") + locks = sendLocksByKey(sendCh[0], "a", "b", "c") sendErr(sendCh[0]) close(sendCh[0]) - c.Assert(<-resCh, DeepEquals, makeLockList("a", "b", "c")) + c.Assert(<-resCh, DeepEquals, locks) c.Assert(scanner.GetSucceededStores(), DeepEquals, makeIDSet(storeIDs)) + // Merge sort locks with different keys. + scanner, sendCh, storeIDs, resCh = makeMergedChannel(c, 2) + locks = sendLocksByKey(sendCh[0], "a", "c", "e") + time.Sleep(time.Millisecond * 100) + locks = append(locks, sendLocksByKey(sendCh[1], "b", "d", "f")...) + close(sendCh[0]) + close(sendCh[1]) + sort.Slice(locks, func(i, j int) bool { + return bytes.Compare(locks[i].Key, locks[j].Key) < 0 + }) + c.Assert(<-resCh, DeepEquals, locks) + c.Assert(scanner.GetSucceededStores(), DeepEquals, makeIDSet(storeIDs, 0, 1)) + + // Merge sort locks with different timestamps. scanner, sendCh, storeIDs, resCh = makeMergedChannel(c, 2) - sendLocks(sendCh[0], "a", "c", "e") + sendLocks(sendCh[0], makeLock("a", 0), makeLock("a", 1)) time.Sleep(time.Millisecond * 100) - sendLocks(sendCh[1], "b", "d", "f") + sendLocks(sendCh[1], makeLock("a", 1), makeLock("a", 2), makeLock("b", 0)) close(sendCh[0]) close(sendCh[1]) - c.Assert(<-resCh, DeepEquals, makeLockList("a", "b", "c", "d", "e", "f")) + c.Assert(<-resCh, DeepEquals, makeLockList(makeLock("a", 0), makeLock("a", 1), makeLock("a", 2), makeLock("b", 0))) c.Assert(scanner.GetSucceededStores(), DeepEquals, makeIDSet(storeIDs, 0, 1)) for _, useMock := range []bool{false, true} { @@ -1023,38 +1135,263 @@ func (s *testGCWorkerSuite) TestMergeLockScanner(c *C) { } scanner, sendCh, storeIDs, resCh = channel(c, 3) - sendLocks(sendCh[0], "a", "d", "g", "h") + sendLocksByKey(sendCh[0], "a", "d", "g", "h") time.Sleep(time.Millisecond * 100) - sendLocks(sendCh[1], "a", "d", "f", "h") + sendLocksByKey(sendCh[1], "a", "d", "f", "h") time.Sleep(time.Millisecond * 100) - sendLocks(sendCh[2], "b", "c", "e", "h") + sendLocksByKey(sendCh[2], "b", "c", "e", "h") close(sendCh[0]) close(sendCh[1]) close(sendCh[2]) - c.Assert(<-resCh, DeepEquals, makeLockList("a", "b", "c", "d", "e", "f", "g", "h")) + c.Assert(<-resCh, DeepEquals, makeLockListByKey("a", "b", "c", "d", "e", "f", "g", "h")) c.Assert(scanner.GetSucceededStores(), DeepEquals, makeIDSet(storeIDs, 0, 1, 2)) scanner, sendCh, storeIDs, resCh = channel(c, 3) - sendLocks(sendCh[0], "a", "d", "g", "h") + sendLocksByKey(sendCh[0], "a", "d", "g", "h") time.Sleep(time.Millisecond * 100) - sendLocks(sendCh[1], "a", "d", "f", "h") + sendLocksByKey(sendCh[1], "a", "d", "f", "h") time.Sleep(time.Millisecond * 100) - sendLocks(sendCh[2], "b", "c", "e", "h") + sendLocksByKey(sendCh[2], "b", "c", "e", "h") sendErr(sendCh[0]) close(sendCh[0]) close(sendCh[1]) close(sendCh[2]) - c.Assert(<-resCh, DeepEquals, makeLockList("a", "b", "c", "d", "e", "f", "g", "h")) + c.Assert(<-resCh, DeepEquals, makeLockListByKey("a", "b", "c", "d", "e", "f", "g", "h")) c.Assert(scanner.GetSucceededStores(), DeepEquals, makeIDSet(storeIDs, 1, 2)) scanner, sendCh, storeIDs, resCh = channel(c, 3) - sendLocks(sendCh[0], "a\x00", "a\x00\x00", "b", "b\x00") - sendLocks(sendCh[1], "a", "a\x00\x00", "a\x00\x00\x00", "c") - sendLocks(sendCh[2], "1", "a\x00", "a\x00\x00", "b") + sendLocksByKey(sendCh[0], "a\x00", "a\x00\x00", "b", "b\x00") + sendLocksByKey(sendCh[1], "a", "a\x00\x00", "a\x00\x00\x00", "c") + sendLocksByKey(sendCh[2], "1", "a\x00", "a\x00\x00", "b") close(sendCh[0]) close(sendCh[1]) close(sendCh[2]) - c.Assert(<-resCh, DeepEquals, makeLockList("1", "a", "a\x00", "a\x00\x00", "a\x00\x00\x00", "b", "b\x00", "c")) + c.Assert(<-resCh, DeepEquals, makeLockListByKey("1", "a", "a\x00", "a\x00\x00", "a\x00\x00\x00", "b", "b\x00", "c")) c.Assert(scanner.GetSucceededStores(), DeepEquals, makeIDSet(storeIDs, 0, 1, 2)) + + scanner, sendCh, storeIDs, resCh = channel(c, 3) + sendLocks(sendCh[0], makeLock("a", 0), makeLock("d", 0), makeLock("g", 0), makeLock("h", 0)) + sendLocks(sendCh[1], makeLock("a", 1), makeLock("b", 0), makeLock("c", 0), makeLock("d", 1)) + sendLocks(sendCh[2], makeLock("e", 0), makeLock("g", 1), makeLock("g", 2), makeLock("h", 0)) + close(sendCh[0]) + close(sendCh[1]) + close(sendCh[2]) + c.Assert(<-resCh, DeepEquals, makeLockList(makeLock("a", 0), makeLock("a", 1), makeLock("b", 0), makeLock("c", 0), + makeLock("d", 0), makeLock("d", 1), makeLock("e", 0), makeLock("g", 0), makeLock("g", 1), makeLock("g", 2), makeLock("h", 0))) + c.Assert(scanner.GetSucceededStores(), DeepEquals, makeIDSet(storeIDs, 0, 1, 2)) + } +} + +func (s *testGCWorkerSuite) TestResolveLocksPhysical(c *C) { + alwaysSucceedHanlder := func(addr string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + switch req.Type { + case tikvrpc.CmdPhysicalScanLock: + return &tikvrpc.Response{Resp: &kvrpcpb.PhysicalScanLockResponse{Locks: nil, Error: ""}}, nil + case tikvrpc.CmdRegisterLockObserver: + return &tikvrpc.Response{Resp: &kvrpcpb.RegisterLockObserverResponse{Error: ""}}, nil + case tikvrpc.CmdCheckLockObserver: + return &tikvrpc.Response{Resp: &kvrpcpb.CheckLockObserverResponse{Error: "", IsClean: true, Locks: nil}}, nil + case tikvrpc.CmdRemoveLockObserver: + return &tikvrpc.Response{Resp: &kvrpcpb.RemoveLockObserverResponse{Error: ""}}, nil + default: + panic("unreachable") + } + } + alwaysFailHandler := func(addr string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + switch req.Type { + case tikvrpc.CmdPhysicalScanLock: + return &tikvrpc.Response{Resp: &kvrpcpb.PhysicalScanLockResponse{Locks: nil, Error: "error"}}, nil + case tikvrpc.CmdRegisterLockObserver: + return &tikvrpc.Response{Resp: &kvrpcpb.RegisterLockObserverResponse{Error: "error"}}, nil + case tikvrpc.CmdCheckLockObserver: + return &tikvrpc.Response{Resp: &kvrpcpb.CheckLockObserverResponse{Error: "error", IsClean: false, Locks: nil}}, nil + case tikvrpc.CmdRemoveLockObserver: + return &tikvrpc.Response{Resp: &kvrpcpb.RemoveLockObserverResponse{Error: "error"}}, nil + default: + panic("unreachable") + } + } + reset := func() { + s.client.physicalScanLockHandler = alwaysSucceedHanlder + s.client.registerLockObserverHandler = alwaysSucceedHanlder + s.client.checkLockObserverHandler = alwaysSucceedHanlder + s.client.removeLockObserverHandler = alwaysSucceedHanlder + } + + ctx := context.Background() + + // No lock + reset() + err := s.gcWorker.resolveLocksPhysical(ctx, 10000) + c.Assert(err, IsNil) + + // Should return error when fails to register lock observers. + reset() + s.client.registerLockObserverHandler = alwaysFailHandler + err = s.gcWorker.resolveLocksPhysical(ctx, 10000) + c.Assert(err, ErrorMatches, "register lock observer.*") + + // Should return error when fails to resolve locks. + reset() + s.client.physicalScanLockHandler = func(addr string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + locks := []*kvrpcpb.LockInfo{{Key: []byte{0}}} + return &tikvrpc.Response{Resp: &kvrpcpb.PhysicalScanLockResponse{Locks: locks, Error: ""}}, nil + } + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/gcworker/resolveLocksAcrossRegionsErr", "return(100)"), IsNil) + err = s.gcWorker.resolveLocksPhysical(ctx, 10000) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/gcworker/resolveLocksAcrossRegionsErr"), IsNil) + c.Assert(err, ErrorMatches, "injectedError") + + // Shouldn't return error when fails to scan locks. + reset() + var returnError uint32 = 1 + s.client.physicalScanLockHandler = func(addr string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + if atomic.CompareAndSwapUint32(&returnError, 1, 0) { + return alwaysFailHandler(addr, req) + } + return alwaysSucceedHanlder(addr, req) + } + err = s.gcWorker.resolveLocksPhysical(ctx, 10000) + c.Assert(err, IsNil) + + // Should return error if reaches retry limit + reset() + s.client.physicalScanLockHandler = alwaysFailHandler + err = s.gcWorker.resolveLocksPhysical(ctx, 10000) + c.Assert(err, ErrorMatches, ".*dirty.*") + + // Should return error when one registered store is dirty. + reset() + s.client.checkLockObserverHandler = func(addr string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + return &tikvrpc.Response{Resp: &kvrpcpb.CheckLockObserverResponse{Error: "", IsClean: false, Locks: nil}}, nil + } + err = s.gcWorker.resolveLocksPhysical(ctx, 10000) + c.Assert(err, ErrorMatches, "store.*dirty") + + // Should return error when fails to check lock observers. + reset() + s.client.checkLockObserverHandler = alwaysFailHandler + err = s.gcWorker.resolveLocksPhysical(ctx, 10000) + // When fails to check lock observer in a store, we assume the store is dirty. + c.Assert(err, ErrorMatches, "store.*dirty") + + // Shouldn't return error when the dirty store is newly added. + reset() + var wg sync.WaitGroup + wg.Add(1) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/gcworker/beforeCheckLockObservers", "pause"), IsNil) + go func() { + defer wg.Done() + err := s.gcWorker.resolveLocksPhysical(ctx, 10000) + c.Assert(err, IsNil) + }() + // Sleep to let the goroutine pause. + time.Sleep(500 * time.Millisecond) + s.cluster.AddStore(100, "store100") + once := true + s.client.checkLockObserverHandler = func(addr string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + // The newly added store returns IsClean=false for the first time. + if addr == "store100" && once { + once = false + return &tikvrpc.Response{Resp: &kvrpcpb.CheckLockObserverResponse{Error: "", IsClean: false, Locks: nil}}, nil + } + return alwaysSucceedHanlder(addr, req) + } + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/gcworker/beforeCheckLockObservers"), IsNil) + wg.Wait() + + // Shouldn't return error when a store is removed. + reset() + wg.Add(1) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/gcworker/beforeCheckLockObservers", "pause"), IsNil) + go func() { + defer wg.Done() + err := s.gcWorker.resolveLocksPhysical(ctx, 10000) + c.Assert(err, IsNil) + }() + // Sleep to let the goroutine pause. + time.Sleep(500 * time.Millisecond) + s.cluster.RemoveStore(100) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/gcworker/beforeCheckLockObservers"), IsNil) + wg.Wait() + + // Should return error when a cleaned store becomes dirty. + reset() + wg.Add(1) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/gcworker/beforeCheckLockObservers", "pause"), IsNil) + go func() { + defer wg.Done() + err := s.gcWorker.resolveLocksPhysical(ctx, 10000) + c.Assert(err, ErrorMatches, "store.*dirty") + }() + // Sleep to let the goroutine pause. + time.Sleep(500 * time.Millisecond) + store := s.cluster.GetAllStores()[0] + var onceClean uint32 = 1 + s.cluster.AddStore(100, "store100") + var onceDirty uint32 = 1 + s.client.checkLockObserverHandler = func(addr string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + switch addr { + case "store100": + // The newly added store returns IsClean=false for the first time. + if atomic.CompareAndSwapUint32(&onceDirty, 1, 0) { + return &tikvrpc.Response{Resp: &kvrpcpb.CheckLockObserverResponse{Error: "", IsClean: false, Locks: nil}}, nil + } + return alwaysSucceedHanlder(addr, req) + case store.Address: + // The store returns IsClean=true for the first time. + if atomic.CompareAndSwapUint32(&onceClean, 1, 0) { + return alwaysSucceedHanlder(addr, req) + } + return &tikvrpc.Response{Resp: &kvrpcpb.CheckLockObserverResponse{Error: "", IsClean: false, Locks: nil}}, nil + default: + return alwaysSucceedHanlder(addr, req) + } + } + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/gcworker/beforeCheckLockObservers"), IsNil) + wg.Wait() +} + +func (s *testGCWorkerSuite) TestPhyscailScanLockDeadlock(c *C) { + ctx := context.Background() + stores := s.cluster.GetAllStores() + c.Assert(len(stores), Greater, 1) + + s.client.physicalScanLockHandler = func(addr string, req *tikvrpc.Request) (*tikvrpc.Response, error) { + c.Assert(addr, Equals, stores[0].Address) + scanReq := req.PhysicalScanLock() + scanLockLimit := int(scanReq.Limit) + locks := make([]*kvrpcpb.LockInfo, 0, scanReq.Limit) + for i := 0; i < scanLockLimit; i++ { + // The order of keys doesn't matter. + locks = append(locks, &kvrpcpb.LockInfo{Key: []byte{byte(i)}}) + } + return &tikvrpc.Response{ + Resp: &kvrpcpb.PhysicalScanLockResponse{ + Locks: locks, + Error: "", + }, + }, nil + } + + // Sleep 1000ms to let the main goroutine block on sending tasks. + // Inject error to the goroutine resolving locks so that the main goroutine will block forever if it doesn't handle channels properly. + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/gcworker/resolveLocksAcrossRegionsErr", "return(1000)"), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/gcworker/resolveLocksAcrossRegionsErr"), IsNil) + }() + + done := make(chan interface{}) + go func() { + defer close(done) + storesMap := map[uint64]*metapb.Store{stores[0].Id: stores[0]} + succeeded, err := s.gcWorker.physicalScanAndResolveLocks(ctx, 10000, storesMap) + c.Assert(succeeded, IsNil) + c.Assert(err, ErrorMatches, "injectedError") + }() + select { + case <-done: + case <-time.After(5 * time.Second): + c.Fatal("physicalScanAndResolveLocks blocks") } } diff --git a/store/tikv/isolation_test.go b/store/tikv/isolation_test.go index 28c2c6fcb2c82..d8dd333201cde 100644 --- a/store/tikv/isolation_test.go +++ b/store/tikv/isolation_test.go @@ -23,7 +23,6 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" ) @@ -72,7 +71,6 @@ func (s *testIsolationSuite) SetWithRetry(c *C, k, v []byte) writeRecord { commitTS: txn.(*tikvTxn).commitTS, } } - c.Assert(kv.IsTxnRetryableError(err) || terror.ErrorEqual(err, terror.ErrResultUndetermined), IsTrue) } } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 8af8050fc5512..262b8046e9e33 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -201,7 +201,7 @@ func newTikvStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Clie clusterID: pdClient.GetClusterID(context.TODO()), uuid: uuid, oracle: o, - client: client, + client: reqCollapse{client}, pdClient: pdClient, regionCache: NewRegionCache(pdClient), coprCache: nil, @@ -213,9 +213,6 @@ func newTikvStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Clie } store.lockResolver = newLockResolver(store) store.enableGC = enableGC - if rpcClient, ok := store.client.(*rpcClient); ok { - rpcClient.dieEventListener = store.regionCache.NotifyNodeDie - } coprCache, err := newCoprCache(coprCacheConfig) if err != nil { diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index fb397932dc8c5..9e002211d8325 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -120,7 +120,7 @@ type TxnStatus struct { func (s TxnStatus) IsCommitted() bool { return s.ttl == 0 && s.commitTS > 0 } // CommitTS returns the txn's commitTS. It is valid iff `IsCommitted` is true. -func (s TxnStatus) CommitTS() uint64 { return uint64(s.commitTS) } +func (s TxnStatus) CommitTS() uint64 { return s.commitTS } // By default, locks after 3000ms is considered unusual (the client created the // lock might be dead). Other client may cleanup this kind of lock. @@ -224,7 +224,7 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc Regi return false, errors.New("TiDB ask TiKV to rollback locks but it doesn't, the protocol maybe wrong") } - txnInfos[l.TxnID] = uint64(status.commitTS) + txnInfos[l.TxnID] = status.commitTS } logutil.BgLogger().Info("BatchResolveLocks: lookup txn status", zap.Duration("cost time", time.Since(startTime)), @@ -363,7 +363,8 @@ func (lr *LockResolver) resolveLocks(bo *Backoffer, callerStartTS uint64, locks pushed = nil } - if msBeforeTxnExpired.value() > 0 { + if msBeforeTxnExpired.value() > 0 && len(pushed) == 0 { + // If len(pushed) > 0, the caller will not block on the locks, it push the minCommitTS instead. tikvLockResolverCountWithWaitExpired.Inc() } return msBeforeTxnExpired.value(), pushed, nil diff --git a/store/tikv/lock_test.go b/store/tikv/lock_test.go index 99acf00ebc515..8160aa652a871 100644 --- a/store/tikv/lock_test.go +++ b/store/tikv/lock_test.go @@ -17,7 +17,9 @@ import ( "context" "fmt" "math" + "math/rand" "runtime" + "strings" "time" . "github.com/pingcap/check" @@ -124,7 +126,7 @@ func (s *testLockSuite) TestScanLockResolveWithSeek(c *C) { for ch := byte('a'); ch <= byte('z'); ch++ { c.Assert(iter.Valid(), IsTrue) c.Assert([]byte(iter.Key()), BytesEquals, []byte{ch}) - c.Assert([]byte(iter.Value()), BytesEquals, []byte{ch}) + c.Assert(iter.Value(), BytesEquals, []byte{ch}) c.Assert(iter.Next(), IsNil) } } @@ -509,7 +511,7 @@ func (s *testLockSuite) TestZeroMinCommitTS(c *C) { expire, pushed, err = newLockResolver(s.store).ResolveLocks(bo, math.MaxUint64, []*Lock{lock}) c.Assert(err, IsNil) - c.Assert(pushed, HasLen, 0) + c.Assert(pushed, HasLen, 1) c.Assert(expire, Greater, int64(0)) // Clean up this test. @@ -518,3 +520,28 @@ func (s *testLockSuite) TestZeroMinCommitTS(c *C) { c.Assert(err, IsNil) c.Assert(expire, Equals, int64(0)) } + +func (s *testLockSuite) TestDeduplicateKeys(c *C) { + inputs := []string{ + "a b c", + "a a b c", + "a a a b c", + "a a a b b b b c", + "a b b b b c c c", + } + r := rand.New(rand.NewSource(time.Now().UnixNano())) + for _, in := range inputs { + strs := strings.Split(in, " ") + keys := make([][]byte, len(strs)) + for _, i := range r.Perm(len(strs)) { + keys[i] = []byte(strs[i]) + } + keys = deduplicateKeys(keys) + strs = strs[:len(keys)] + for i := range keys { + strs[i] = string(keys[i]) + } + out := strings.Join(strs, " ") + c.Assert(out, Equals, "a b c") + } +} diff --git a/store/tikv/oracle/oracles/local_test.go b/store/tikv/oracle/oracles/local_test.go index 678eb005bc8ff..d2dd31bfe5ee5 100644 --- a/store/tikv/oracle/oracles/local_test.go +++ b/store/tikv/oracle/oracles/local_test.go @@ -45,11 +45,11 @@ func TestIsExpired(t *testing.T) { oracles.SetOracleHookCurrentTime(o, start) ts, _ := o.GetTimestamp(context.Background()) oracles.SetOracleHookCurrentTime(o, start.Add(10*time.Millisecond)) - expire := o.IsExpired(uint64(ts), 5) + expire := o.IsExpired(ts, 5) if !expire { t.Error("should expired") } - expire = o.IsExpired(uint64(ts), 200) + expire = o.IsExpired(ts, 200) if expire { t.Error("should not expired") } @@ -62,7 +62,7 @@ func TestLocalOracle_UntilExpired(t *testing.T) { oracles.SetOracleHookCurrentTime(o, start) ts, _ := o.GetTimestamp(context.Background()) oracles.SetOracleHookCurrentTime(o, start.Add(10*time.Millisecond)) - if o.UntilExpired(uint64(ts), 5) != -5 || o.UntilExpired(uint64(ts), 15) != 5 { + if o.UntilExpired(ts, 5) != -5 || o.UntilExpired(ts, 15) != 5 { t.Error("until expired should be +-5") } } diff --git a/store/tikv/pd_codec.go b/store/tikv/pd_codec.go index 17e1c9b5f34c9..436b929055154 100644 --- a/store/tikv/pd_codec.go +++ b/store/tikv/pd_codec.go @@ -59,7 +59,7 @@ func (c *codecPDClient) ScanRegions(ctx context.Context, startKey []byte, endKey } for _, region := range regions { if region != nil { - err = decodeRegionMetaKey(region) + err = decodeRegionMetaKeyInPlace(region) if err != nil { return nil, nil, errors.Trace(err) } @@ -75,14 +75,14 @@ func processRegionResult(region *metapb.Region, peer *metapb.Peer, err error) (* if region == nil { return nil, nil, nil } - err = decodeRegionMetaKey(region) + err = decodeRegionMetaKeyInPlace(region) if err != nil { return nil, nil, errors.Trace(err) } return region, peer, nil } -func decodeRegionMetaKey(r *metapb.Region) error { +func decodeRegionMetaKeyInPlace(r *metapb.Region) error { if len(r.StartKey) != 0 { _, decoded, err := codec.DecodeBytes(r.StartKey, nil) if err != nil { @@ -99,3 +99,22 @@ func decodeRegionMetaKey(r *metapb.Region) error { } return nil } + +func decodeRegionMetaKeyWithShallowCopy(r *metapb.Region) (*metapb.Region, error) { + nr := *r + if len(r.StartKey) != 0 { + _, decoded, err := codec.DecodeBytes(r.StartKey, nil) + if err != nil { + return nil, errors.Trace(err) + } + nr.StartKey = decoded + } + if len(r.EndKey) != 0 { + _, decoded, err := codec.DecodeBytes(r.EndKey, nil) + if err != nil { + return nil, errors.Trace(err) + } + nr.EndKey = decoded + } + return &nr, nil +} diff --git a/store/tikv/range_task_test.go b/store/tikv/range_task_test.go index b8e3705d04a11..06baee0f3ba91 100644 --- a/store/tikv/range_task_test.go +++ b/store/tikv/range_task_test.go @@ -21,12 +21,13 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" ) type testRangeTaskSuite struct { OneByOneSuite - cluster *mocktikv.Cluster + cluster cluster.Cluster store *tikvStore testRanges []kv.KeyRange @@ -59,9 +60,10 @@ func (s *testRangeTaskSuite) SetUpTest(c *C) { } allRegionRanges = append(allRegionRanges, makeRange("z", "")) - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithMultiRegions(s.cluster, splitKeys...) - client, pdClient, err := mocktikv.NewTiKVAndPDClient(s.cluster, nil, "") + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithMultiRegions(cluster, splitKeys...) + s.cluster = cluster + client, pdClient, err := mocktikv.NewTiKVAndPDClient(cluster, nil, "") c.Assert(err, IsNil) store, err := NewTestTiKVStore(client, pdClient, nil, nil, 0) diff --git a/store/tikv/rawkv_test.go b/store/tikv/rawkv_test.go index ffcc97346aa20..f5bd479946819 100644 --- a/store/tikv/rawkv_test.go +++ b/store/tikv/rawkv_test.go @@ -19,12 +19,13 @@ import ( "fmt" . "github.com/pingcap/check" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" ) type testRawKVSuite struct { OneByOneSuite - cluster *mocktikv.Cluster + cluster cluster.Cluster client *RawKVClient bo *Backoffer } @@ -32,15 +33,16 @@ type testRawKVSuite struct { var _ = Suite(&testRawKVSuite{}) func (s *testRawKVSuite) SetUpTest(c *C) { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - pdClient := mocktikv.NewPDClient(s.cluster) + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + pdClient := mocktikv.NewPDClient(cluster) mvccStore := mocktikv.MustNewMVCCStore() s.client = &RawKVClient{ clusterID: 0, regionCache: NewRegionCache(pdClient), pdClient: pdClient, - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore), + rpcClient: mocktikv.NewRPCClient(cluster, mvccStore), } s.bo = NewBackoffer(context.Background(), 5000) } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 9f5dfd92874e1..2cc1593e611fc 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -17,6 +17,7 @@ import ( "bytes" "context" "fmt" + "net/http" "sync" "sync/atomic" "time" @@ -28,11 +29,14 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" pd "github.com/pingcap/pd/v4/client" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" atomic2 "go.uber.org/atomic" "go.uber.org/zap" + "golang.org/x/sync/singleflight" ) const ( @@ -56,6 +60,9 @@ var ( tikvRegionCacheCounterWithGetStoreOK = metrics.TiKVRegionCacheCounter.WithLabelValues("get_store", "ok") tikvRegionCacheCounterWithGetStoreError = metrics.TiKVRegionCacheCounter.WithLabelValues("get_store", "err") tikvRegionCacheCounterWithInvalidateStoreRegionsOK = metrics.TiKVRegionCacheCounter.WithLabelValues("invalidate_store_regions", "ok") + + tikvStatusCountWithOK = metrics.TiKVStatusCounter.WithLabelValues("ok") + tikvStatusCountWithError = metrics.TiKVStatusCounter.WithLabelValues("err") ) const ( @@ -77,18 +84,18 @@ type RegionStore struct { workTiKVIdx int32 // point to current work peer in meta.Peers and work store in stores(same idx) for tikv peer workTiFlashIdx int32 // point to current work peer in meta.Peers and work store in stores(same idx) for tiflash peer stores []*Store // stores in this region - storeFails []uint32 // snapshots of store's fail, need reload when `storeFails[curr] != stores[cur].fail` + storeEpochs []uint32 // snapshots of store's epoch, need reload when `storeEpochs[curr] != stores[cur].fail` } // clone clones region store struct. func (r *RegionStore) clone() *RegionStore { - storeFails := make([]uint32, len(r.stores)) - copy(storeFails, r.storeFails) + storeEpochs := make([]uint32, len(r.stores)) + copy(storeEpochs, r.storeEpochs) return &RegionStore{ workTiFlashIdx: r.workTiFlashIdx, workTiKVIdx: r.workTiKVIdx, stores: r.stores, - storeFails: storeFails, + storeEpochs: storeEpochs, } } @@ -107,7 +114,7 @@ func (r *RegionStore) follower(seed uint32) int32 { if r.stores[followerIdx].storeType != kv.TiKV { continue } - if r.storeFails[followerIdx] == atomic.LoadUint32(&r.stores[followerIdx].fail) { + if r.storeEpochs[followerIdx] == atomic.LoadUint32(&r.stores[followerIdx].epoch) { return followerIdx } seed++ @@ -122,7 +129,7 @@ func (r *RegionStore) peer(seed uint32) int32 { if r.stores[i].storeType != kv.TiKV { continue } - if r.storeFails[i] != atomic.LoadUint32(&r.stores[i].fail) { + if r.storeEpochs[i] != atomic.LoadUint32(&r.stores[i].epoch) { continue } candidates = append(candidates, int32(i)) @@ -142,7 +149,7 @@ func (r *Region) init(c *RegionCache) { workTiKVIdx: 0, workTiFlashIdx: 0, stores: make([]*Store, 0, len(r.meta.Peers)), - storeFails: make([]uint32, 0, len(r.meta.Peers)), + storeEpochs: make([]uint32, 0, len(r.meta.Peers)), } for _, p := range r.meta.Peers { c.storeMu.RLock() @@ -152,7 +159,7 @@ func (r *Region) init(c *RegionCache) { store = c.getStoreByStoreID(p.StoreId) } rs.stores = append(rs.stores, store) - rs.storeFails = append(rs.storeFails, atomic.LoadUint32(&store.fail)) + rs.storeEpochs = append(rs.storeEpochs, atomic.LoadUint32(&store.epoch)) } atomic.StorePointer(&r.store, unsafe.Pointer(rs)) @@ -219,7 +226,6 @@ type RegionCache struct { stores map[uint64]*Store flashStoreNumber int32 } - notifyDieCh chan []string notifyCheckCh chan struct{} closeCh chan struct{} } @@ -234,7 +240,6 @@ func NewRegionCache(pdClient pd.Client) *RegionCache { c.storeMu.stores = make(map[uint64]*Store) c.storeMu.flashStoreNumber = 0 c.notifyCheckCh = make(chan struct{}, 1) - c.notifyDieCh = make(chan []string, 1) c.closeCh = make(chan struct{}) go c.asyncCheckAndResolveLoop() return c @@ -255,8 +260,6 @@ func (c *RegionCache) asyncCheckAndResolveLoop() { case <-c.notifyCheckCh: needCheckStores = needCheckStores[:0] c.checkAndResolve(needCheckStores) - case addrs := <-c.notifyDieCh: - c.invalidStore(addrs) } } } @@ -287,19 +290,6 @@ func (c *RegionCache) checkAndResolve(needCheckStores []*Store) { } } -func (c *RegionCache) invalidStore(sAddrs []string) { - c.storeMu.RLock() - for _, store := range c.storeMu.stores { - for _, sAddr := range sAddrs { - if store.addr == sAddr { - atomic.AddUint32(&store.fail, 1) - } - } - - } - c.storeMu.RUnlock() -} - // RPCContext contains data that is needed to send RPC to a region. type RPCContext struct { Region RegionVerID @@ -357,8 +347,8 @@ func (c *RegionCache) GetTiKVRPCContext(bo *Backoffer, id RegionVerID, replicaRe return nil, nil } - storeFailEpoch := atomic.LoadUint32(&store.fail) - if storeFailEpoch != regionStore.storeFails[storeIdx] { + storeFailEpoch := atomic.LoadUint32(&store.epoch) + if storeFailEpoch != regionStore.storeEpochs[storeIdx] { cachedRegion.invalidate() logutil.BgLogger().Info("invalidate current region, because others failed on same store", zap.Uint64("region", id.GetID()), @@ -412,8 +402,8 @@ func (c *RegionCache) GetTiFlashRPCContext(bo *Backoffer, id RegionVerID) (*RPCC } atomic.StoreInt32(®ionStore.workTiFlashIdx, int32(storeIdx)) peer := cachedRegion.meta.Peers[storeIdx] - storeFailEpoch := atomic.LoadUint32(&store.fail) - if storeFailEpoch != regionStore.storeFails[storeIdx] { + storeFailEpoch := atomic.LoadUint32(&store.epoch) + if storeFailEpoch != regionStore.storeEpochs[storeIdx] { cachedRegion.invalidate() logutil.BgLogger().Info("invalidate current region, because others failed on same store", zap.Uint64("region", id.GetID()), @@ -511,11 +501,34 @@ func (c *RegionCache) OnSendFail(bo *Backoffer, ctx *RPCContext, scheduleReload tikvRegionCacheCounterWithSendFail.Inc() r := c.getCachedRegionWithRLock(ctx.Region) if r != nil { + rs := r.getStore() + if err != nil { + s := rs.stores[ctx.PeerIdx] + + // send fail but store is reachable, keep retry current peer. + if s.requestLiveness(bo) == reachable { + return + } + + // invalidate regions in store. + epoch := rs.storeEpochs[ctx.PeerIdx] + if atomic.CompareAndSwapUint32(&s.epoch, epoch, epoch+1) { + logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) + tikvRegionCacheCounterWithInvalidateStoreRegionsOK.Inc() + } + + // schedule a store addr resolve. + s.markNeedCheck(c.notifyCheckCh) + } + + // try next peer to found new leader. if ctx.Store.storeType == kv.TiKV { - c.switchNextPeer(r, ctx.PeerIdx, err) + rs.switchNextPeer(r, ctx.PeerIdx) } else { - c.switchNextFlashPeer(r, ctx.PeerIdx, err) + rs.switchNextFlashPeer(r, ctx.PeerIdx) } + + // force reload region when retry all known peers in region. if scheduleReload { r.scheduleReload() } @@ -724,7 +737,8 @@ func (c *RegionCache) UpdateLeader(regionID RegionVerID, leaderStoreID uint64, c } if leaderStoreID == 0 { - c.switchNextPeer(r, currentPeerIdx, nil) + rs := r.getStore() + rs.switchNextPeer(r, currentPeerIdx) logutil.BgLogger().Info("switch region peer to next due to NotLeader with NULL leader", zap.Int("currIdx", currentPeerIdx), zap.Uint64("regionID", regionID.GetID())) @@ -789,12 +803,34 @@ func (c *RegionCache) searchCachedRegion(key []byte, isEndKey bool) *Region { // `getCachedRegion`, it should be called with c.mu.RLock(), and the returned // Region should not be used after c.mu is RUnlock(). func (c *RegionCache) getRegionByIDFromCache(regionID uint64) *Region { + var newestRegion *Region + ts := time.Now().Unix() for v, r := range c.mu.regions { if v.id == regionID { - return r + lastAccess := atomic.LoadInt64(&r.lastAccess) + if ts-lastAccess > RegionCacheTTLSec { + continue + } + if newestRegion == nil { + newestRegion = r + continue + } + nv := newestRegion.VerID() + cv := r.VerID() + if nv.GetConfVer() < cv.GetConfVer() { + newestRegion = r + continue + } + if nv.GetVer() < cv.GetVer() { + newestRegion = r + continue + } } } - return nil + if newestRegion != nil { + atomic.CompareAndSwapInt64(&newestRegion.lastAccess, atomic.LoadInt64(&newestRegion.lastAccess), ts) + } + return newestRegion } // loadRegion loads region from pd client, and picks the first peer as leader. @@ -1025,7 +1061,8 @@ func (c *RegionCache) OnRegionEpochNotMatch(bo *Backoffer, ctx *RPCContext, curr // If the region epoch is not ahead of TiKV's, replace region meta in region cache. for _, meta := range currentRegions { if _, ok := c.pdClient.(*codecPDClient); ok { - if err := decodeRegionMetaKey(meta); err != nil { + var err error + if meta, err = decodeRegionMetaKeyWithShallowCopy(meta); err != nil { return errors.Errorf("newRegion's range key is not encoded: %v, %v", meta, err) } } @@ -1173,57 +1210,25 @@ func (c *RegionCache) switchToPeer(r *Region, targetStoreID uint64) (found bool) return } -func (c *RegionCache) switchNextFlashPeer(r *Region, currentPeerIdx int, err error) { - rs := r.getStore() - - if err != nil { // TODO: refine err, only do this for some errors. - s := rs.stores[currentPeerIdx] - epoch := rs.storeFails[currentPeerIdx] - if atomic.CompareAndSwapUint32(&s.fail, epoch, epoch+1) { - logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) - tikvRegionCacheCounterWithInvalidateStoreRegionsOK.Inc() - } - s.markNeedCheck(c.notifyCheckCh) - } - - nextIdx := (currentPeerIdx + 1) % len(rs.stores) - newRegionStore := rs.clone() +func (r *RegionStore) switchNextFlashPeer(rr *Region, currentPeerIdx int) { + nextIdx := (currentPeerIdx + 1) % len(r.stores) + newRegionStore := r.clone() newRegionStore.workTiFlashIdx = int32(nextIdx) - r.compareAndSwapStore(rs, newRegionStore) -} - -// NotifyNodeDie is used for TiClient notify RegionCache a die node. -func (c *RegionCache) NotifyNodeDie(addrs []string) { - select { - case c.notifyDieCh <- addrs: - default: - } + rr.compareAndSwapStore(r, newRegionStore) } -func (c *RegionCache) switchNextPeer(r *Region, currentPeerIdx int, err error) { - rs := r.getStore() - - if err != nil { // TODO: refine err, only do this for some errors. - s := rs.stores[currentPeerIdx] - epoch := rs.storeFails[currentPeerIdx] - if atomic.CompareAndSwapUint32(&s.fail, epoch, epoch+1) { - logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) - tikvRegionCacheCounterWithInvalidateStoreRegionsOK.Inc() - } - s.markNeedCheck(c.notifyCheckCh) - } - - if int(rs.workTiKVIdx) != currentPeerIdx { +func (r *RegionStore) switchNextPeer(rr *Region, currentPeerIdx int) { + if int(r.workTiKVIdx) != currentPeerIdx { return } - nextIdx := (currentPeerIdx + 1) % len(rs.stores) - for rs.stores[nextIdx].storeType == kv.TiFlash { - nextIdx = (nextIdx + 1) % len(rs.stores) + nextIdx := (currentPeerIdx + 1) % len(r.stores) + for r.stores[nextIdx].storeType == kv.TiFlash { + nextIdx = (nextIdx + 1) % len(r.stores) } - newRegionStore := rs.clone() + newRegionStore := r.clone() newRegionStore.workTiKVIdx = int32(nextIdx) - r.compareAndSwapStore(rs, newRegionStore) + rr.compareAndSwapStore(r, newRegionStore) } func (c *RegionCache) getPeerStoreIndex(r *Region, id uint64) (idx int, found bool) { @@ -1272,10 +1277,11 @@ func (r *Region) ContainsByEnd(key []byte) bool { // Store contains a kv process's address. type Store struct { addr string // loaded store address + saddr string // loaded store status address storeID uint64 // store's id state uint64 // unsafe store storeState resolveMutex sync.Mutex // protect pd from concurrent init requests - fail uint32 // store fail count, see RegionStore.storeFails + epoch uint32 // store fail epoch, see RegionStore.storeEpochs storeType kv.StoreType // type of the store tokenCount atomic2.Int64 // used store token count } @@ -1322,6 +1328,7 @@ func (s *Store) initResolve(bo *Backoffer, c *RegionCache) (addr string, err err } addr = store.GetAddress() s.addr = addr + s.saddr = store.GetStatusAddress() s.storeType = GetStoreTypeByMeta(store) retry: state = s.getResolveState() @@ -1368,7 +1375,7 @@ func (s *Store) reResolve(c *RegionCache) { // store has be removed in PD, we should invalidate all regions using those store. logutil.BgLogger().Info("invalidate regions in removed store", zap.Uint64("store", s.storeID), zap.String("add", s.addr)) - atomic.AddUint32(&s.fail, 1) + atomic.AddUint32(&s.epoch, 1) tikvRegionCacheCounterWithInvalidateStoreRegionsOK.Inc() return } @@ -1377,8 +1384,8 @@ func (s *Store) reResolve(c *RegionCache) { addr = store.GetAddress() if s.addr != addr { state := resolved - newStore := &Store{storeID: s.storeID, addr: addr, storeType: storeType} - newStore.state = *(*uint64)(unsafe.Pointer(&state)) + newStore := &Store{storeID: s.storeID, addr: addr, saddr: store.GetStatusAddress(), storeType: storeType} + newStore.state = *(*uint64)(&state) c.storeMu.Lock() orgStore, exists := c.storeMu.stores[newStore.storeID] if exists && orgStore.storeType == kv.TiFlash { @@ -1440,3 +1447,91 @@ retry: } } + +type livenessState uint32 + +var ( + livenessSf singleflight.Group + storeLivenessTimeout time.Duration +) + +const ( + unknown livenessState = iota + reachable + unreachable + offline +) + +func init() { + t, err := time.ParseDuration(config.GetGlobalConfig().TiKVClient.StoreLivenessTimeout) + if err != nil { + logutil.BgLogger().Fatal("invalid duration value for store-liveness-timeout", + zap.String("currentValue", config.GetGlobalConfig().TiKVClient.StoreLivenessTimeout)) + } + storeLivenessTimeout = t +} + +func (s *Store) requestLiveness(bo *Backoffer) (l livenessState) { + saddr := s.saddr + if len(saddr) == 0 { + l = unknown + return + } + rsCh := livenessSf.DoChan(saddr, func() (interface{}, error) { + return invokeKVStatusAPI(saddr, storeLivenessTimeout), nil + }) + var ctx context.Context + if bo != nil { + ctx = bo.ctx + } else { + ctx = context.Background() + } + select { + case rs := <-rsCh: + l = rs.Val.(livenessState) + case <-ctx.Done(): + l = unknown + return + } + return +} + +func invokeKVStatusAPI(saddr string, timeout time.Duration) (l livenessState) { + start := time.Now() + defer func() { + if l == reachable { + tikvStatusCountWithOK.Inc() + } else { + tikvStatusCountWithError.Inc() + } + metrics.TiKVStatusDuration.WithLabelValues(saddr).Observe(time.Since(start).Seconds()) + }() + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer cancel() + url := fmt.Sprintf("%s://%s/status", util.InternalHTTPSchema(), saddr) + req, err := http.NewRequestWithContext(ctx, http.MethodGet, url, nil) + if err != nil { + logutil.BgLogger().Info("[liveness] build kv status request fail", zap.String("store", saddr), zap.Error(err)) + l = unreachable + return + } + resp, err := util.InternalHTTPClient().Do(req) + if err != nil { + logutil.BgLogger().Info("[liveness] request kv status fail", zap.String("store", saddr), zap.Error(err)) + l = unreachable + return + } + defer func() { + err1 := resp.Body.Close() + if err1 != nil { + logutil.BgLogger().Debug("[liveness] close kv status api body failed", zap.String("store", saddr), zap.Error(err)) + } + }() + if resp.StatusCode != http.StatusOK { + logutil.BgLogger().Info("[liveness] request kv status fail", zap.String("store", saddr), zap.String("status", resp.Status)) + l = unreachable + return + } + l = reachable + return +} diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index adf164135cd21..06cdca6d56cdb 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -20,6 +20,7 @@ import ( "math/rand" "testing" "time" + "unsafe" "github.com/google/btree" . "github.com/pingcap/check" @@ -1033,6 +1034,44 @@ func (s *testRegionCacheSuite) TestMixedMeetEpochNotMatch(c *C) { c.Assert(followReqSeed, Equals, uint32(1)) } +func (s *testRegionRequestSuite) TestGetRegionByIDFromCache(c *C) { + region, err := s.cache.LocateRegionByID(s.bo, s.region) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + + // test kv epochNotMatch return empty regions + s.cache.OnRegionEpochNotMatch(s.bo, &RPCContext{Region: region.Region, Store: &Store{storeID: s.store}}, []*metapb.Region{}) + c.Assert(err, IsNil) + r := s.cache.getRegionByIDFromCache(s.region) + c.Assert(r, IsNil) + + // refill cache + region, err = s.cache.LocateRegionByID(s.bo, s.region) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + + // test kv load new region with new start-key and new epoch + v2 := region.Region.confVer + 1 + r2 := metapb.Region{Id: region.Region.id, RegionEpoch: &metapb.RegionEpoch{Version: region.Region.ver, ConfVer: v2}, StartKey: []byte{1}} + st := &Store{storeID: s.store} + s.cache.insertRegionToCache(&Region{meta: &r2, store: unsafe.Pointer(st), lastAccess: time.Now().Unix()}) + region, err = s.cache.LocateRegionByID(s.bo, s.region) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + c.Assert(region.Region.confVer, Equals, v2) + c.Assert(region.Region.ver, Equals, region.Region.ver) + + v3 := region.Region.confVer + 1 + r3 := metapb.Region{Id: region.Region.id, RegionEpoch: &metapb.RegionEpoch{Version: v3, ConfVer: region.Region.confVer}, StartKey: []byte{2}} + st = &Store{storeID: s.store} + s.cache.insertRegionToCache(&Region{meta: &r3, store: unsafe.Pointer(st), lastAccess: time.Now().Unix()}) + region, err = s.cache.LocateRegionByID(s.bo, s.region) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + c.Assert(region.Region.confVer, Equals, region.Region.confVer) + c.Assert(region.Region.ver, Equals, v3) +} + func createSampleRegion(startKey, endKey []byte) *Region { return &Region{ meta: &metapb.Region{ @@ -1100,8 +1139,8 @@ func BenchmarkOnRequestFail(b *testing.B) { Store: store, } r := cache.getCachedRegionWithRLock(rpcCtx.Region) - if r == nil { - cache.switchNextPeer(r, rpcCtx.PeerIdx, nil) + if r != nil { + r.getStore().switchNextPeer(r, rpcCtx.PeerIdx) } } }) diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index ac671683eb58b..568a641975e86 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -378,6 +378,10 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed } if regionErr.GetStaleCommand() != nil { logutil.BgLogger().Debug("tikv reports `StaleCommand`", zap.Stringer("ctx", ctx)) + err = bo.Backoff(boStaleCmd, errors.Errorf("stale command, ctx: %v", ctx)) + if err != nil { + return false, errors.Trace(err) + } return true, nil } if regionErr.GetRaftEntryTooLarge() != nil { diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index e45b33cf57321..8c4dad22b0f4d 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -23,6 +23,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/coprocessor" + "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/tikvpb" "github.com/pingcap/tidb/config" @@ -89,6 +90,47 @@ func (s *testStoreLimitSuite) TearDownTest(c *C) { s.cache.Close() } +type fnClient struct { + fn func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) +} + +func (f *fnClient) Close() error { + return nil +} + +func (f *fnClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { + return f.fn(ctx, addr, req, timeout) +} + +func (s *testRegionRequestSuite) TestOnRegionError(c *C) { + req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ + Key: []byte("key"), + Value: []byte("value"), + }) + region, err := s.cache.LocateRegionByID(s.bo, s.region) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + + // test stale command retry. + func() { + oc := s.regionRequestSender.client + defer func() { + s.regionRequestSender.client = oc + }() + s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + staleResp := &tikvrpc.Response{Resp: &kvrpcpb.GetResponse{ + RegionError: &errorpb.Error{StaleCommand: &errorpb.StaleCommand{}}, + }} + return staleResp, nil + }} + bo := NewBackoffer(context.Background(), 5) + resp, err := s.regionRequestSender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, NotNil) + c.Assert(resp, IsNil) + }() + +} + func (s *testStoreLimitSuite) TestStoreTokenLimit(c *C) { req := tikvrpc.NewRequest(tikvrpc.CmdPrewrite, &kvrpcpb.PrewriteRequest{}, kvrpcpb.Context{}) region, err := s.cache.LocateRegionByID(s.bo, s.regionID) @@ -380,6 +422,30 @@ func (s *mockTikvGrpcServer) ReadIndex(context.Context, *kvrpcpb.ReadIndexReques return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) VerGet(context.Context, *kvrpcpb.VerGetRequest) (*kvrpcpb.VerGetResponse, error) { + return nil, errors.New("unreachable") +} + +func (s *mockTikvGrpcServer) VerBatchGet(context.Context, *kvrpcpb.VerBatchGetRequest) (*kvrpcpb.VerBatchGetResponse, error) { + return nil, errors.New("unreachable") +} + +func (s *mockTikvGrpcServer) VerMut(context.Context, *kvrpcpb.VerMutRequest) (*kvrpcpb.VerMutResponse, error) { + return nil, errors.New("unreachable") +} + +func (s *mockTikvGrpcServer) VerBatchMut(context.Context, *kvrpcpb.VerBatchMutRequest) (*kvrpcpb.VerBatchMutResponse, error) { + return nil, errors.New("unreachable") +} + +func (s *mockTikvGrpcServer) VerScan(context.Context, *kvrpcpb.VerScanRequest) (*kvrpcpb.VerScanResponse, error) { + return nil, errors.New("unreachable") +} + +func (s *mockTikvGrpcServer) VerDeleteRange(context.Context, *kvrpcpb.VerDeleteRangeRequest) (*kvrpcpb.VerDeleteRangeResponse, error) { + return nil, errors.New("unreachable") +} + func (s *testRegionRequestSuite) TestNoReloadRegionForGrpcWhenCtxCanceled(c *C) { // prepare a mock tikv grpc server addr := "localhost:56341" diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 755a3097e748a..b104b83c13508 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -109,8 +109,8 @@ func (s *Scanner) Next() error { } current := s.cache[s.idx] - if (!s.reverse && (len(s.endKey) > 0 && kv.Key(current.Key).Cmp(kv.Key(s.endKey)) >= 0)) || - (s.reverse && len(s.nextStartKey) > 0 && kv.Key(current.Key).Cmp(kv.Key(s.nextStartKey)) < 0) { + if (!s.reverse && (len(s.endKey) > 0 && kv.Key(current.Key).Cmp(s.endKey) >= 0)) || + (s.reverse && len(s.nextStartKey) > 0 && kv.Key(current.Key).Cmp(s.nextStartKey) < 0) { s.eof = true s.Close() return nil @@ -144,7 +144,7 @@ func (s *Scanner) startTS() uint64 { } func (s *Scanner) resolveCurrentLock(bo *Backoffer, current *pb.KvPair) error { - val, err := s.snapshot.get(bo, kv.Key(current.Key)) + val, err := s.snapshot.get(bo, current.Key) if err != nil { return errors.Trace(err) } @@ -250,8 +250,8 @@ func (s *Scanner) getData(bo *Backoffer) error { } else { s.nextEndKey = reqStartKey } - if (!s.reverse && (len(loc.EndKey) == 0 || (len(s.endKey) > 0 && kv.Key(s.nextStartKey).Cmp(kv.Key(s.endKey)) >= 0))) || - (s.reverse && (len(loc.StartKey) == 0 || (len(s.nextStartKey) > 0 && kv.Key(s.nextStartKey).Cmp(kv.Key(s.nextEndKey)) >= 0))) { + if (!s.reverse && (len(loc.EndKey) == 0 || (len(s.endKey) > 0 && s.nextStartKey.Cmp(s.endKey) >= 0))) || + (s.reverse && (len(loc.StartKey) == 0 || (len(s.nextStartKey) > 0 && s.nextStartKey.Cmp(s.nextEndKey) >= 0))) { // Current Region is the last one. s.eof = true } @@ -265,7 +265,7 @@ func (s *Scanner) getData(bo *Backoffer) error { if !s.reverse { s.nextStartKey = kv.Key(lastKey).Next() } else { - s.nextEndKey = kv.Key(lastKey) + s.nextEndKey = lastKey } return nil } diff --git a/store/tikv/scan_mock_test.go b/store/tikv/scan_mock_test.go index 05cd2f89d3898..f7f2628d63e61 100644 --- a/store/tikv/scan_mock_test.go +++ b/store/tikv/scan_mock_test.go @@ -78,7 +78,7 @@ func (s *testScanMockSuite) TestReverseScan(c *C) { scanner, err := newScanner(snapshot, nil, []byte("z"), 10, true) c.Assert(err, IsNil) for ch := byte('y'); ch >= byte('a'); ch-- { - c.Assert(string([]byte{ch}), Equals, string([]byte(scanner.Key()))) + c.Assert(string([]byte{ch}), Equals, string(scanner.Key())) c.Assert(scanner.Next(), IsNil) } c.Assert(scanner.Valid(), IsFalse) @@ -86,7 +86,7 @@ func (s *testScanMockSuite) TestReverseScan(c *C) { scanner, err = newScanner(snapshot, []byte("a"), []byte("i"), 10, true) c.Assert(err, IsNil) for ch := byte('h'); ch >= byte('a'); ch-- { - c.Assert(string([]byte{ch}), Equals, string([]byte(scanner.Key()))) + c.Assert(string([]byte{ch}), Equals, string(scanner.Key())) c.Assert(scanner.Next(), IsNil) } c.Assert(scanner.Valid(), IsFalse) diff --git a/store/tikv/snapshot_test.go b/store/tikv/snapshot_test.go index e8ce0427a7a6f..b2273f86f3553 100644 --- a/store/tikv/snapshot_test.go +++ b/store/tikv/snapshot_test.go @@ -225,7 +225,7 @@ func (s *testSnapshotSuite) TestSkipLargeTxnLock(c *C) { txn1 := s.beginTxn(c) // txn1 is not blocked by txn in the large txn protocol. - _, err = txn1.Get(ctx, kv.Key(x)) + _, err = txn1.Get(ctx, x) c.Assert(kv.IsErrNotFound(errors.Trace(err)), IsTrue) res, err := txn1.BatchGet(ctx, []kv.Key{x, y, kv.Key("z")}) @@ -240,3 +240,37 @@ func (s *testSnapshotSuite) TestSkipLargeTxnLock(c *C) { c.Assert(status.IsCommitted(), IsTrue) c.Assert(status.CommitTS(), Greater, txn1.StartTS()) } + +func (s *testSnapshotSuite) TestPointGetSkipTxnLock(c *C) { + x := kv.Key("x_key_TestPointGetSkipTxnLock") + y := kv.Key("y_key_TestPointGetSkipTxnLock") + txn := s.beginTxn(c) + c.Assert(txn.Set(x, []byte("x")), IsNil) + c.Assert(txn.Set(y, []byte("y")), IsNil) + ctx := context.Background() + bo := NewBackoffer(ctx, PrewriteMaxBackoff) + committer, err := newTwoPhaseCommitterWithInit(txn, 0) + c.Assert(err, IsNil) + committer.lockTTL = 3000 + c.Assert(committer.prewriteMutations(bo, committer.mutations), IsNil) + + snapshot := newTiKVSnapshot(s.store, kv.MaxVersion, 0) + start := time.Now() + c.Assert(committer.primary(), BytesEquals, []byte(x)) + // Point get secondary key. Shouldn't be blocked by the lock and read old data. + _, err = snapshot.Get(ctx, y) + c.Assert(kv.IsErrNotFound(errors.Trace(err)), IsTrue) + c.Assert(time.Since(start), Less, 500*time.Millisecond) + + // Commit the primary key + committer.commitTS = txn.StartTS() + 1 + committer.commitMutations(bo, committer.mutationsOfKeys([][]byte{committer.primary()})) + + snapshot = newTiKVSnapshot(s.store, kv.MaxVersion, 0) + start = time.Now() + // Point get secondary key. Should read committed data. + value, err := snapshot.Get(ctx, y) + c.Assert(err, IsNil) + c.Assert(value, BytesEquals, []byte("y")) + c.Assert(time.Since(start), Less, 500*time.Millisecond) +} diff --git a/store/tikv/split_test.go b/store/tikv/split_test.go index 1f70994501a4a..3ed762909b81a 100644 --- a/store/tikv/split_test.go +++ b/store/tikv/split_test.go @@ -18,12 +18,13 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" ) type testSplitSuite struct { OneByOneSuite - cluster *mocktikv.Cluster + cluster cluster.Cluster store *tikvStore bo *Backoffer } @@ -31,9 +32,10 @@ type testSplitSuite struct { var _ = Suite(&testSplitSuite{}) func (s *testSplitSuite) SetUpTest(c *C) { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - client, pdClient, err := mocktikv.NewTiKVAndPDClient(s.cluster, nil, "") + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + client, pdClient, err := mocktikv.NewTiKVAndPDClient(cluster, nil, "") c.Assert(err, IsNil) store, err := NewTestTiKVStore(client, pdClient, nil, nil, 0) diff --git a/store/tikv/store_test.go b/store/tikv/store_test.go index 21dde748e0b15..30a0745ebd2c7 100644 --- a/store/tikv/store_test.go +++ b/store/tikv/store_test.go @@ -203,6 +203,10 @@ func (c *mockPDClient) UpdateGCSafePoint(ctx context.Context, safePoint uint64) panic("unimplemented") } +func (c *mockPDClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { + panic("unimplemented") +} + func (c *mockPDClient) Close() {} func (c *mockPDClient) ScatterRegion(ctx context.Context, regionID uint64) error { @@ -224,8 +228,10 @@ func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req * resp, err := c.Client.SendRequest(ctx, addr, req, timeout) if c.priority != req.Priority { if resp.Resp != nil { - (resp.Resp.(*pb.GetResponse)).Error = &pb.KeyError{ - Abort: "request check error", + if getResp, ok := resp.Resp.(*pb.GetResponse); ok { + getResp.Error = &pb.KeyError{ + Abort: "request check error", + } } } } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 151d44e246e07..b148686fb8892 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -14,8 +14,10 @@ package tikv import ( + "bytes" "context" "fmt" + "sort" "sync" "sync/atomic" "time" @@ -357,6 +359,7 @@ func (txn *tikvTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput if len(keys) == 0 { return nil } + keys = deduplicateKeys(keys) if txn.IsPessimistic() && lockCtx.ForUpdateTS > 0 { if txn.committer == nil { // connID is used for log. @@ -428,6 +431,20 @@ func (txn *tikvTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput return nil } +// deduplicateKeys deduplicate the keys, it use sort instead of map to avoid memory allocation. +func deduplicateKeys(keys [][]byte) [][]byte { + sort.Slice(keys, func(i, j int) bool { + return bytes.Compare(keys[i], keys[j]) < 0 + }) + deduped := keys[:1] + for i := 1; i < len(keys); i++ { + if !bytes.Equal(deduped[len(deduped)-1], keys[i]) { + deduped = append(deduped, keys[i]) + } + } + return deduped +} + func (txn *tikvTxn) asyncPessimisticRollback(ctx context.Context, keys [][]byte) *sync.WaitGroup { // Clone a new committer for execute in background. committer := &twoPhaseCommitter{ diff --git a/table/column.go b/table/column.go index 60c4b4f974f65..5ddf268d0bd07 100644 --- a/table/column.go +++ b/table/column.go @@ -505,6 +505,8 @@ func GetZeroValue(col *model.ColumnInfo) types.Datum { case mysql.TypeDouble: d.SetFloat64(0) case mysql.TypeNewDecimal: + d.SetLength(col.Flen) + d.SetFrac(col.Decimal) d.SetMysqlDecimal(new(types.MyDecimal)) case mysql.TypeString: if col.Flen > 0 && col.Charset == charset.CharsetBin { diff --git a/table/index.go b/table/index.go index a90619de7c0a7..a2fa22e6bfbe8 100644 --- a/table/index.go +++ b/table/index.go @@ -25,7 +25,7 @@ import ( // IndexIterator is the interface for iterator of index data on KV store. type IndexIterator interface { - Next() (k []types.Datum, h int64, err error) + Next() (k []types.Datum, h kv.Handle, err error) Close() } @@ -64,15 +64,15 @@ type Index interface { // Meta returns IndexInfo. Meta() *model.IndexInfo // Create supports insert into statement. - Create(ctx sessionctx.Context, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64, opts ...CreateIdxOptFunc) (int64, error) + Create(ctx sessionctx.Context, rm kv.RetrieverMutator, indexedValues []types.Datum, h kv.Handle, opts ...CreateIdxOptFunc) (kv.Handle, error) // Delete supports delete from statement. - Delete(sc *stmtctx.StatementContext, m kv.Mutator, indexedValues []types.Datum, h int64) error + Delete(sc *stmtctx.StatementContext, m kv.Mutator, indexedValues []types.Datum, h kv.Handle) error // Drop supports drop table, drop index statements. Drop(rm kv.RetrieverMutator) error // Exist supports check index exists or not. - Exist(sc *stmtctx.StatementContext, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64) (bool, int64, error) + Exist(sc *stmtctx.StatementContext, rm kv.RetrieverMutator, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) // GenIndexKey generates an index key. - GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h int64, buf []byte) (key []byte, distinct bool, err error) + GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h kv.Handle, buf []byte) (key []byte, distinct bool, err error) // Seek supports where clause. Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues []types.Datum) (iter IndexIterator, hit bool, err error) // SeekFirst supports aggregate min and ascend order by. diff --git a/table/table.go b/table/table.go index 25ac7df26ccd8..072184e944115 100644 --- a/table/table.go +++ b/table/table.go @@ -148,10 +148,10 @@ type Table interface { IterRecords(ctx sessionctx.Context, startKey kv.Key, cols []*Column, fn RecordIterFunc) error // RowWithCols returns a row that contains the given cols. - RowWithCols(ctx sessionctx.Context, h int64, cols []*Column) ([]types.Datum, error) + RowWithCols(ctx sessionctx.Context, h kv.Handle, cols []*Column) ([]types.Datum, error) // Row returns a row for all columns. - Row(ctx sessionctx.Context, h int64) ([]types.Datum, error) + Row(ctx sessionctx.Context, h kv.Handle) ([]types.Datum, error) // Cols returns the columns of the table which is used in select, including hidden columns. Cols() []*Column @@ -189,16 +189,16 @@ type Table interface { FirstKey() kv.Key // RecordKey returns the key in KV storage for the row. - RecordKey(h int64) kv.Key + RecordKey(h kv.Handle) kv.Key // AddRecord inserts a row which should contain only public columns - AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...AddRecordOption) (recordID int64, err error) + AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...AddRecordOption) (recordID kv.Handle, err error) // UpdateRecord updates a row which should contain only writable columns. - UpdateRecord(ctx sessionctx.Context, h int64, currData, newData []types.Datum, touched []bool) error + UpdateRecord(ctx sessionctx.Context, h kv.Handle, currData, newData []types.Datum, touched []bool) error // RemoveRecord removes a row in the table. - RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error + RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []types.Datum) error // Allocators returns all allocators. Allocators(ctx sessionctx.Context) autoid.Allocators @@ -206,13 +206,13 @@ type Table interface { // RebaseAutoID rebases the auto_increment ID base. // If allocIDs is true, it will allocate some IDs and save to the cache. // If allocIDs is false, it will not allocate IDs. - RebaseAutoID(ctx sessionctx.Context, newBase int64, allocIDs bool) error + RebaseAutoID(ctx sessionctx.Context, newBase int64, allocIDs bool, tp autoid.AllocatorType) error // Meta returns TableInfo. Meta() *model.TableInfo // Seek returns the handle greater or equal to h. - Seek(ctx sessionctx.Context, h int64) (handle int64, found bool, err error) + Seek(ctx sessionctx.Context, h kv.Handle) (handle kv.Handle, found bool, err error) // Type returns the type of table Type() Type diff --git a/table/tables/index.go b/table/tables/index.go index fb670a7c7533d..a7bffae071962 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -36,15 +36,15 @@ import ( "github.com/pingcap/tidb/util/rowcodec" ) -// EncodeHandle encodes handle in data. -func EncodeHandle(h int64) []byte { +// EncodeHandleInUniqueIndexValue encodes handle in data. +func EncodeHandleInUniqueIndexValue(h int64) []byte { var data [8]byte binary.BigEndian.PutUint64(data[:], uint64(h)) return data[:] } -// DecodeHandle decodes handle in data. -func DecodeHandle(data []byte) (int64, error) { +// DecodeHandleInUniqueIndexValue decodes handle in data. +func DecodeHandleInUniqueIndexValue(data []byte) (int64, error) { dLen := len(data) if dLen <= tablecodec.MaxOldEncodeValueLen { return int64(binary.BigEndian.Uint64(data)), nil @@ -68,34 +68,36 @@ func (c *indexIter) Close() { } // Next returns current key and moves iterator to the next step. -func (c *indexIter) Next() (val []types.Datum, h int64, err error) { +func (c *indexIter) Next() (val []types.Datum, h kv.Handle, err error) { if !c.it.Valid() { - return nil, 0, errors.Trace(io.EOF) + return nil, nil, errors.Trace(io.EOF) } if !c.it.Key().HasPrefix(c.prefix) { - return nil, 0, errors.Trace(io.EOF) + return nil, nil, errors.Trace(io.EOF) } // get indexedValues buf := c.it.Key()[len(c.prefix):] vv, err := codec.Decode(buf, len(c.idx.idxInfo.Columns)) if err != nil { - return nil, 0, err + return nil, nil, err } if len(vv) > len(c.idx.idxInfo.Columns) { - h = vv[len(vv)-1].GetInt64() + h = kv.IntHandle(vv[len(vv)-1].GetInt64()) val = vv[0 : len(vv)-1] } else { // If the index is unique and the value isn't nil, the handle is in value. - h, err = DecodeHandle(c.it.Value()) + var iv int64 + iv, err = DecodeHandleInUniqueIndexValue(c.it.Value()) if err != nil { - return nil, 0, err + return nil, nil, err } + h = kv.IntHandle(iv) val = vv } // update new iter to next err = c.it.Next() if err != nil { - return nil, 0, err + return nil, nil, err } return } @@ -178,7 +180,7 @@ func TruncateIndexValuesIfNeeded(tblInfo *model.TableInfo, idxInfo *model.IndexI // GenIndexKey generates storage key for index values. Returned distinct indicates whether the // indexed values should be distinct in storage (i.e. whether handle is encoded in the key). -func (c *index) GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h int64, buf []byte) (key []byte, distinct bool, err error) { +func (c *index) GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h kv.Handle, buf []byte) (key []byte, distinct bool, err error) { if c.idxInfo.Unique { // See https://dev.mysql.com/doc/refman/5.7/en/create-index.html // A UNIQUE index creates a constraint such that all values in the index must be distinct. @@ -200,7 +202,7 @@ func (c *index) GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types. key = append(key, []byte(c.prefix)...) key, err = codec.EncodeKey(sc, key, indexedValues...) if !distinct && err == nil { - key, err = codec.EncodeKey(sc, key, types.NewDatum(h)) + key, err = codec.EncodeKey(sc, key, types.NewDatum(h.IntValue())) } if err != nil { return nil, false, err @@ -271,7 +273,7 @@ func (c *index) GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types. // // Layout: Handle | Flag // Length: 8 | 1 -func (c *index) Create(sctx sessionctx.Context, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64, opts ...table.CreateIdxOptFunc) (int64, error) { +func (c *index) Create(sctx sessionctx.Context, rm kv.RetrieverMutator, indexedValues []types.Datum, h kv.Handle, opts ...table.CreateIdxOptFunc) (kv.Handle, error) { var opt table.CreateIdxOpt for _, fn := range opts { fn(&opt) @@ -281,21 +283,21 @@ func (c *index) Create(sctx sessionctx.Context, rm kv.RetrieverMutator, indexedV skipCheck := vars.StmtCtx.BatchCheck key, distinct, err := c.GenIndexKey(vars.StmtCtx, indexedValues, h, writeBufs.IndexKeyBuf) if err != nil { - return 0, err + return nil, err } ctx := opt.Ctx if opt.Untouched { txn, err1 := sctx.Txn(true) if err1 != nil { - return 0, err1 + return nil, err1 } // If the index kv was untouched(unchanged), and the key/value already exists in mem-buffer, // should not overwrite the key with un-commit flag. // So if the key exists, just do nothing and return. _, err = txn.GetMemBuffer().Get(ctx, key) if err == nil { - return 0, nil + return nil, nil } } @@ -310,7 +312,7 @@ func (c *index) Create(sctx sessionctx.Context, rm kv.RetrieverMutator, indexedV rd := rowcodec.Encoder{Enable: true} rowRestoredValue, err := rd.Encode(sctx.GetSessionVars().StmtCtx, colIds, indexedValues, nil) if err != nil { - return 0, err + return nil, err } idxVal = make([]byte, 1+len(rowRestoredValue)) copy(idxVal[1:], rowRestoredValue) @@ -318,7 +320,7 @@ func (c *index) Create(sctx sessionctx.Context, rm kv.RetrieverMutator, indexedV if distinct { // The len of the idxVal is always >= 10 since len (restoredValue) > 0. tailLen += 8 - idxVal = append(idxVal, EncodeHandle(h)...) + idxVal = append(idxVal, EncodeHandleInUniqueIndexValue(h.IntValue())...) } else if len(idxVal) < 10 { // Padding the len to 10 paddingLen := 10 - len(idxVal) @@ -336,7 +338,7 @@ func (c *index) Create(sctx sessionctx.Context, rm kv.RetrieverMutator, indexedV } else { idxVal = make([]byte, 0) if distinct { - idxVal = EncodeHandle(h) + idxVal = EncodeHandleInUniqueIndexValue(h.IntValue()) } if opt.Untouched { // If index is untouched and fetch here means the key is exists in TiKV, but not in txn mem-buffer, @@ -351,7 +353,7 @@ func (c *index) Create(sctx sessionctx.Context, rm kv.RetrieverMutator, indexedV if !distinct || skipCheck || opt.Untouched { err = rm.Set(key, idxVal) - return 0, err + return nil, err } if ctx != nil { @@ -368,20 +370,20 @@ func (c *index) Create(sctx sessionctx.Context, rm kv.RetrieverMutator, indexedV if err != nil { if kv.IsErrNotFound(err) { err = rm.Set(key, idxVal) - return 0, err + return nil, err } - return 0, err + return nil, err } - handle, err := DecodeHandle(value) + handle, err := DecodeHandleInUniqueIndexValue(value) if err != nil { - return 0, err + return nil, err } - return handle, kv.ErrKeyExists + return kv.IntHandle(handle), kv.ErrKeyExists } // Delete removes the entry for handle h and indexdValues from KV index. -func (c *index) Delete(sc *stmtctx.StatementContext, m kv.Mutator, indexedValues []types.Datum, h int64) error { +func (c *index) Delete(sc *stmtctx.StatementContext, m kv.Mutator, indexedValues []types.Datum, h kv.Handle) error { key, _, err := c.GenIndexKey(sc, indexedValues, h, nil) if err != nil { return err @@ -417,7 +419,7 @@ func (c *index) Drop(rm kv.RetrieverMutator) error { // Seek searches KV index for the entry with indexedValues. func (c *index) Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues []types.Datum) (iter table.IndexIterator, hit bool, err error) { - key, _, err := c.GenIndexKey(sc, indexedValues, 0, nil) + key, _, err := c.GenIndexKey(sc, indexedValues, kv.IntHandle(0), nil) if err != nil { return nil, false, err } @@ -445,28 +447,29 @@ func (c *index) SeekFirst(r kv.Retriever) (iter table.IndexIterator, err error) return &indexIter{it: it, idx: c, prefix: c.prefix}, nil } -func (c *index) Exist(sc *stmtctx.StatementContext, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64) (bool, int64, error) { +func (c *index) Exist(sc *stmtctx.StatementContext, rm kv.RetrieverMutator, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) { key, distinct, err := c.GenIndexKey(sc, indexedValues, h, nil) if err != nil { - return false, 0, err + return false, nil, err } value, err := rm.Get(context.TODO(), key) if kv.IsErrNotFound(err) { - return false, 0, nil + return false, nil, nil } if err != nil { - return false, 0, err + return false, nil, err } // For distinct index, the value of key is handle. if distinct { - handle, err := DecodeHandle(value) + var iv int64 + iv, err := DecodeHandleInUniqueIndexValue(value) if err != nil { - return false, 0, err + return false, nil, err } - - if handle != h { + handle := kv.IntHandle(iv) + if !handle.Equal(h) { return true, handle, kv.ErrKeyExists } diff --git a/table/tables/index_test.go b/table/tables/index_test.go index 1ec6cc8e850d1..c147bd78179c8 100644 --- a/table/tables/index_test.go +++ b/table/tables/index_test.go @@ -15,12 +15,12 @@ package tables_test import ( "context" - "github.com/pingcap/parser/mysql" "io" "time" . "github.com/pingcap/check" "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -82,7 +82,7 @@ func (s *testIndexSuite) TestIndex(c *C) { values := types.MakeDatums(1, 2) mockCtx := mock.NewContext() - _, err = index.Create(mockCtx, txn, values, 1) + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) it, err := index.SeekFirst(txn) @@ -93,18 +93,18 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(getValues, HasLen, 2) c.Assert(getValues[0].GetInt64(), Equals, int64(1)) c.Assert(getValues[1].GetInt64(), Equals, int64(2)) - c.Assert(h, Equals, int64(1)) + c.Assert(h.IntValue(), Equals, int64(1)) it.Close() sc := &stmtctx.StatementContext{TimeZone: time.Local} - exist, _, err := index.Exist(sc, txn, values, 100) + exist, _, err := index.Exist(sc, txn, values, kv.IntHandle(100)) c.Assert(err, IsNil) c.Assert(exist, IsFalse) - exist, _, err = index.Exist(sc, txn, values, 1) + exist, _, err = index.Exist(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) c.Assert(exist, IsTrue) - err = index.Delete(sc, txn, values, 1) + err = index.Delete(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) it, err = index.SeekFirst(txn) @@ -114,7 +114,7 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(terror.ErrorEqual(err, io.EOF), IsTrue, Commentf("err %v", err)) it.Close() - _, err = index.Create(mockCtx, txn, values, 0) + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(0)) c.Assert(err, IsNil) _, err = index.SeekFirst(txn) @@ -169,10 +169,10 @@ func (s *testIndexSuite) TestIndex(c *C) { txn, err = s.s.Begin() c.Assert(err, IsNil) - _, err = index.Create(mockCtx, txn, values, 1) + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) - _, err = index.Create(mockCtx, txn, values, 2) + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(2)) c.Assert(err, NotNil) it, err = index.SeekFirst(txn) @@ -183,17 +183,17 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(getValues, HasLen, 2) c.Assert(getValues[0].GetInt64(), Equals, int64(1)) c.Assert(getValues[1].GetInt64(), Equals, int64(2)) - c.Assert(h, Equals, int64(1)) + c.Assert(h.IntValue(), Equals, int64(1)) it.Close() - exist, h, err = index.Exist(sc, txn, values, 1) + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) - c.Assert(h, Equals, int64(1)) + c.Assert(h.IntValue(), Equals, int64(1)) c.Assert(exist, IsTrue) - exist, h, err = index.Exist(sc, txn, values, 2) + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(2)) c.Assert(err, NotNil) - c.Assert(h, Equals, int64(1)) + c.Assert(h.IntValue(), Equals, int64(1)) c.Assert(exist, IsTrue) err = txn.Commit(context.Background()) @@ -204,7 +204,7 @@ func (s *testIndexSuite) TestIndex(c *C) { // Test the function of Next when the value of unique key is nil. values2 := types.MakeDatums(nil, nil) - _, err = index.Create(mockCtx, txn, values2, 2) + _, err = index.Create(mockCtx, txn, values2, kv.IntHandle(2)) c.Assert(err, IsNil) it, err = index.SeekFirst(txn) c.Assert(err, IsNil) @@ -213,7 +213,7 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(getValues, HasLen, 2) c.Assert(getValues[0].GetInterface(), Equals, nil) c.Assert(getValues[1].GetInterface(), Equals, nil) - c.Assert(h, Equals, int64(2)) + c.Assert(h.IntValue(), Equals, int64(2)) it.Close() } @@ -243,7 +243,7 @@ func (s *testIndexSuite) TestCombineIndexSeek(c *C) { mockCtx := mock.NewContext() values := types.MakeDatums("abc", "def") - _, err = index.Create(mockCtx, txn, values, 1) + _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) index2 := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) @@ -254,5 +254,5 @@ func (s *testIndexSuite) TestCombineIndexSeek(c *C) { c.Assert(hit, IsFalse) _, h, err := iter.Next() c.Assert(err, IsNil) - c.Assert(h, Equals, int64(1)) + c.Assert(h.IntValue(), Equals, int64(1)) } diff --git a/table/tables/partition.go b/table/tables/partition.go index 66e2cc688a2a1..eac6a82768486 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -245,7 +245,7 @@ func (t *partitionedTable) PartitionExpr() (*PartitionExpr, error) { func partitionRecordKey(pid int64, handle int64) kv.Key { recordPrefix := tablecodec.GenTableRecordPrefix(pid) - return tablecodec.EncodeRecordKey(recordPrefix, handle) + return tablecodec.EncodeRecordKey(recordPrefix, kv.IntHandle(handle)) } // locatePartition returns the partition ID of the input record. @@ -343,11 +343,11 @@ func (t *partitionedTable) GetPartitionByRow(ctx sessionctx.Context, r []types.D } // AddRecord implements the AddRecord method for the table.Table interface. -func (t *partitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID int64, err error) { +func (t *partitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) { partitionInfo := t.meta.GetPartitionInfo() pid, err := t.locatePartition(ctx, partitionInfo, r) if err != nil { - return 0, errors.Trace(err) + return nil, errors.Trace(err) } tbl := t.GetPartition(pid) @@ -355,7 +355,7 @@ func (t *partitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, op } // RemoveRecord implements table.Table RemoveRecord interface. -func (t *partitionedTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { +func (t *partitionedTable) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []types.Datum) error { partitionInfo := t.meta.GetPartitionInfo() pid, err := t.locatePartition(ctx, partitionInfo, r) if err != nil { @@ -369,7 +369,7 @@ func (t *partitionedTable) RemoveRecord(ctx sessionctx.Context, h int64, r []typ // UpdateRecord implements table.Table UpdateRecord interface. // `touched` means which columns are really modified, used for secondary indices. // Length of `oldData` and `newData` equals to length of `t.WritableCols()`. -func (t *partitionedTable) UpdateRecord(ctx sessionctx.Context, h int64, currData, newData []types.Datum, touched []bool) error { +func (t *partitionedTable) UpdateRecord(ctx sessionctx.Context, h kv.Handle, currData, newData []types.Datum, touched []bool) error { partitionInfo := t.meta.GetPartitionInfo() from, err := t.locatePartition(ctx, partitionInfo, currData) if err != nil { diff --git a/table/tables/partition_test.go b/table/tables/partition_test.go index eb1ce833a13b5..858cbd35f7cba 100644 --- a/table/tables/partition_test.go +++ b/table/tables/partition_test.go @@ -84,10 +84,10 @@ PARTITION BY RANGE ( id ) ( // Check that add record writes to the partition, rather than the table. txn, err := ts.se.Txn(true) c.Assert(err, IsNil) - val, err := txn.Get(context.TODO(), tables.PartitionRecordKey(p0.ID, rid)) + val, err := txn.Get(context.TODO(), tables.PartitionRecordKey(p0.ID, rid.IntValue())) c.Assert(err, IsNil) c.Assert(len(val), Greater, 0) - _, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.ID, rid)) + _, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.ID, rid.IntValue())) c.Assert(kv.ErrNotExist.Equal(err), IsTrue) // Cover more code. @@ -175,10 +175,10 @@ func (ts *testSuite) TestHashPartitionAddRecord(c *C) { // Check that add record writes to the partition, rather than the table. txn, err := ts.se.Txn(true) c.Assert(err, IsNil) - val, err := txn.Get(context.TODO(), tables.PartitionRecordKey(p0.ID, rid)) + val, err := txn.Get(context.TODO(), tables.PartitionRecordKey(p0.ID, rid.IntValue())) c.Assert(err, IsNil) c.Assert(len(val), Greater, 0) - _, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.ID, rid)) + _, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.ID, rid.IntValue())) c.Assert(kv.ErrNotExist.Equal(err), IsTrue) // Cover more code. @@ -211,10 +211,10 @@ func (ts *testSuite) TestHashPartitionAddRecord(c *C) { c.Assert(err, IsNil) txn, err = ts.se.Txn(true) c.Assert(err, IsNil) - val, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.Partition.Definitions[i].ID, rid)) + val, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.Partition.Definitions[i].ID, rid.IntValue())) c.Assert(err, IsNil) c.Assert(len(val), Greater, 0) - _, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.ID, rid)) + _, err = txn.Get(context.TODO(), tables.PartitionRecordKey(tbInfo.ID, rid.IntValue())) c.Assert(kv.ErrNotExist.Equal(err), IsTrue) } _, err = ts.se.Execute(context.Background(), "drop table if exists t1, t2;") diff --git a/table/tables/tables.go b/table/tables/tables.go index 3f1e9b85be41a..7d17fa58ed8d6 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -298,19 +298,19 @@ func (t *TableCommon) IndexPrefix() kv.Key { } // RecordKey implements table.Table interface. -func (t *TableCommon) RecordKey(h int64) kv.Key { +func (t *TableCommon) RecordKey(h kv.Handle) kv.Key { return tablecodec.EncodeRecordKey(t.recordPrefix, h) } // FirstKey implements table.Table interface. func (t *TableCommon) FirstKey() kv.Key { - return t.RecordKey(math.MinInt64) + return t.RecordKey(kv.IntHandle(math.MinInt64)) } // UpdateRecord implements table.Table UpdateRecord interface. // `touched` means which columns are really modified, used for secondary indices. // Length of `oldData` and `newData` equals to length of `t.WritableCols()`. -func (t *TableCommon) UpdateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, touched []bool) error { +func (t *TableCommon) UpdateRecord(ctx sessionctx.Context, h kv.Handle, oldData, newData []types.Datum, touched []bool) error { txn, err := ctx.Txn(true) if err != nil { return err @@ -375,8 +375,8 @@ func (t *TableCommon) UpdateRecord(ctx sessionctx.Context, h int64, oldData, new if shouldWriteBinlog(ctx) { if !t.meta.PKIsHandle { binlogColIDs = append(binlogColIDs, model.ExtraHandleID) - binlogOldRow = append(binlogOldRow, types.NewIntDatum(h)) - binlogNewRow = append(binlogNewRow, types.NewIntDatum(h)) + binlogOldRow = append(binlogOldRow, types.NewIntDatum(h.IntValue())) + binlogNewRow = append(binlogNewRow, types.NewIntDatum(h.IntValue())) } err = t.addUpdateBinlog(ctx, binlogOldRow, binlogNewRow, binlogColIDs) if err != nil { @@ -401,7 +401,7 @@ func (t *TableCommon) UpdateRecord(ctx sessionctx.Context, h int64, oldData, new return nil } -func (t *TableCommon) rebuildIndices(ctx sessionctx.Context, rm kv.RetrieverMutator, h int64, touched []bool, oldData []types.Datum, newData []types.Datum) error { +func (t *TableCommon) rebuildIndices(ctx sessionctx.Context, rm kv.RetrieverMutator, h kv.Handle, touched []bool, oldData []types.Datum, newData []types.Datum) error { txn, err := ctx.Txn(true) if err != nil { return err @@ -457,7 +457,7 @@ func adjustRowValuesBuf(writeBufs *variable.WriteStmtBufs, rowLen int) { } // AddRecord implements table.Table AddRecord interface. -func (t *TableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID int64, err error) { +func (t *TableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) { var opt table.AddRecordOpt for _, fn := range opts { fn.ApplyOn(&opt) @@ -469,12 +469,12 @@ func (t *TableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts .. // Currently, only insert can set _tidb_rowid, update can not update _tidb_rowid. if len(r) > len(cols) && !opt.IsUpdate { // The last value is _tidb_rowid. - recordID = r[len(r)-1].GetInt64() + recordID = kv.IntHandle(r[len(r)-1].GetInt64()) hasRecordID = true } else { tblInfo := t.Meta() if tblInfo.PKIsHandle { - recordID = r[tblInfo.GetPkColInfo().Offset].GetInt64() + recordID = kv.IntHandle(r[tblInfo.GetPkColInfo().Offset].GetInt64()) hasRecordID = true } } @@ -487,19 +487,19 @@ func (t *TableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts .. stmtCtx := ctx.GetSessionVars().StmtCtx stmtCtx.BaseRowID, stmtCtx.MaxRowID, err = allocHandleIDs(ctx, t, uint64(opt.ReserveAutoID)) if err != nil { - return 0, err + return nil, err } } recordID, err = AllocHandle(ctx, t) if err != nil { - return 0, err + return nil, err } } txn, err := ctx.Txn(true) if err != nil { - return 0, err + return nil, err } execBuf := kv.NewStagingBufferStore(txn) defer execBuf.Discard() @@ -533,7 +533,7 @@ func (t *TableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts .. // If col is in write only or write reorganization state, we must add it with its default value. value, err = table.GetColOriginDefaultValue(ctx, col.ToInfo()) if err != nil { - return 0, err + return nil, err } // add value to `r` for dirty db in transaction. // Otherwise when update will panic cause by get value of column in write only state from dirty db. @@ -556,15 +556,15 @@ func (t *TableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts .. sc, rd := sessVars.StmtCtx, &sessVars.RowEncoder writeBufs.RowValBuf, err = tablecodec.EncodeRow(sc, row, colIDs, writeBufs.RowValBuf, writeBufs.AddRowValues, rd) if err != nil { - return 0, err + return nil, err } value := writeBufs.RowValBuf if err = execBuf.Set(key, value); err != nil { - return 0, err + return nil, err } if _, err := execBuf.Flush(); err != nil { - return 0, err + return nil, err } ctx.StmtAddDirtyTableOP(table.DirtyTableAddRow, t.physicalTableID, recordID) @@ -574,7 +574,7 @@ func (t *TableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts .. binlogColIDs = colIDs err = t.addInsertBinlog(ctx, recordID, binlogRow, binlogColIDs) if err != nil { - return 0, err + return nil, err } } sc.AddAffectedRows(1) @@ -612,11 +612,11 @@ func (t *TableCommon) genIndexKeyStr(colVals []types.Datum) (string, error) { } // addIndices adds data into indices. If any key is duplicated, returns the original handle. -func (t *TableCommon) addIndices(sctx sessionctx.Context, recordID int64, r []types.Datum, rm kv.RetrieverMutator, - opts []table.CreateIdxOptFunc) (int64, error) { +func (t *TableCommon) addIndices(sctx sessionctx.Context, recordID kv.Handle, r []types.Datum, rm kv.RetrieverMutator, + opts []table.CreateIdxOptFunc) (kv.Handle, error) { txn, err := sctx.Txn(true) if err != nil { - return 0, err + return nil, err } // Clean up lazy check error environment defer txn.DelOption(kv.PresumeKeyNotExistsError) @@ -642,13 +642,13 @@ func (t *TableCommon) addIndices(sctx sessionctx.Context, recordID int64, r []ty for _, v := range t.WritableIndices() { indexVals, err = v.FetchValues(r, indexVals) if err != nil { - return 0, err + return nil, err } var dupErr error if !skipCheck && v.Meta().Unique { entryKey, err := t.genIndexKeyStr(indexVals) if err != nil { - return 0, err + return nil, err } existErrInfo := kv.NewExistErrInfo(v.Meta().Name.String(), entryKey) txn.SetOption(kv.PresumeKeyNotExistsError, existErrInfo) @@ -658,17 +658,17 @@ func (t *TableCommon) addIndices(sctx sessionctx.Context, recordID int64, r []ty if kv.ErrKeyExists.Equal(err) { return dupHandle, dupErr } - return 0, err + return nil, err } txn.DelOption(kv.PresumeKeyNotExistsError) } // save the buffer, multi rows insert can use it. writeBufs.IndexValsBuf = indexVals - return 0, nil + return nil, nil } // RowWithCols implements table.Table RowWithCols interface. -func (t *TableCommon) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Column) ([]types.Datum, error) { +func (t *TableCommon) RowWithCols(ctx sessionctx.Context, h kv.Handle, cols []*table.Column) ([]types.Datum, error) { // Get raw row data from kv. key := t.RecordKey(h) txn, err := ctx.Txn(true) @@ -687,7 +687,7 @@ func (t *TableCommon) RowWithCols(ctx sessionctx.Context, h int64, cols []*table } // DecodeRawRowData decodes raw row data into a datum slice and a (columnID:columnValue) map. -func DecodeRawRowData(ctx sessionctx.Context, meta *model.TableInfo, h int64, cols []*table.Column, +func DecodeRawRowData(ctx sessionctx.Context, meta *model.TableInfo, h kv.Handle, cols []*table.Column, value []byte) ([]types.Datum, map[int64]types.Datum, error) { v := make([]types.Datum, len(cols)) colTps := make(map[int64]*types.FieldType, len(cols)) @@ -697,9 +697,9 @@ func DecodeRawRowData(ctx sessionctx.Context, meta *model.TableInfo, h int64, co } if col.IsPKHandleColumn(meta) { if mysql.HasUnsignedFlag(col.Flag) { - v[i].SetUint64(uint64(h)) + v[i].SetUint64(uint64(h.IntValue())) } else { - v[i].SetInt64(h) + v[i].SetInt64(h.IntValue()) } continue } @@ -731,12 +731,12 @@ func DecodeRawRowData(ctx sessionctx.Context, meta *model.TableInfo, h int64, co } // Row implements table.Table Row interface. -func (t *TableCommon) Row(ctx sessionctx.Context, h int64) ([]types.Datum, error) { +func (t *TableCommon) Row(ctx sessionctx.Context, h kv.Handle) ([]types.Datum, error) { return t.RowWithCols(ctx, h, t.Cols()) } // RemoveRecord implements table.Table RemoveRecord interface. -func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { +func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []types.Datum) error { err := t.removeRowData(ctx, h) if err != nil { return err @@ -758,7 +758,7 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Da colIDs = append(colIDs, model.ExtraHandleID) binlogRow = make([]types.Datum, 0, len(r)+1) binlogRow = append(binlogRow, r...) - binlogRow = append(binlogRow, types.NewIntDatum(h)) + binlogRow = append(binlogRow, types.NewIntDatum(h.IntValue())) } else { binlogRow = r } @@ -777,9 +777,9 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Da return err } -func (t *TableCommon) addInsertBinlog(ctx sessionctx.Context, h int64, row []types.Datum, colIDs []int64) error { +func (t *TableCommon) addInsertBinlog(ctx sessionctx.Context, h kv.Handle, row []types.Datum, colIDs []int64) error { mutation := t.getMutation(ctx) - pk, err := codec.EncodeValue(ctx.GetSessionVars().StmtCtx, nil, types.NewIntDatum(h)) + pk, err := codec.EncodeValue(ctx.GetSessionVars().StmtCtx, nil, types.NewIntDatum(h.IntValue())) if err != nil { return err } @@ -843,7 +843,7 @@ func writeSequenceUpdateValueBinlog(ctx sessionctx.Context, db, sequence string, return err } -func (t *TableCommon) removeRowData(ctx sessionctx.Context, h int64) error { +func (t *TableCommon) removeRowData(ctx sessionctx.Context, h kv.Handle) error { // Remove row data. txn, err := ctx.Txn(true) if err != nil { @@ -851,7 +851,7 @@ func (t *TableCommon) removeRowData(ctx sessionctx.Context, h int64) error { } key := t.RecordKey(h) - err = txn.Delete([]byte(key)) + err = txn.Delete(key) if err != nil { return err } @@ -859,7 +859,7 @@ func (t *TableCommon) removeRowData(ctx sessionctx.Context, h int64) error { } // removeRowIndices removes all the indices of a row. -func (t *TableCommon) removeRowIndices(ctx sessionctx.Context, h int64, rec []types.Datum) error { +func (t *TableCommon) removeRowIndices(ctx sessionctx.Context, h kv.Handle, rec []types.Datum) error { txn, err := ctx.Txn(true) if err != nil { return err @@ -867,14 +867,14 @@ func (t *TableCommon) removeRowIndices(ctx sessionctx.Context, h int64, rec []ty for _, v := range t.DeletableIndices() { vals, err := v.FetchValues(rec, nil) if err != nil { - logutil.BgLogger().Info("remove row index failed", zap.Any("index", v.Meta()), zap.Uint64("txnStartTS", txn.StartTS()), zap.Int64("handle", h), zap.Any("record", rec), zap.Error(err)) + logutil.BgLogger().Info("remove row index failed", zap.Any("index", v.Meta()), zap.Uint64("txnStartTS", txn.StartTS()), zap.String("handle", h.String()), zap.Any("record", rec), zap.Error(err)) return err } if err = v.Delete(ctx.GetSessionVars().StmtCtx, txn, vals, h); err != nil { if v.Meta().State != model.StatePublic && kv.ErrNotExist.Equal(err) { // If the index is not in public state, we may have not created the index, // or already deleted the index, so skip ErrNotExist error. - logutil.BgLogger().Debug("row index not exists", zap.Any("index", v.Meta()), zap.Uint64("txnStartTS", txn.StartTS()), zap.Int64("handle", h)) + logutil.BgLogger().Debug("row index not exists", zap.Any("index", v.Meta()), zap.Uint64("txnStartTS", txn.StartTS()), zap.String("handle", h.String())) continue } return err @@ -884,12 +884,12 @@ func (t *TableCommon) removeRowIndices(ctx sessionctx.Context, h int64, rec []ty } // removeRowIndex implements table.Table RemoveRowIndex interface. -func (t *TableCommon) removeRowIndex(sc *stmtctx.StatementContext, rm kv.RetrieverMutator, h int64, vals []types.Datum, idx table.Index, txn kv.Transaction) error { +func (t *TableCommon) removeRowIndex(sc *stmtctx.StatementContext, rm kv.RetrieverMutator, h kv.Handle, vals []types.Datum, idx table.Index, txn kv.Transaction) error { return idx.Delete(sc, rm, vals, h) } // buildIndexForRow implements table.Table BuildIndexForRow interface. -func (t *TableCommon) buildIndexForRow(ctx sessionctx.Context, rm kv.RetrieverMutator, h int64, vals []types.Datum, idx table.Index, txn kv.Transaction, untouched bool) error { +func (t *TableCommon) buildIndexForRow(ctx sessionctx.Context, rm kv.RetrieverMutator, h kv.Handle, vals []types.Datum, idx table.Index, txn kv.Transaction, untouched bool) error { var opts []table.CreateIdxOptFunc if untouched { opts = append(opts, table.IndexIsUntouched) @@ -952,9 +952,9 @@ func (t *TableCommon) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols for _, col := range cols { if col.IsPKHandleColumn(t.meta) { if mysql.HasUnsignedFlag(col.Flag) { - data[col.Offset].SetUint64(uint64(handle)) + data[col.Offset].SetUint64(uint64(handle.IntValue())) } else { - data[col.Offset].SetInt64(handle) + data[col.Offset].SetInt64(handle.IntValue()) } continue } @@ -967,7 +967,7 @@ func (t *TableCommon) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols return err } } - more, err := fn(handle, data, cols) + more, err := fn(handle.IntValue(), data, cols) if !more || err != nil { return err } @@ -1007,19 +1007,19 @@ func GetColDefaultValue(ctx sessionctx.Context, col *table.Column, defaultVals [ // AllocHandle allocate a new handle. // A statement could reserve some ID in the statement context, try those ones first. -func AllocHandle(ctx sessionctx.Context, t table.Table) (int64, error) { +func AllocHandle(ctx sessionctx.Context, t table.Table) (kv.Handle, error) { if ctx != nil { if stmtCtx := ctx.GetSessionVars().StmtCtx; stmtCtx != nil { // First try to alloc if the statement has reserved auto ID. if stmtCtx.BaseRowID < stmtCtx.MaxRowID { stmtCtx.BaseRowID += 1 - return stmtCtx.BaseRowID, nil + return kv.IntHandle(stmtCtx.BaseRowID), nil } } } _, rowID, err := allocHandleIDs(ctx, t, 1) - return rowID, err + return kv.IntHandle(rowID), err } func allocHandleIDs(ctx sessionctx.Context, t table.Table, n uint64) (int64, int64, error) { @@ -1091,28 +1091,29 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { } // RebaseAutoID implements table.Table RebaseAutoID interface. -func (t *TableCommon) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { - return t.Allocators(ctx).Get(autoid.RowIDAllocType).Rebase(t.tableID, newBase, isSetStep) +// Both auto-increment and auto-random can use this function to do rebase on explicit newBase value (without shadow bits). +func (t *TableCommon) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool, tp autoid.AllocatorType) error { + return t.Allocators(ctx).Get(tp).Rebase(t.tableID, newBase, isSetStep) } // Seek implements table.Table Seek interface. -func (t *TableCommon) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) { +func (t *TableCommon) Seek(ctx sessionctx.Context, h kv.Handle) (kv.Handle, bool, error) { txn, err := ctx.Txn(true) if err != nil { - return 0, false, err + return nil, false, err } seekKey := tablecodec.EncodeRowKeyWithHandle(t.physicalTableID, h) iter, err := txn.Iter(seekKey, t.RecordPrefix().PrefixNext()) if err != nil { - return 0, false, err + return nil, false, err } if !iter.Valid() || !iter.Key().HasPrefix(t.RecordPrefix()) { // No more records in the table, skip to the end. - return 0, false, nil + return nil, false, nil } handle, err := tablecodec.DecodeRowKey(iter.Key()) if err != nil { - return 0, false, err + return nil, false, err } return handle, true, nil } @@ -1179,7 +1180,7 @@ func FindIndexByColName(t table.Table, name string) table.Index { // CheckHandleExists check whether recordID key exists. if not exists, return nil, // otherwise return kv.ErrKeyExists error. -func CheckHandleExists(ctx context.Context, sctx sessionctx.Context, t table.Table, recordID int64, data []types.Datum) error { +func CheckHandleExists(ctx context.Context, sctx sessionctx.Context, t table.Table, recordID kv.Handle, data []types.Datum) error { if pt, ok := t.(*partitionedTable); ok { info := t.Meta().GetPartitionInfo() pid, err := pt.locatePartition(sctx, info, data) @@ -1194,7 +1195,7 @@ func CheckHandleExists(ctx context.Context, sctx sessionctx.Context, t table.Tab } // Check key exists. recordKey := t.RecordKey(recordID) - existErrInfo := kv.NewExistErrInfo("PRIMARY", strconv.Itoa(int(recordID))) + existErrInfo := kv.NewExistErrInfo("PRIMARY", recordID.String()) txn.SetOption(kv.PresumeKeyNotExistsError, existErrInfo) defer txn.DelOption(kv.PresumeKeyNotExistsError) _, err = txn.Get(ctx, recordKey) diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index bb2bc25c0378c..04f9144ca6ec2 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -103,12 +103,12 @@ func (ts *testSuite) TestBasic(c *C) { handle, err := tables.AllocHandle(nil, tb) c.Assert(err, IsNil) - c.Assert(handle, Greater, int64(0)) + c.Assert(handle.IntValue(), Greater, int64(0)) ctx := ts.se rid, err := tb.AddRecord(ctx, types.MakeDatums(1, "abc")) c.Assert(err, IsNil) - c.Assert(rid, Greater, int64(0)) + c.Assert(rid.IntValue(), Greater, int64(0)) row, err := tb.Row(ctx, rid) c.Assert(err, IsNil) c.Assert(len(row), Equals, 2) @@ -132,12 +132,12 @@ func (ts *testSuite) TestBasic(c *C) { } // RowWithCols test - vals, err := tb.RowWithCols(ctx, 1, tb.Cols()) + vals, err := tb.RowWithCols(ctx, kv.IntHandle(1), tb.Cols()) c.Assert(err, IsNil) c.Assert(vals, HasLen, 2) c.Assert(vals[0].GetInt64(), Equals, int64(1)) cols := []*table.Column{tb.Cols()[1]} - vals, err = tb.RowWithCols(ctx, 1, cols) + vals, err = tb.RowWithCols(ctx, kv.IntHandle(1), cols) c.Assert(err, IsNil) c.Assert(vals, HasLen, 1) c.Assert(vals[0].GetBytes(), DeepEquals, []byte("cba")) @@ -150,8 +150,8 @@ func (ts *testSuite) TestBasic(c *C) { _, err = tb.AddRecord(ctx, types.MakeDatums(1, "abc")) c.Assert(err, IsNil) c.Assert(indexCnt(), Greater, 0) - handle, found, err := tb.Seek(ctx, 0) - c.Assert(handle, Equals, int64(1)) + handle, found, err := tb.Seek(ctx, kv.IntHandle(0)) + c.Assert(handle.IntValue(), Equals, int64(1)) c.Assert(found, Equals, true) c.Assert(err, IsNil) _, err = ts.se.Execute(context.Background(), "drop table test.t") @@ -161,7 +161,7 @@ func (ts *testSuite) TestBasic(c *C) { alc := tb.Allocators(nil).Get(autoid.RowIDAllocType) c.Assert(alc, NotNil) - err = tb.RebaseAutoID(nil, 0, false) + err = tb.RebaseAutoID(nil, 0, false, autoid.RowIDAllocType) c.Assert(err, IsNil) } @@ -247,7 +247,7 @@ func (ts *testSuite) TestUniqueIndexMultipleNullEntries(c *C) { handle, err := tables.AllocHandle(nil, tb) c.Assert(err, IsNil) - c.Assert(handle, Greater, int64(0)) + c.Assert(handle.IntValue(), Greater, int64(0)) autoid, err := table.AllocAutoIncrementValue(context.Background(), tb, ts.se) c.Assert(err, IsNil) @@ -279,15 +279,15 @@ func (ts *testSuite) TestRowKeyCodec(c *C) { } for _, t := range tableVal { - b := tablecodec.EncodeRowKeyWithHandle(t.tableID, t.h) + b := tablecodec.EncodeRowKeyWithHandle(t.tableID, kv.IntHandle(t.h)) tableID, handle, err := tablecodec.DecodeRecordKey(b) c.Assert(err, IsNil) c.Assert(tableID, Equals, t.tableID) - c.Assert(handle, Equals, t.h) + c.Assert(handle.IntValue(), Equals, t.h) handle, err = tablecodec.DecodeRowKey(b) c.Assert(err, IsNil) - c.Assert(handle, Equals, t.h) + c.Assert(handle.IntValue(), Equals, t.h) } // test error @@ -388,7 +388,7 @@ func (ts *testSuite) TestTableFromMeta(c *C) { c.Assert(err, IsNil) maxID := 1<<(64-15-1) - 1 - err = tb.RebaseAutoID(tk.Se, int64(maxID), false) + err = tb.RebaseAutoID(tk.Se, int64(maxID), false, autoid.RowIDAllocType) c.Assert(err, IsNil) _, err = tables.AllocHandle(tk.Se, tb) diff --git a/tablecodec/bench_test.go b/tablecodec/bench_test.go index 8cc5c09549763..b025e8beb96bc 100644 --- a/tablecodec/bench_test.go +++ b/tablecodec/bench_test.go @@ -15,18 +15,20 @@ package tablecodec import ( "testing" + + "github.com/pingcap/tidb/kv" ) func BenchmarkEncodeRowKeyWithHandle(b *testing.B) { for i := 0; i < b.N; i++ { - EncodeRowKeyWithHandle(100, 100) + EncodeRowKeyWithHandle(100, kv.IntHandle(100)) } } func BenchmarkEncodeEndKey(b *testing.B) { for i := 0; i < b.N; i++ { - EncodeRowKeyWithHandle(100, 100) - EncodeRowKeyWithHandle(100, 101) + EncodeRowKeyWithHandle(100, kv.IntHandle(100)) + EncodeRowKeyWithHandle(100, kv.IntHandle(101)) } } @@ -36,13 +38,13 @@ func BenchmarkEncodeEndKey(b *testing.B) { // BenchmarkEncodeRowKeyWithPrefixNex-4 10000000 121 ns/op func BenchmarkEncodeRowKeyWithPrefixNex(b *testing.B) { for i := 0; i < b.N; i++ { - sk := EncodeRowKeyWithHandle(100, 100) + sk := EncodeRowKeyWithHandle(100, kv.IntHandle(100)) sk.PrefixNext() } } func BenchmarkDecodeRowKey(b *testing.B) { - rowKey := EncodeRowKeyWithHandle(100, 100) + rowKey := EncodeRowKeyWithHandle(100, kv.IntHandle(100)) for i := 0; i < b.N; i++ { DecodeRowKey(rowKey) } diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index 8871a21dbf1a3..ad007400ba2d5 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -66,18 +66,15 @@ func TablePrefix() []byte { // EncodeRowKey encodes the table id and record handle into a kv.Key func EncodeRowKey(tableID int64, encodedHandle []byte) kv.Key { - buf := make([]byte, 0, RecordRowKeyLen) + buf := make([]byte, 0, prefixLen+len(encodedHandle)) buf = appendTableRecordPrefix(buf, tableID) buf = append(buf, encodedHandle...) return buf } // EncodeRowKeyWithHandle encodes the table id, row handle into a kv.Key -func EncodeRowKeyWithHandle(tableID int64, handle int64) kv.Key { - buf := make([]byte, 0, RecordRowKeyLen) - buf = appendTableRecordPrefix(buf, tableID) - buf = codec.EncodeInt(buf, handle) - return buf +func EncodeRowKeyWithHandle(tableID int64, handle kv.Handle) kv.Key { + return EncodeRowKey(tableID, handle.Encoded()) } // CutRowKeyPrefix cuts the row key prefix. @@ -86,10 +83,10 @@ func CutRowKeyPrefix(key kv.Key) []byte { } // EncodeRecordKey encodes the recordPrefix, row handle into a kv.Key. -func EncodeRecordKey(recordPrefix kv.Key, h int64) kv.Key { - buf := make([]byte, 0, len(recordPrefix)+idLen) +func EncodeRecordKey(recordPrefix kv.Key, h kv.Handle) kv.Key { + buf := make([]byte, 0, len(recordPrefix)+h.Len()) buf = append(buf, recordPrefix...) - buf = codec.EncodeInt(buf, h) + buf = append(buf, h.Encoded()...) return buf } @@ -102,31 +99,33 @@ func hasRecordPrefixSep(key kv.Key) bool { } // DecodeRecordKey decodes the key and gets the tableID, handle. -func DecodeRecordKey(key kv.Key) (tableID int64, handle int64, err error) { +func DecodeRecordKey(key kv.Key) (tableID int64, handle kv.Handle, err error) { if len(key) <= prefixLen { - return 0, 0, errInvalidRecordKey.GenWithStack("invalid record key - %q", key) + return 0, nil, errInvalidRecordKey.GenWithStack("invalid record key - %q", key) } k := key if !hasTablePrefix(key) { - return 0, 0, errInvalidRecordKey.GenWithStack("invalid record key - %q", k) + return 0, nil, errInvalidRecordKey.GenWithStack("invalid record key - %q", k) } key = key[tablePrefixLength:] key, tableID, err = codec.DecodeInt(key) if err != nil { - return 0, 0, errors.Trace(err) + return 0, nil, errors.Trace(err) } if !hasRecordPrefixSep(key) { - return 0, 0, errInvalidRecordKey.GenWithStack("invalid record key - %q", k) + return 0, nil, errInvalidRecordKey.GenWithStack("invalid record key - %q", k) } key = key[recordPrefixSepLength:] - key, handle, err = codec.DecodeInt(key) + var intHandle int64 + key, intHandle, err = codec.DecodeInt(key) if err != nil { - return 0, 0, errors.Trace(err) + return 0, nil, errors.Trace(err) } + handle = kv.IntHandle(intHandle) return } @@ -241,12 +240,12 @@ func DecodeTableID(key kv.Key) int64 { } // DecodeRowKey decodes the key and gets the handle. -func DecodeRowKey(key kv.Key) (int64, error) { +func DecodeRowKey(key kv.Key) (kv.Handle, error) { if len(key) != RecordRowKeyLen || !hasTablePrefix(key) || !hasRecordPrefixSep(key[prefixLen-2:]) { - return 0, errInvalidKey.GenWithStack("invalid key - %q", key) + return kv.IntHandle(0), errInvalidKey.GenWithStack("invalid key - %q", key) } u := binary.BigEndian.Uint64(key[prefixLen:]) - return codec.DecodeCmpUintToInt(u), nil + return kv.IntHandle(codec.DecodeCmpUintToInt(u)), nil } // EncodeValue encodes a go value to bytes. @@ -379,8 +378,8 @@ func DecodeRowWithMapNew(b []byte, cols map[int64]*types.FieldType, loc *time.Lo // for decodeToMap: // - no need handle // - no need get default value - rd := rowcodec.NewDatumMapDecoder(reqCols, -1, loc) - return rd.DecodeToDatumMap(b, -1, row) + rd := rowcodec.NewDatumMapDecoder(reqCols, nil, loc) + return rd.DecodeToDatumMap(b, nil, row) } // DecodeRowWithMap decodes a byte slice into datums with a existing row map. @@ -648,7 +647,7 @@ func decodeIndexKvNewCollation(key, value []byte, colsLen int, hdStatus HandleSt colIDs[col.ID] = i } // We don't need to decode handle here, and colIDs >= 0 always. - rd := rowcodec.NewByteDecoder(columns, -1, nil, nil) + rd := rowcodec.NewByteDecoder(columns, []int64{-1}, nil, nil) vLen := len(value) tailLen := int(value[0]) resultValues, err := rd.DecodeToBytesNoHandle(colIDs, value[1:vLen-tailLen]) @@ -829,8 +828,8 @@ func TruncateToRowKeyLen(key kv.Key) kv.Key { // GetTableHandleKeyRange returns table handle's key range with tableID. func GetTableHandleKeyRange(tableID int64) (startKey, endKey []byte) { - startKey = EncodeRowKeyWithHandle(tableID, math.MinInt64) - endKey = EncodeRowKeyWithHandle(tableID, math.MaxInt64) + startKey = EncodeRowKeyWithHandle(tableID, kv.IntHandle(math.MinInt64)) + endKey = EncodeRowKeyWithHandle(tableID, kv.IntHandle(math.MaxInt64)) return } diff --git a/tablecodec/tablecodec_test.go b/tablecodec/tablecodec_test.go index c45aadd211afc..64a11abc184f3 100644 --- a/tablecodec/tablecodec_test.go +++ b/tablecodec/tablecodec_test.go @@ -46,12 +46,12 @@ func (s *testTableCodecSuite) TestTableCodec(c *C) { key := EncodeRowKey(1, codec.EncodeInt(nil, 2)) h, err := DecodeRowKey(key) c.Assert(err, IsNil) - c.Assert(h, Equals, int64(2)) + c.Assert(h.IntValue(), Equals, int64(2)) - key = EncodeRowKeyWithHandle(1, 2) + key = EncodeRowKeyWithHandle(1, kv.IntHandle(2)) h, err = DecodeRowKey(key) c.Assert(err, IsNil) - c.Assert(h, Equals, int64(2)) + c.Assert(h.IntValue(), Equals, int64(2)) } // column is a structure used for test @@ -345,7 +345,7 @@ func (s *testTableCodecSuite) TestIndexKey(c *C) { func (s *testTableCodecSuite) TestRecordKey(c *C) { tableID := int64(55) - tableKey := EncodeRowKeyWithHandle(tableID, math.MaxUint32) + tableKey := EncodeRowKeyWithHandle(tableID, kv.IntHandle(math.MaxUint32)) tTableID, _, isRecordKey, err := DecodeKeyHead(tableKey) c.Assert(err, IsNil) c.Assert(tTableID, Equals, tableID) @@ -357,10 +357,10 @@ func (s *testTableCodecSuite) TestRecordKey(c *C) { tTableID, handle, err := DecodeRecordKey(rowKey) c.Assert(err, IsNil) c.Assert(tTableID, Equals, tableID) - c.Assert(handle, Equals, int64(math.MaxUint32)) + c.Assert(handle.IntValue(), Equals, int64(math.MaxUint32)) recordPrefix := GenTableRecordPrefix(tableID) - rowKey = EncodeRecordKey(recordPrefix, math.MaxUint32) + rowKey = EncodeRecordKey(recordPrefix, kv.IntHandle(math.MaxUint32)) c.Assert([]byte(tableKey), BytesEquals, []byte(rowKey)) _, _, err = DecodeRecordKey(nil) @@ -374,9 +374,9 @@ func (s *testTableCodecSuite) TestPrefix(c *C) { const tableID int64 = 66 key := EncodeTablePrefix(tableID) tTableID := DecodeTableID(key) - c.Assert(tTableID, Equals, int64(tableID)) + c.Assert(tTableID, Equals, tableID) - c.Assert([]byte(TablePrefix()), BytesEquals, tablePrefix) + c.Assert(TablePrefix(), BytesEquals, tablePrefix) tablePrefix1 := GenTablePrefix(tableID) c.Assert([]byte(tablePrefix1), BytesEquals, []byte(key)) @@ -397,7 +397,7 @@ func (s *testTableCodecSuite) TestPrefix(c *C) { func (s *testTableCodecSuite) TestReplaceRecordKeyTableID(c *C) { tableID := int64(1) - tableKey := EncodeRowKeyWithHandle(tableID, 1) + tableKey := EncodeRowKeyWithHandle(tableID, kv.IntHandle(1)) tTableID, _, _, err := DecodeKeyHead(tableKey) c.Assert(err, IsNil) c.Assert(tTableID, Equals, tableID) @@ -467,20 +467,20 @@ func (s *testTableCodecSuite) TestCutPrefix(c *C) { func (s *testTableCodecSuite) TestRange(c *C) { s1, e1 := GetTableHandleKeyRange(22) s2, e2 := GetTableHandleKeyRange(23) - c.Assert([]byte(s1), Less, []byte(e1)) - c.Assert([]byte(e1), Less, []byte(s2)) - c.Assert([]byte(s2), Less, []byte(e2)) + c.Assert(s1, Less, e1) + c.Assert(e1, Less, s2) + c.Assert(s2, Less, e2) s1, e1 = GetTableIndexKeyRange(42, 666) s2, e2 = GetTableIndexKeyRange(42, 667) - c.Assert([]byte(s1), Less, []byte(e1)) - c.Assert([]byte(e1), Less, []byte(s2)) - c.Assert([]byte(s2), Less, []byte(e2)) + c.Assert(s1, Less, e1) + c.Assert(e1, Less, s2) + c.Assert(s2, Less, e2) } func (s *testTableCodecSuite) TestDecodeAutoIDMeta(c *C) { keyBytes := []byte{0x6d, 0x44, 0x42, 0x3a, 0x35, 0x36, 0x0, 0x0, 0x0, 0xfc, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x68, 0x54, 0x49, 0x44, 0x3a, 0x31, 0x30, 0x38, 0x0, 0xfe} - key, field, err := DecodeMetaKey(kv.Key(keyBytes)) + key, field, err := DecodeMetaKey(keyBytes) c.Assert(err, IsNil) c.Assert(string(key), Equals, "DB:56") c.Assert(string(field), Equals, "TID:108") diff --git a/tidb-server/main.go b/tidb-server/main.go index a873bbaaac673..e50e1c89defd1 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -62,6 +62,7 @@ import ( "github.com/pingcap/tidb/util/signal" "github.com/pingcap/tidb/util/storeutil" "github.com/pingcap/tidb/util/sys/linux" + storageSys "github.com/pingcap/tidb/util/sys/storage" "github.com/pingcap/tidb/util/systimemon" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" @@ -168,6 +169,7 @@ func main() { if config.GetGlobalConfig().OOMUseTmpStorage { config.GetGlobalConfig().UpdateTempStoragePath() initializeTempDir() + checkTempStorageQuota() } setGlobalVars() setCPUAffinity() @@ -232,6 +234,21 @@ func initializeTempDir() { } } +func checkTempStorageQuota() { + // check capacity and the quota when OOMUseTmpStorage is enabled + c := config.GetGlobalConfig() + if c.TempStorageQuota < 0 { + // means unlimited, do nothing + } else { + capacityByte, err := storageSys.GetTargetDirectoryCapacity(c.TempStoragePath) + if err != nil { + log.Fatal(err.Error()) + } else if capacityByte < uint64(c.TempStorageQuota) { + log.Fatal(fmt.Sprintf("value of [tmp-storage-quota](%d byte) exceeds the capacity(%d byte) of the [%s] directory", c.TempStorageQuota, capacityByte, c.TempStoragePath)) + } + } +} + func setCPUAffinity() { if affinityCPU == nil || len(*affinityCPU) == 0 { return @@ -542,6 +559,7 @@ func setGlobalVars() { priority := mysql.Str2Priority(cfg.Performance.ForcePriority) variable.ForcePriority = int32(priority) variable.SysVars[variable.TiDBForcePriority].Value = mysql.Priority2Str[priority] + variable.SysVars[variable.TiDBOptDistinctAggPushDown].Value = variable.BoolToIntStr(cfg.Performance.DistinctAggPushDown) variable.SysVars[variable.TIDBMemQuotaQuery].Value = strconv.FormatInt(cfg.MemQuotaQuery, 10) variable.SysVars["lower_case_table_names"].Value = strconv.Itoa(cfg.LowerCaseTableNames) @@ -551,6 +569,7 @@ func setGlobalVars() { variable.SysVars[variable.Socket].Value = cfg.Socket variable.SysVars[variable.DataDir].Value = cfg.Path variable.SysVars[variable.TiDBSlowQueryFile].Value = cfg.Log.SlowQueryFile + variable.SysVars[variable.TiDBIsolationReadEngines].Value = strings.Join(cfg.IsolationRead.Engines, ", ") // For CI environment we default enable prepare-plan-cache. plannercore.SetPreparedPlanCache(config.CheckTableBeforeDrop || cfg.PreparedPlanCache.Enabled) diff --git a/tidb-server/main_test.go b/tidb-server/main_test.go index 747b40505cf90..f745c7533904e 100644 --- a/tidb-server/main_test.go +++ b/tidb-server/main_test.go @@ -17,6 +17,8 @@ import ( "testing" . "github.com/pingcap/check" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/sessionctx/variable" ) var isCoverageServer = "0" @@ -32,3 +34,19 @@ func TestRunMain(t *testing.T) { func TestT(t *testing.T) { TestingT(t) } + +var _ = Suite(&testMainSuite{}) + +type testMainSuite struct{} + +func (t *testMainSuite) TestSetGlobalVars(c *C) { + c.Assert(variable.SysVars[variable.TiDBIsolationReadEngines].Value, Equals, "tikv, tiflash, tidb") + c.Assert(variable.SysVars[variable.TIDBMemQuotaQuery].Value, Equals, "1073741824") + + config.GetGlobalConfig().IsolationRead.Engines = []string{"tikv", "tidb"} + config.GetGlobalConfig().MemQuotaQuery = 9999999 + setGlobalVars() + + c.Assert(variable.SysVars[variable.TiDBIsolationReadEngines].Value, Equals, "tikv, tidb") + c.Assert(variable.SysVars[variable.TIDBMemQuotaQuery].Value, Equals, "9999999") +} diff --git a/tools/check/go.mod b/tools/check/go.mod index 91fc1c9d7b14a..4f947512c6455 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -9,11 +9,11 @@ require ( github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf // indirect github.com/gordonklaus/ineffassign v0.0.0-20180909121442-1003c8bd00dc github.com/kisielk/errcheck v1.2.0 + github.com/mdempsky/unconvert v0.0.0-20200228143138-95ecdbfc0b5f // indirect github.com/mgechev/revive v0.0.0-20181210140514-b4cc152955fb github.com/nicksnyder/go-i18n v1.10.0 // indirect github.com/pelletier/go-toml v1.2.0 // indirect github.com/securego/gosec v0.0.0-20181211171558-12400f9a1ca7 - golang.org/x/tools v0.0.0-20190925020647-22afafe3322a // indirect gopkg.in/alecthomas/gometalinter.v2 v2.0.12 // indirect gopkg.in/alecthomas/gometalinter.v3 v3.0.0 // indirect gopkg.in/alecthomas/kingpin.v2 v2.2.6 // indirect diff --git a/tools/check/go.sum b/tools/check/go.sum index 2d4ff41ed2ab0..16fc9a9e462a0 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -30,6 +30,8 @@ github.com/mattn/go-isatty v0.0.4 h1:bnP0vzxcAdeI1zdubAl5PjU6zsERjGZb7raWodagDYs github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-runewidth v0.0.3 h1:a+kO+98RDGEfo6asOGMmpodZq4FNtnGP54yps8BzLR4= github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= +github.com/mdempsky/unconvert v0.0.0-20200228143138-95ecdbfc0b5f h1:Kc3s6QFyh9DLgInXpWKuG+8I7R7lXbnP7mcoOVIt6KY= +github.com/mdempsky/unconvert v0.0.0-20200228143138-95ecdbfc0b5f/go.mod h1:AmCV4WB3cDMZqgPk+OUQKumliiQS4ZYsBt3AXekyuAU= github.com/mgechev/dots v0.0.0-20180605013149-8e09d8ea2757 h1:KTwJ7Lo3KDKMknRYN5JEFRGIM4IkG59QjFFM2mxsMEU= github.com/mgechev/dots v0.0.0-20180605013149-8e09d8ea2757/go.mod h1:KQ7+USdGKfpPjXk4Ga+5XxQM4Lm4e3gAogrreFAYpOg= github.com/mgechev/revive v0.0.0-20181210140514-b4cc152955fb h1:bLiKpCHe+ngBsF1o7DjZTmoffHEy2gdQ/+9NunuJ4ZY= @@ -53,22 +55,32 @@ github.com/securego/gosec v0.0.0-20181211171558-12400f9a1ca7 h1:Ca7U7/rZ+caxjW2n github.com/securego/gosec v0.0.0-20181211171558-12400f9a1ca7/go.mod h1:m3KbCTwh9vLhm6AKBjE+ALesKilKcQHezI1uVOti0Ks= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/net v0.0.0-20170915142106-8351a756f30f/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20171026204733-164713f0dfce/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.0.0-20170915090833-1cbadb444a80/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/tools v0.0.0-20170915040203-e531a2a1c15f/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180911133044-677d2ff680c1/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563 h1:NIou6eNFigscvKJmsbyez16S2cIS6idossORlFtSt2E= golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190925020647-22afafe3322a h1:3GxqzBPBt1O2dIiPnzldQ5d25CAMWJFBZTpqxLPfjs8= golang.org/x/tools v0.0.0-20190925020647-22afafe3322a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20200225230052-807dcd883420 h1:4RJNOV+2rLxMEfr6QIpC7GEv9MjD6ApGXTCLrNF9+eA= +golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gopkg.in/alecthomas/gometalinter.v2 v2.0.12 h1:/xBWwtjmOmVxn8FXfIk9noV8m2E2Id9jFfUY/Mh9QAI= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= gopkg.in/alecthomas/gometalinter.v3 v3.0.0 h1:tKnpLD70cWDacxrv9JZ4atld7RPoPiHOBfad6mPmyBw= diff --git a/types/benchmark_test.go b/types/benchmark_test.go index 44c2a01366018..91d3d5cce9f0e 100644 --- a/types/benchmark_test.go +++ b/types/benchmark_test.go @@ -37,7 +37,7 @@ func BenchmarkDefaultTypeForValue(b *testing.B) { b.StartTimer() var ft FieldType for i := 0; i < b.N; i++ { - DefaultTypeForValue(numsFull[int(i)%lenNums], &ft, mysql.DefaultCharset, mysql.DefaultCollationName) + DefaultTypeForValue(numsFull[i%lenNums], &ft, mysql.DefaultCharset, mysql.DefaultCollationName) } }) @@ -45,7 +45,7 @@ func BenchmarkDefaultTypeForValue(b *testing.B) { b.StartTimer() var ft FieldType for i := 0; i < b.N; i++ { - DefaultTypeForValue(nums64k[int(i)%lenNums], &ft, mysql.DefaultCharset, mysql.DefaultCollationName) + DefaultTypeForValue(nums64k[i%lenNums], &ft, mysql.DefaultCharset, mysql.DefaultCollationName) } }) @@ -53,7 +53,7 @@ func BenchmarkDefaultTypeForValue(b *testing.B) { b.StartTimer() var ft FieldType for i := 0; i < b.N; i++ { - DefaultTypeForValue(nums512[int(i)%lenNums], &ft, mysql.DefaultCharset, mysql.DefaultCollationName) + DefaultTypeForValue(nums512[i%lenNums], &ft, mysql.DefaultCharset, mysql.DefaultCollationName) } }) } diff --git a/types/binary_literal.go b/types/binary_literal.go index 5725b34d36d24..0da3e5c0bb982 100644 --- a/types/binary_literal.go +++ b/types/binary_literal.go @@ -108,7 +108,7 @@ func (b BinaryLiteral) ToInt(sc *stmtctx.StatementContext) (uint64, error) { return 0, nil } if length > 8 { - var err error = ErrTruncatedWrongVal.GenWithStackByArgs("BINARY", b) + var err = ErrTruncatedWrongVal.GenWithStackByArgs("BINARY", b) if sc != nil { err = sc.HandleTruncate(err) } diff --git a/types/compare.go b/types/compare.go index fc519b3d780d5..560631130f9d0 100644 --- a/types/compare.go +++ b/types/compare.go @@ -90,7 +90,7 @@ func VecCompareUI(x []uint64, y, res []int64) { func VecCompareIU(x []int64, y []uint64, res []int64) { n := len(x) for i := 0; i < n; i++ { - if x[i] < 0 || uint64(y[i]) > math.MaxInt64 { + if x[i] < 0 || y[i] > math.MaxInt64 { res[i] = -1 } else if x[i] < int64(y[i]) { res[i] = -1 diff --git a/types/compare_test.go b/types/compare_test.go index a14a58e711abd..4e509ec55c2ee 100644 --- a/types/compare_test.go +++ b/types/compare_test.go @@ -71,15 +71,15 @@ func (s *testCompareSuite) TestCompare(c *C) { {float64(354.23), float64(3.45), 1}, {float64(3.452), float64(3.452), 0}, - {int(432), nil, 1}, - {-4, int(32), -1}, - {int(4), -32, 1}, - {int(432), int64(12), 1}, - {int(23), int64(128), -1}, - {int(123), int64(123), 0}, - {int(432), int(12), 1}, - {int(23), int(123), -1}, - {int64(133), int(183), -1}, + {432, nil, 1}, + {-4, 32, -1}, + {4, -32, 1}, + {432, int64(12), 1}, + {23, int64(128), -1}, + {123, int64(123), 0}, + {432, 12, 1}, + {23, 123, -1}, + {int64(133), 183, -1}, {uint64(133), uint64(183), -1}, {uint64(2), int64(-2), 1}, diff --git a/types/convert.go b/types/convert.go index 3e9f9629b77ec..0d6b7ff2a3e6b 100644 --- a/types/convert.go +++ b/types/convert.go @@ -616,7 +616,7 @@ func ConvertJSONToDecimal(sc *stmtctx.StatementContext, j json.BinaryJSON) (*MyD err = res.FromFloat64(f64) return res, errors.Trace(err) } - err := sc.HandleTruncate(res.FromString([]byte(j.GetString()))) + err := sc.HandleTruncate(res.FromString(j.GetString())) return res, errors.Trace(err) } diff --git a/types/convert_test.go b/types/convert_test.go index 96d74410184ae..22d7478f76f11 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -197,7 +197,7 @@ func (s *testTypeConvertSuite) TestConvertType(c *C) { c.Assert(v, Equals, uint64(100)) // issue 3470 ft = NewFieldType(mysql.TypeLonglong) - v, err = Convert(Duration{Duration: time.Duration(12*time.Hour + 59*time.Minute + 59*time.Second + 555*time.Millisecond), Fsp: 3}, ft) + v, err = Convert(Duration{Duration: 12*time.Hour + 59*time.Minute + 59*time.Second + 555*time.Millisecond, Fsp: 3}, ft) c.Assert(err, IsNil) c.Assert(v, Equals, int64(130000)) v, err = Convert(NewTime(FromDate(2017, 1, 1, 12, 59, 59, 555000), mysql.TypeDatetime, MaxFsp), ft) @@ -317,7 +317,7 @@ func (s *testTypeConvertSuite) TestConvertToString(c *C) { testToString(c, "0", "0") testToString(c, true, "1") testToString(c, "false", "false") - testToString(c, int(0), "0") + testToString(c, 0, "0") testToString(c, int64(0), "0") testToString(c, uint64(0), "0") testToString(c, float32(1.6), "1.6") @@ -663,7 +663,7 @@ func (s *testTypeConvertSuite) TestConvert(c *C) { signedAccept(c, mysql.TypeFloat, "23.523", "23.523") signedAccept(c, mysql.TypeFloat, int64(123), "123") signedAccept(c, mysql.TypeFloat, uint64(123), "123") - signedAccept(c, mysql.TypeFloat, int(123), "123") + signedAccept(c, mysql.TypeFloat, 123, "123") signedAccept(c, mysql.TypeFloat, float32(123), "123") signedAccept(c, mysql.TypeFloat, float64(123), "123") signedAccept(c, mysql.TypeDouble, " -23.54", "-23.54") diff --git a/types/datum.go b/types/datum.go index 132ce47738566..f4bf7f9e316d4 100644 --- a/types/datum.go +++ b/types/datum.go @@ -198,6 +198,7 @@ func (d *Datum) GetBytes() []byte { func (d *Datum) SetBytes(b []byte) { d.k = KindBytes d.b = b + d.collation = charset.CollationBin } // SetBytesAsString sets bytes value to datum as string type. @@ -1433,19 +1434,18 @@ func (d *Datum) ToBool(sc *stmtctx.StatementContext) (int64, error) { case KindUint64: isZero = d.GetUint64() == 0 case KindFloat32: - isZero = RoundFloat(d.GetFloat64()) == 0 + isZero = d.GetFloat64() == 0 case KindFloat64: - isZero = RoundFloat(d.GetFloat64()) == 0 + isZero = d.GetFloat64() == 0 case KindString, KindBytes: - iVal, err1 := StrToInt(sc, d.GetString()) + iVal, err1 := StrToFloat(sc, d.GetString()) isZero, err = iVal == 0, err1 case KindMysqlTime: isZero = d.GetMysqlTime().IsZero() case KindMysqlDuration: isZero = d.GetMysqlDuration().Duration == 0 case KindMysqlDecimal: - v, err1 := d.GetMysqlDecimal().ToFloat64() - isZero, err = RoundFloat(v) == 0, err1 + isZero = d.GetMysqlDecimal().IsZero() case KindMysqlEnum: isZero = d.GetMysqlEnum().ToNumber() == 0 case KindMysqlSet: diff --git a/types/datum_test.go b/types/datum_test.go index 831c8c20bc0f5..c9adb255cf1d6 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -64,17 +64,17 @@ func testDatumToBool(c *C, in interface{}, res int) { } func (ts *testDatumSuite) TestToBool(c *C) { - testDatumToBool(c, int(0), 0) + testDatumToBool(c, 0, 0) testDatumToBool(c, int64(0), 0) testDatumToBool(c, uint64(0), 0) - testDatumToBool(c, float32(0.1), 0) - testDatumToBool(c, float64(0.1), 0) + testDatumToBool(c, float32(0.1), 1) + testDatumToBool(c, float64(0.1), 1) testDatumToBool(c, float64(0.5), 1) - testDatumToBool(c, float64(0.499), 0) + testDatumToBool(c, float64(0.499), 1) testDatumToBool(c, "", 0) - testDatumToBool(c, "0.1", 0) + testDatumToBool(c, "0.1", 1) testDatumToBool(c, []byte{}, 0) - testDatumToBool(c, []byte("0.1"), 0) + testDatumToBool(c, []byte("0.1"), 1) testDatumToBool(c, NewBinaryLiteralFromUint(0, -1), 0) testDatumToBool(c, Enum{Name: "a", Value: 1}, 1) testDatumToBool(c, Set{Name: "a", Value: 1}, 1) @@ -91,7 +91,7 @@ func (ts *testDatumSuite) TestToBool(c *C) { ft.Decimal = 5 v, err := Convert(0.1415926, ft) c.Assert(err, IsNil) - testDatumToBool(c, v, 0) + testDatumToBool(c, v, 1) d := NewDatum(&invalidMockType{}) sc := new(stmtctx.StatementContext) sc.IgnoreTruncate = true @@ -146,7 +146,7 @@ func testDatumToInt64(c *C, val interface{}, expect int64) { func (ts *testTypeConvertSuite) TestToInt64(c *C) { testDatumToInt64(c, "0", int64(0)) - testDatumToInt64(c, int(0), int64(0)) + testDatumToInt64(c, 0, int64(0)) testDatumToInt64(c, int64(0), int64(0)) testDatumToInt64(c, uint64(0), int64(0)) testDatumToInt64(c, float32(3.1), int64(3)) diff --git a/types/enum.go b/types/enum.go index 45c15a3d2208f..1aafdc345bcf4 100644 --- a/types/enum.go +++ b/types/enum.go @@ -18,6 +18,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/tidb/util/stringutil" ) // Enum is for MySQL enum type. @@ -26,6 +27,14 @@ type Enum struct { Value uint64 } +// Copy deep copy an Enum. +func (e Enum) Copy() Enum { + return Enum{ + Name: stringutil.Copy(e.Name), + Value: e.Value, + } +} + // String implements fmt.Stringer interface. func (e Enum) String() string { return e.Name diff --git a/types/field_type_test.go b/types/field_type_test.go index 31c5fd97a2e4d..e3f156e0fd3d2 100644 --- a/types/field_type_test.go +++ b/types/field_type_test.go @@ -348,13 +348,13 @@ func (s *testFieldTypeSuite) TestAggregateEvalType(c *C) { case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeBit, mysql.TypeInt24, mysql.TypeYear: c.Assert(aggregatedEvalType, Equals, ETInt) - c.Assert(flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(flag, Equals, mysql.BinaryFlag) case mysql.TypeFloat, mysql.TypeDouble: c.Assert(aggregatedEvalType, Equals, ETReal) - c.Assert(flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(flag, Equals, mysql.BinaryFlag) case mysql.TypeNewDecimal: c.Assert(aggregatedEvalType, Equals, ETDecimal) - c.Assert(flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(flag, Equals, mysql.BinaryFlag) } flag = 0 @@ -370,13 +370,13 @@ func (s *testFieldTypeSuite) TestAggregateEvalType(c *C) { case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeBit, mysql.TypeInt24, mysql.TypeYear: c.Assert(aggregatedEvalType, Equals, ETInt) - c.Assert(flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(flag, Equals, mysql.BinaryFlag) case mysql.TypeFloat, mysql.TypeDouble: c.Assert(aggregatedEvalType, Equals, ETReal) - c.Assert(flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(flag, Equals, mysql.BinaryFlag) case mysql.TypeNewDecimal: c.Assert(aggregatedEvalType, Equals, ETDecimal) - c.Assert(flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(flag, Equals, mysql.BinaryFlag) } flag = 0 aggregatedEvalType = AggregateEvalType([]*FieldType{fts[i], NewFieldType(mysql.TypeLong)}, &flag) @@ -391,13 +391,13 @@ func (s *testFieldTypeSuite) TestAggregateEvalType(c *C) { case mysql.TypeDecimal, mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeNull, mysql.TypeBit, mysql.TypeLonglong, mysql.TypeYear, mysql.TypeInt24: c.Assert(aggregatedEvalType, Equals, ETInt) - c.Assert(flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(flag, Equals, mysql.BinaryFlag) case mysql.TypeFloat, mysql.TypeDouble: c.Assert(aggregatedEvalType, Equals, ETReal) - c.Assert(flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(flag, Equals, mysql.BinaryFlag) case mysql.TypeNewDecimal: c.Assert(aggregatedEvalType, Equals, ETDecimal) - c.Assert(flag, Equals, uint(mysql.BinaryFlag)) + c.Assert(flag, Equals, mysql.BinaryFlag) } } } diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index b936b0eafd765..735e298e41e5c 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/stringutil" ) // Type returns type of BinaryJSON as string. @@ -778,7 +779,7 @@ func PeekBytesAsJSON(b []byte) (n int, err error) { err = errors.New("Cant peek from empty bytes") return } - switch c := TypeCode(b[0]); c { + switch c := b[0]; c { case TypeCodeObject, TypeCodeArray: if len(b) >= valTypeSize+headerSize { size := endian.Uint32(b[valTypeSize+dataSizeOff:]) @@ -880,6 +881,47 @@ func (bj BinaryJSON) GetElemDepth() int { } } +// Search for JSON_Search +// rules referenced by MySQL JSON_SEARCH function +// [https://dev.mysql.com/doc/refman/5.7/en/json-search-functions.html#function_json-search] +func (bj BinaryJSON) Search(containType string, search string, escape byte, pathExpres []PathExpression) (res BinaryJSON, isNull bool, err error) { + if containType != ContainsPathOne && containType != ContainsPathAll { + return res, true, ErrInvalidJSONPath + } + patChars, patTypes := stringutil.CompilePattern(search, escape) + + result := make([]interface{}, 0) + walkFn := func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) { + if bj.TypeCode == TypeCodeString && stringutil.DoMatch(string(bj.GetString()), patChars, patTypes) { + result = append(result, fullpath.String()) + if containType == ContainsPathOne { + return true, nil + } + } + return false, nil + } + if len(pathExpres) != 0 { + err := bj.Walk(walkFn, pathExpres...) + if err != nil { + return res, true, err + } + } else { + err := bj.Walk(walkFn) + if err != nil { + return res, true, err + } + } + switch len(result) { + case 0: + return res, true, nil + case 1: + return CreateBinary(result[0]), false, nil + default: + return CreateBinary(result), false, nil + } + +} + // extractCallbackFn: the type of CALLBACK function for extractToCallback type extractCallbackFn func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) diff --git a/types/parser_driver/special_cmt_ctrl.go b/types/parser_driver/special_cmt_ctrl.go index 8a43241411c83..ad34ae2307985 100644 --- a/types/parser_driver/special_cmt_ctrl.go +++ b/types/parser_driver/special_cmt_ctrl.go @@ -32,12 +32,19 @@ import ( // and we want to comment out this part of SQL in binlog. func init() { parser.SpecialCommentsController.Register(string(FeatureIDAutoRandom)) + parser.SpecialCommentsController.Register(string(FeatureIDAutoIDCache)) + parser.SpecialCommentsController.Register(string(FeatureIDAutoRandomBase)) } // SpecialCommentVersionPrefix is the prefix of TiDB executable comments. const SpecialCommentVersionPrefix = `/*T!` // BuildSpecialCommentPrefix returns the prefix of `featureID` special comment. +// For some special feature in TiDB, we will refine ddl query with special comment, +// which may be useful when +// A: the downstream is directly MySQL instance (treat it as comment for compatibility). +// B: the downstream is lower version TiDB (ignore the unknown feature comment). +// C: the downstream is same/higher version TiDB (parse the feature syntax out). func BuildSpecialCommentPrefix(featureID featureID) string { return fmt.Sprintf("%s[%s]", SpecialCommentVersionPrefix, featureID) } @@ -47,9 +54,15 @@ type featureID string const ( // FeatureIDAutoRandom is the `auto_random` feature. FeatureIDAutoRandom featureID = "auto_rand" + // FeatureIDAutoIDCache is the `auto_id_cache` feature. + FeatureIDAutoIDCache featureID = "auto_id_cache" + // FeatureIDAutoRandomBase is the `auto_random_base` feature. + FeatureIDAutoRandomBase featureID = "auto_rand_base" ) // FeatureIDPatterns is used to record special comments patterns. var FeatureIDPatterns = map[featureID]*regexp.Regexp{ - FeatureIDAutoRandom: regexp.MustCompile(`(?i)AUTO_RANDOM\s*(\(\s*\d+\s*\))?\s*`), + FeatureIDAutoRandom: regexp.MustCompile(`(?i)AUTO_RANDOM\s*(\(\s*\d+\s*\)|\s+)\s*`), + FeatureIDAutoIDCache: regexp.MustCompile(`(?i)AUTO_ID_CACHE\s*=?\s*\d+\s*`), + FeatureIDAutoRandomBase: regexp.MustCompile(`(?i)AUTO_RANDOM_BASE\s*=?\s*\d+\s*`), } diff --git a/types/parser_driver/value_expr.go b/types/parser_driver/value_expr.go index 6d24ebbf0f081..a2398f42ad6f5 100644 --- a/types/parser_driver/value_expr.go +++ b/types/parser_driver/value_expr.go @@ -173,8 +173,9 @@ func newValueExpr(value interface{}, charset string, collate string) ast.ValueEx return ve } ve := &ValueExpr{} - ve.SetValue(value) + // We need to keep the ve.Type.Collate equals to ve.Datum.collation. types.DefaultTypeForValue(value, &ve.Type, charset, collate) + ve.Datum.SetValue(value, &ve.Type) ve.projectionOffset = -1 return ve } diff --git a/types/set.go b/types/set.go index 0a337780c452f..57145855c9c43 100644 --- a/types/set.go +++ b/types/set.go @@ -18,6 +18,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/tidb/util/stringutil" ) var zeroSet = Set{Name: "", Value: 0} @@ -38,6 +39,14 @@ func (e Set) ToNumber() float64 { return float64(e.Value) } +// Copy deep copy a Set. +func (e Set) Copy() Set { + return Set{ + Name: stringutil.Copy(e.Name), + Value: e.Value, + } +} + // ParseSetName creates a Set with name. func ParseSetName(elems []string, name string) (Set, error) { if len(name) == 0 { diff --git a/types/time.go b/types/time.go index f163c7fa40039..e2d691904272f 100644 --- a/types/time.go +++ b/types/time.go @@ -254,7 +254,7 @@ const ( fspTtForDate uint8 = 0b1110 fspBitFieldMask uint64 = 0b1110 - coreTimeBitFieldMask uint64 = ^fspTtBitFieldMask + coreTimeBitFieldMask = ^fspTtBitFieldMask ) // NewTime constructs time from core time, type and fsp. diff --git a/util/admin/admin.go b/util/admin/admin.go index 025298f143829..8576f03f7e0e4 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -110,7 +110,7 @@ func IsJobRollbackable(job *model.Job) bool { model.ActionTruncateTable, model.ActionAddForeignKey, model.ActionDropForeignKey, model.ActionRenameTable, model.ActionModifyTableCharsetAndCollate, model.ActionTruncateTablePartition, - model.ActionModifySchemaCharsetAndCollate, model.ActionRepairTable: + model.ActionModifySchemaCharsetAndCollate, model.ActionRepairTable, model.ActionModifyTableAutoIdCache: return job.SchemaState == model.StateNone } return true @@ -244,6 +244,41 @@ func GetHistoryDDLJobs(txn kv.Transaction, maxNumJobs int) ([]*model.Job, error) return jobs, nil } +// IterHistoryDDLJobs iterates history DDL jobs until the `finishFn` return true or error. +func IterHistoryDDLJobs(txn kv.Transaction, finishFn func([]*model.Job) (bool, error)) error { + txnMeta := meta.NewMeta(txn) + iter, err := txnMeta.GetLastHistoryDDLJobsIterator() + if err != nil { + return err + } + cacheJobs := make([]*model.Job, 0, DefNumHistoryJobs) + for { + cacheJobs, err = iter.GetLastJobs(DefNumHistoryJobs, cacheJobs) + if err != nil || len(cacheJobs) == 0 { + return err + } + finish, err := finishFn(cacheJobs) + if err != nil || finish { + return err + } + } +} + +// IterAllDDLJobs will iterates running DDL jobs first, return directly if `finishFn` return true or error, +// then iterates history DDL jobs until the `finishFn` return true or error. +func IterAllDDLJobs(txn kv.Transaction, finishFn func([]*model.Job) (bool, error)) error { + jobs, err := GetDDLJobs(txn) + if err != nil { + return err + } + + finish, err := finishFn(jobs) + if err != nil || finish { + return err + } + return IterHistoryDDLJobs(txn, finishFn) +} + // RecordData is the record data composed of a handle and values. type RecordData struct { Handle int64 @@ -311,7 +346,7 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table cols[i] = t.Cols()[col.Offset] } - startKey := t.RecordKey(math.MinInt64) + startKey := t.RecordKey(kv.IntHandle(math.MinInt64)) filterFunc := func(h1 int64, vals1 []types.Datum, cols []*table.Column) (bool, error) { for i, val := range vals1 { col := cols[i] @@ -327,10 +362,10 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table vals1[i] = colDefVal } } - isExist, h2, err := idx.Exist(sc, txn, vals1, h1) + isExist, h2, err := idx.Exist(sc, txn, vals1, kv.IntHandle(h1)) if kv.ErrKeyExists.Equal(err) { record1 := &RecordData{Handle: h1, Values: vals1} - record2 := &RecordData{Handle: h2, Values: vals1} + record2 := &RecordData{Handle: h2.IntValue(), Values: vals1} return false, ErrDataInConsistent.GenWithStack("index:%#v != record:%#v", record2, record1) } if err != nil { @@ -404,15 +439,15 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab for _, col := range cols { if col.IsPKHandleColumn(t.Meta()) { if mysql.HasUnsignedFlag(col.Flag) { - data = append(data, types.NewUintDatum(uint64(handle))) + data = append(data, types.NewUintDatum(uint64(handle.IntValue()))) } else { - data = append(data, types.NewIntDatum(handle)) + data = append(data, types.NewIntDatum(handle.IntValue())) } } else { data = append(data, rowMap[col.ID]) } } - more, err := fn(handle, data, cols) + more, err := fn(handle.IntValue(), data, cols) if !more || err != nil { return errors.Trace(err) } diff --git a/util/admin/admin_integration_test.go b/util/admin/admin_integration_test.go index c6a226e01f4da..003f6ce984d28 100644 --- a/util/admin/admin_integration_test.go +++ b/util/admin/admin_integration_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/cluster" "github.com/pingcap/tidb/store/mockstore/mocktikv" "github.com/pingcap/tidb/util/testkit" ) @@ -26,19 +27,21 @@ import ( var _ = Suite(&testAdminSuite{}) type testAdminSuite struct { - cluster *mocktikv.Cluster - mvccStore mocktikv.MVCCStore - store kv.Storage - domain *domain.Domain + cluster cluster.Cluster + store kv.Storage + domain *domain.Domain } func (s *testAdminSuite) SetUpSuite(c *C) { - s.cluster = mocktikv.NewCluster() - mocktikv.BootstrapWithSingleStore(s.cluster) - s.mvccStore = mocktikv.MustNewMVCCStore() + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + s.cluster = cluster + + mvccStore := mocktikv.MustNewMVCCStore() + cluster.SetMvccStore(mvccStore) store, err := mockstore.NewMockTikvStore( - mockstore.WithCluster(s.cluster), - mockstore.WithMVCCStore(s.mvccStore), + mockstore.WithCluster(cluster), + mockstore.WithMVCCStore(mvccStore), ) c.Assert(err, IsNil) s.store = store diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index 2a78e0a013715..9a55a2d4f90db 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -102,6 +102,7 @@ func (s *testSuite) TestGetDDLJobs(c *C) { t := meta.NewMeta(txn) cnt := 10 jobs := make([]*model.Job, cnt) + var currJobs2 []*model.Job for i := 0; i < cnt; i++ { jobs[i] = &model.Job{ ID: int64(i), @@ -113,6 +114,19 @@ func (s *testSuite) TestGetDDLJobs(c *C) { currJobs, err1 := GetDDLJobs(txn) c.Assert(err1, IsNil) c.Assert(currJobs, HasLen, i+1) + currJobs2 = currJobs2[:0] + err = IterAllDDLJobs(txn, func(jobs []*model.Job) (b bool, e error) { + for _, job := range jobs { + if job.State == model.JobStateNone { + currJobs2 = append(currJobs2, job) + } else { + return true, nil + } + } + return false, nil + }) + c.Assert(err, IsNil) + c.Assert(currJobs2, HasLen, i+1) } currJobs, err := GetDDLJobs(txn) @@ -122,6 +136,7 @@ func (s *testSuite) TestGetDDLJobs(c *C) { c.Assert(job.SchemaID, Equals, int64(1)) c.Assert(job.Type, Equals, model.ActionCreateTable) } + c.Assert(currJobs, DeepEquals, currJobs2) err = txn.Rollback() c.Assert(err, IsNil) @@ -322,6 +337,19 @@ func (s *testSuite) TestGetHistoryDDLJobs(c *C) { c.Assert(job.Type, Equals, model.ActionCreateTable) } + var historyJobs2 []*model.Job + err = IterHistoryDDLJobs(txn, func(jobs []*model.Job) (b bool, e error) { + for _, job := range jobs { + historyJobs2 = append(historyJobs2, job) + if len(historyJobs2) == DefNumHistoryJobs { + return true, nil + } + } + return false, nil + }) + c.Assert(err, IsNil) + c.Assert(historyJobs2, DeepEquals, historyJobs) + err = txn.Rollback() c.Assert(err, IsNil) } diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index 3fde95035051e..b6244029cbd5a 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -350,7 +350,7 @@ func (c *Chunk) GetRow(idx int) Row { // logical 2 -> physical 6. // Then when we iterate this Chunk according to Row, only selected rows will be // accessed while all filtered rows will be ignored. - return Row{c: c, idx: int(c.sel[idx])} + return Row{c: c, idx: c.sel[idx]} } return Row{c: c, idx: idx} } diff --git a/util/chunk/chunk_util.go b/util/chunk/chunk_util.go index 2a72d6770eaeb..b0d20b2c2af67 100644 --- a/util/chunk/chunk_util.go +++ b/util/chunk/chunk_util.go @@ -158,7 +158,7 @@ func copySameOuterRows(outerColOffset, outerColLen int, src *Chunk, numRows int, offsets := dstCol.offsets elemLen := srcCol.offsets[row.idx+1] - srcCol.offsets[row.idx] for j := 0; j < numRows; j++ { - offsets = append(offsets, int64(offsets[len(offsets)-1]+elemLen)) + offsets = append(offsets, offsets[len(offsets)-1]+elemLen) } dstCol.offsets = offsets } diff --git a/util/chunk/column.go b/util/chunk/column.go index 4089389872318..f55ee44116315 100644 --- a/util/chunk/column.go +++ b/util/chunk/column.go @@ -372,7 +372,7 @@ func (c *Column) nullCount() int { var cnt, i int for ; i+8 <= c.length; i += 8 { // 0 is null and 1 is not null - cnt += 8 - bits.OnesCount8(uint8(c.nullBitmap[i>>3])) + cnt += 8 - bits.OnesCount8(c.nullBitmap[i>>3]) } for ; i < c.length; i++ { if c.IsNull(i) { diff --git a/util/chunk/column_test.go b/util/chunk/column_test.go index def82a7416fcb..922b115054e13 100644 --- a/util/chunk/column_test.go +++ b/util/chunk/column_test.go @@ -775,7 +775,7 @@ func (s *testChunkSuite) TestGetRaw(c *check.C) { i = 0 for row := it.Begin(); row != it.End(); row = it.Next() { c.Assert(row.GetRaw(0), check.DeepEquals, []byte(fmt.Sprint(i))) - c.Assert(col.GetRaw(int(i)), check.DeepEquals, []byte(fmt.Sprint(i))) + c.Assert(col.GetRaw(i), check.DeepEquals, []byte(fmt.Sprint(i))) i++ } } diff --git a/util/chunk/compare.go b/util/chunk/compare.go index 27081dce90458..2c93082ab7351 100644 --- a/util/chunk/compare.go +++ b/util/chunk/compare.go @@ -14,6 +14,7 @@ package chunk import ( + "bytes" "sort" "github.com/pingcap/parser/mysql" @@ -191,8 +192,10 @@ func Compare(row Row, colIdx int, ad *types.Datum) int { return types.CompareFloat64(float64(row.GetFloat32(colIdx)), float64(ad.GetFloat32())) case types.KindFloat64: return types.CompareFloat64(row.GetFloat64(colIdx), ad.GetFloat64()) - case types.KindString, types.KindBytes, types.KindBinaryLiteral, types.KindMysqlBit: + case types.KindString: return types.CompareString(row.GetString(colIdx), ad.GetString(), ad.Collation()) + case types.KindBytes, types.KindBinaryLiteral, types.KindMysqlBit: + return bytes.Compare(row.GetBytes(colIdx), ad.GetBytes()) case types.KindMysqlDecimal: l, r := row.GetMyDecimal(colIdx), ad.GetMysqlDecimal() return l.Compare(r) diff --git a/util/chunk/list.go b/util/chunk/list.go index a4ae550d34e63..006b4071ce41e 100644 --- a/util/chunk/list.go +++ b/util/chunk/list.go @@ -184,7 +184,7 @@ func (l *List) preAlloc4Row(row Row) (ptr RowPtr) { chk := l.chunks[chkIdx] rowIdx := chk.preAlloc(row) l.length++ - return RowPtr{ChkIdx: uint32(chkIdx), RowIdx: uint32(rowIdx)} + return RowPtr{ChkIdx: uint32(chkIdx), RowIdx: rowIdx} } // Insert inserts `row` on the position specified by `ptr`. diff --git a/util/chunk/mutrow.go b/util/chunk/mutrow.go index da081deeae6fd..f0f2138cbd6e8 100644 --- a/util/chunk/mutrow.go +++ b/util/chunk/mutrow.go @@ -38,6 +38,18 @@ func (mr MutRow) Len() int { return len(mr.c.columns) } +// Clone deep clone a MutRow. +func (mr MutRow) Clone() MutRow { + newChk := mr.c + if mr.c != nil { + newChk = mr.c.CopyConstruct() + } + return MutRow{ + c: newChk, + idx: mr.idx, + } +} + // MutRowFromValues creates a MutRow from a interface slice. func MutRowFromValues(vals ...interface{}) MutRow { c := &Chunk{columns: make([]*Column, 0, len(vals))} diff --git a/util/collate/collate.go b/util/collate/collate.go index 1ff42a5321ad2..8a592809e5888 100644 --- a/util/collate/collate.go +++ b/util/collate/collate.go @@ -35,6 +35,8 @@ var ( // ErrUnsupportedCollation is returned when an unsupported collation is specified. ErrUnsupportedCollation = terror.ClassDDL.New(mysql.ErrUnknownCollation, "Unsupported collation when new collation is enabled: '%-.64s'") + // ErrIllegalMixCollation is returned when illegal mix of collations. + ErrIllegalMixCollation = terror.ClassExpression.New(mysql.ErrCantAggregate2collations, mysql.MySQLErrName[mysql.ErrCantAggregate2collations]) ) // DefaultLen is set for datum if the string datum don't know its length. diff --git a/util/disk/tracker.go b/util/disk/tracker.go index 1a94494597090..bcdd13720a8fe 100644 --- a/util/disk/tracker.go +++ b/util/disk/tracker.go @@ -24,3 +24,6 @@ type Tracker = memory.Tracker // 1. "label" is the label used in the usage string. // 2. "bytesLimit <= 0" means no limit. var NewTracker = memory.NewTracker + +// NewGlobalTrcaker creates a global disk tracker. +var NewGlobalTrcaker = memory.NewGlobalTracker diff --git a/util/encrypt/aes.go b/util/encrypt/aes.go index ddada4b9eba78..f158c60f2ed66 100644 --- a/util/encrypt/aes.go +++ b/util/encrypt/aes.go @@ -214,7 +214,7 @@ func AESDecryptWithCFB(cryptStr, key []byte, iv []byte) ([]byte, error) { if err != nil { return nil, errors.Trace(err) } - cfb := cipher.NewCFBDecrypter(cb, []byte(iv)) + cfb := cipher.NewCFBDecrypter(cb, iv) dst := make([]byte, len(cryptStr)) cfb.XORKeyStream(dst, cryptStr) return dst, nil diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index c1f4ad98fc205..f126b9821a858 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -34,6 +34,7 @@ var CommitDetailCtxKey = commitDetailCtxKeyType{} // ExecDetails contains execution detail information. type ExecDetails struct { CalleeAddress string + CopTime time.Duration ProcessTime time.Duration WaitTime time.Duration BackoffTime time.Duration @@ -66,6 +67,8 @@ type CommitDetails struct { } const ( + // CopTimeStr represents the sum of cop-task time spend in TiDB distSQL. + CopTimeStr = "Cop_time" // ProcessTimeStr represents the sum of process time of all the coprocessor tasks. ProcessTimeStr = "Process_time" // WaitTimeStr means the time of all coprocessor wait. @@ -108,7 +111,10 @@ const ( // String implements the fmt.Stringer interface. func (d ExecDetails) String() string { - parts := make([]string, 0, 6) + parts := make([]string, 0, 8) + if d.CopTime > 0 { + parts = append(parts, CopTimeStr+": "+strconv.FormatFloat(d.CopTime.Seconds(), 'f', -1, 64)) + } if d.ProcessTime > 0 { parts = append(parts, ProcessTimeStr+": "+strconv.FormatFloat(d.ProcessTime.Seconds(), 'f', -1, 64)) } @@ -180,11 +186,14 @@ func (d ExecDetails) String() string { // ToZapFields wraps the ExecDetails as zap.Fields. func (d ExecDetails) ToZapFields() (fields []zap.Field) { fields = make([]zap.Field, 0, 16) + if d.CopTime > 0 { + fields = append(fields, zap.String(strings.ToLower(CopTimeStr), strconv.FormatFloat(d.CopTime.Seconds(), 'f', -1, 64)+"s")) + } if d.ProcessTime > 0 { fields = append(fields, zap.String(strings.ToLower(ProcessTimeStr), strconv.FormatFloat(d.ProcessTime.Seconds(), 'f', -1, 64)+"s")) } if d.WaitTime > 0 { - fields = append(fields, zap.String(strings.ToLower(WaitTimeStr), strconv.FormatFloat(d.ProcessTime.Seconds(), 'f', -1, 64)+"s")) + fields = append(fields, zap.String(strings.ToLower(WaitTimeStr), strconv.FormatFloat(d.WaitTime.Seconds(), 'f', -1, 64)+"s")) } if d.BackoffTime > 0 { fields = append(fields, zap.String(strings.ToLower(BackoffTimeStr), strconv.FormatFloat(d.BackoffTime.Seconds(), 'f', -1, 64)+"s")) diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index eff1b61784b7e..4805ee672c0df 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -30,6 +30,7 @@ func TestT(t *testing.T) { func TestString(t *testing.T) { detail := &ExecDetails{ + CopTime: time.Second + 3*time.Millisecond, ProcessTime: 2*time.Second + 5*time.Millisecond, WaitTime: time.Second, BackoffTime: time.Second, @@ -60,7 +61,7 @@ func TestString(t *testing.T) { TxnRetry: 1, }, } - expected := "Process_time: 2.005 Wait_time: 1 Backoff_time: 1 Request_count: 1 Total_keys: 100 Process_keys: 10 Prewrite_time: 1 Commit_time: 1 " + + expected := "Cop_time: 1.003 Process_time: 2.005 Wait_time: 1 Backoff_time: 1 Request_count: 1 Total_keys: 100 Process_keys: 10 Prewrite_time: 1 Commit_time: 1 " + "Get_commit_ts_time: 1 Commit_backoff_time: 1 Backoff_types: [backoff1 backoff2] Resolve_lock_time: 1 Local_latch_wait_time: 1 Write_keys: 1 Write_size: 1 Prewrite_region: 1 Txn_retry: 1" if str := detail.String(); str != expected { t.Errorf("got:\n%s\nexpected:\n%s", str, expected) diff --git a/util/hint/hint_processor.go b/util/hint/hint_processor.go index e2724b7e7e7cf..b93f655329f38 100644 --- a/util/hint/hint_processor.go +++ b/util/hint/hint_processor.go @@ -185,14 +185,17 @@ func BindHint(stmt ast.StmtNode, hintsSet *HintsSet) ast.StmtNode { } // ParseHintsSet parses a SQL string, then collects and normalizes the HintsSet. -func ParseHintsSet(p *parser.Parser, sql, charset, collation, db string) (*HintsSet, error) { - stmtNode, err := p.ParseOneStmt(sql, charset, collation) +func ParseHintsSet(p *parser.Parser, sql, charset, collation, db string) (*HintsSet, []error, error) { + stmtNodes, warns, err := p.Parse(sql, charset, collation) if err != nil { - return nil, err + return nil, nil, err } - hs := CollectHint(stmtNode) + if len(stmtNodes) != 1 { + return nil, nil, errors.New(fmt.Sprintf("bind_sql must be a single statement: %s", sql)) + } + hs := CollectHint(stmtNodes[0]) processor := &BlockHintProcessor{} - stmtNode.Accept(processor) + stmtNodes[0].Accept(processor) for i, tblHints := range hs.tableHints { newHints := make([]*ast.TableOptimizerHint, 0, len(tblHints)) for _, tblHint := range tblHints { @@ -202,7 +205,7 @@ func ParseHintsSet(p *parser.Parser, sql, charset, collation, db string) (*Hints offset := processor.GetHintOffset(tblHint.QBName, TypeSelect, i+1) if offset < 0 || !processor.checkTableQBName(tblHint.Tables, TypeSelect) { hintStr := RestoreTableOptimizerHint(tblHint) - return nil, errors.New(fmt.Sprintf("Unknown query block name in hint %s", hintStr)) + return nil, nil, errors.New(fmt.Sprintf("Unknown query block name in hint %s", hintStr)) } tblHint.QBName = GenerateQBName(TypeSelect, offset) for i, tbl := range tblHint.Tables { @@ -214,7 +217,22 @@ func ParseHintsSet(p *parser.Parser, sql, charset, collation, db string) (*Hints } hs.tableHints[i] = newHints } - return hs, nil + return hs, extractHintWarns(warns), nil +} + +func extractHintWarns(warns []error) []error { + for _, w := range warns { + if parser.ErrWarnOptimizerHintUnsupportedHint.Equal(w) || + parser.ErrWarnOptimizerHintInvalidToken.Equal(w) || + parser.ErrWarnMemoryQuotaOverflow.Equal(w) || + parser.ErrWarnOptimizerHintParseError.Equal(w) || + parser.ErrWarnOptimizerHintInvalidInteger.Equal(w) { + // Just one warning is enough, however we use a slice here to stop golint complaining + // "error should be the last type when returning multiple items" for `ParseHintsSet`. + return []error{w} + } + } + return nil } // BlockHintProcessor processes hints at different level of sql statement. diff --git a/util/logutil/log.go b/util/logutil/log.go index c310a306dc89b..49429c70a965f 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -105,7 +105,7 @@ type contextHook struct{} // https://github.com/sirupsen/logrus/issues/63 func (hook *contextHook) Fire(entry *log.Entry) error { pc := make([]uintptr, 4) - cnt := runtime.Callers(6, pc) + cnt := runtime.Callers(8, pc) for i := 0; i < cnt; i++ { fu := runtime.FuncForPC(pc[i] - 1) @@ -235,9 +235,9 @@ func initFileLog(cfg *zaplog.FileLogConfig, logger *log.Logger) error { // use lumberjack to logrotate output := &lumberjack.Logger{ Filename: cfg.Filename, - MaxSize: int(cfg.MaxSize), - MaxBackups: int(cfg.MaxBackups), - MaxAge: int(cfg.MaxDays), + MaxSize: cfg.MaxSize, + MaxBackups: cfg.MaxBackups, + MaxAge: cfg.MaxDays, LocalTime: true, } diff --git a/util/memory/tracker.go b/util/memory/tracker.go index 509941dd08eb1..181029a36ef11 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -40,7 +40,9 @@ import ( type Tracker struct { mu struct { sync.Mutex - children []*Tracker // The children memory trackers + // The children memory trackers. If the Tracker is the Global Tracker, like executor.GlobalDiskUsageTracker, + // we wouldn't maintain its children in order to avoiding mutex contention. + children []*Tracker } actionMu struct { sync.Mutex @@ -52,17 +54,31 @@ type Tracker struct { bytesLimit int64 // bytesLimit <= 0 means no limit. maxConsumed int64 // max number of bytes consumed during execution. parent *Tracker // The parent memory tracker. + isGlobal bool // isGlobal indicates whether this tracker is global tracker } // NewTracker creates a memory tracker. // 1. "label" is the label used in the usage string. // 2. "bytesLimit <= 0" means no limit. +// For the common tracker, isGlobal is default as false func NewTracker(label fmt.Stringer, bytesLimit int64) *Tracker { t := &Tracker{ label: label, bytesLimit: bytesLimit, } t.actionMu.actionOnExceed = &LogOnExceed{} + t.isGlobal = false + return t +} + +// NewGlobalTracker creates a global tracker, its isGlobal is default as true +func NewGlobalTracker(label fmt.Stringer, bytesLimit int64) *Tracker { + t := &Tracker{ + label: label, + bytesLimit: bytesLimit, + } + t.actionMu.actionOnExceed = &LogOnExceed{} + t.isGlobal = true return t } @@ -275,3 +291,38 @@ func (t *Tracker) BytesToString(numBytes int64) string { return fmt.Sprintf("%v Bytes", numBytes) } + +// AttachToGlobalTracker attach the tracker to the global tracker +// AttachToGlobalTracker should be called at the initialization for the session executor's tracker +func (t *Tracker) AttachToGlobalTracker(globalTracker *Tracker) { + if globalTracker == nil { + return + } + if !globalTracker.isGlobal { + panic("Attach to a non-GlobalTracker") + } + if t.parent != nil { + if t.parent.isGlobal { + t.parent.Consume(-t.BytesConsumed()) + } else { + t.parent.remove(t) + } + } + t.parent = globalTracker + t.parent.Consume(t.BytesConsumed()) +} + +// DetachFromGlobalTracker detach itself from its parent +// Note that only the parent of this tracker is Global Tracker could call this function +// Otherwise it should use Detach +func (t *Tracker) DetachFromGlobalTracker() { + if t.parent == nil { + return + } + if !t.parent.isGlobal { + panic("Detach from a non-GlobalTracker") + } + parent := t.parent + parent.Consume(-t.BytesConsumed()) + t.parent = nil +} diff --git a/util/memory/tracker_test.go b/util/memory/tracker_test.go index 05a4e3189eeb8..672a475afc7c6 100644 --- a/util/memory/tracker_test.go +++ b/util/memory/tracker_test.go @@ -279,6 +279,55 @@ func (s *testSuite) TestMaxConsumed(c *C) { } } +func (s *testSuite) TestGlobalTracker(c *C) { + r := NewGlobalTracker(stringutil.StringerStr("root"), -1) + c1 := NewTracker(stringutil.StringerStr("child 1"), -1) + c2 := NewTracker(stringutil.StringerStr("child 2"), -1) + c1.Consume(100) + c2.Consume(200) + + c1.AttachToGlobalTracker(r) + c2.AttachToGlobalTracker(r) + c.Assert(r.BytesConsumed(), Equals, int64(300)) + c.Assert(c1.parent, DeepEquals, r) + c.Assert(c2.parent, DeepEquals, r) + c.Assert(len(r.mu.children), Equals, 0) + + c1.DetachFromGlobalTracker() + c2.DetachFromGlobalTracker() + c.Assert(r.BytesConsumed(), Equals, int64(0)) + c.Assert(c1.parent, IsNil) + c.Assert(c2.parent, IsNil) + c.Assert(len(r.mu.children), Equals, 0) + + defer func() { + v := recover() + c.Assert(v, Equals, "Attach to a non-GlobalTracker") + }() + commonTracker := NewTracker(stringutil.StringerStr("common"), -1) + c1.AttachToGlobalTracker(commonTracker) + + c1.AttachTo(commonTracker) + c.Assert(commonTracker.BytesConsumed(), Equals, int64(100)) + c.Assert(len(commonTracker.mu.children), Equals, 1) + c.Assert(c1.parent, DeepEquals, commonTracker) + + c1.AttachToGlobalTracker(r) + c.Assert(commonTracker.BytesConsumed(), Equals, int64(0)) + c.Assert(len(commonTracker.mu.children), Equals, 0) + c.Assert(r.BytesConsumed(), Equals, int64(100)) + c.Assert(c1.parent, DeepEquals, r) + c.Assert(len(r.mu.children), Equals, 0) + + defer func() { + v := recover() + c.Assert(v, Equals, "Detach from a non-GlobalTracker") + }() + c2.AttachTo(commonTracker) + c2.DetachFromGlobalTracker() + +} + func BenchmarkConsume(b *testing.B) { tracker := NewTracker(stringutil.StringerStr("root"), -1) b.RunParallel(func(pb *testing.PB) { diff --git a/util/mock/context.go b/util/mock/context.go index 089edaafb171c..aa16b35d8b9ca 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -222,7 +222,7 @@ func (c *Context) StmtGetMutation(tableID int64) *binlog.TableMutation { } // StmtAddDirtyTableOP implements the sessionctx.Context interface. -func (c *Context) StmtAddDirtyTableOP(op int, tid int64, handle int64) { +func (c *Context) StmtAddDirtyTableOP(op int, tid int64, handle kv.Handle) { } // AddTableLock implements the sessionctx.Context interface. diff --git a/util/ranger/checker.go b/util/ranger/checker.go index c244d5d80c2e1..76963e042c8cd 100644 --- a/util/ranger/checker.go +++ b/util/ranger/checker.go @@ -32,6 +32,10 @@ func (c *conditionChecker) check(condition expression.Expression) bool { case *expression.ScalarFunction: return c.checkScalarFunction(x) case *expression.Column: + s, _ := condition.(*expression.Column) + if s.RetType.EvalType() == types.ETString { + return false + } return c.checkColumn(x) case *expression.Constant: return true @@ -63,7 +67,14 @@ func (c *conditionChecker) checkScalarFunction(scalar *expression.ScalarFunction return scalar.FuncName.L != ast.NE || c.length == types.UnspecifiedLength } } - case ast.IsNull, ast.IsTruth, ast.IsFalsity: + case ast.IsNull: + return c.checkColumn(scalar.GetArgs()[0]) + case ast.IsTruth, ast.IsFalsity: + if s, ok := scalar.GetArgs()[0].(*expression.Column); ok { + if s.RetType.EvalType() == types.ETString { + return false + } + } return c.checkColumn(scalar.GetArgs()[0]) case ast.UnaryNot: // TODO: support "not like" convert to access conditions. @@ -98,6 +109,10 @@ func (c *conditionChecker) checkScalarFunction(scalar *expression.ScalarFunction } func (c *conditionChecker) checkLikeFunc(scalar *expression.ScalarFunction) bool { + _, collation, _ := scalar.CharsetAndCollation(scalar.GetCtx()) + if !collate.CompatibleCollate(scalar.GetArgs()[0].GetType().Collate, collation) { + return false + } if !c.checkColumn(scalar.GetArgs()[0]) { return false } diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index fbe9963abfe87..19fcdc182defe 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -37,14 +37,14 @@ func validInterval(sc *stmtctx.StatementContext, low, high point) (bool, error) return false, errors.Trace(err) } if low.excl { - l = []byte(kv.Key(l).PrefixNext()) + l = kv.Key(l).PrefixNext() } r, err := codec.EncodeKey(sc, nil, high.value) if err != nil { return false, errors.Trace(err) } if !high.excl { - r = []byte(kv.Key(r).PrefixNext()) + r = kv.Key(r).PrefixNext() } return bytes.Compare(l, r) < 0, nil } diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index d9e620f3aea93..d29853c5b83f9 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -619,6 +619,13 @@ create table t( filterConds: "[eq(test.t.f, B)]", resultStr: "[[\"a\",\"a\"]]", }, + { + indexPos: 4, + exprStr: "f like '@%' collate utf8mb4_bin", + accessConds: "[]", + filterConds: "[like(test.t.f, @%, 92)]", + resultStr: "[[NULL,+inf]]", + }, } collate.SetNewCollationEnabledForTest(true) @@ -1161,3 +1168,34 @@ func (s *testRangerSuite) TestCompIndexInExprCorrCol(c *C) { testKit.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) } } + +func (s *testRangerSuite) TestIndexStringIsTrueRange(c *C) { + defer testleak.AfterTest(c)() + dom, store, err := newDomainStoreWithBootstrap(c) + defer func() { + dom.Close() + store.Close() + }() + c.Assert(err, IsNil) + testKit := testkit.NewTestKit(c, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t0") + testKit.MustExec("CREATE TABLE t0(c0 TEXT(10));") + testKit.MustExec("INSERT INTO t0(c0) VALUES (1);") + testKit.MustExec("CREATE INDEX i0 ON t0(c0(10));") + testKit.MustExec("analyze table t0;") + + var input []string + var output []struct { + SQL string + Result []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Result = s.testData.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + }) + testKit.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) + } +} diff --git a/util/ranger/testdata/ranger_suite_in.json b/util/ranger/testdata/ranger_suite_in.json index a83b5d41d7f0b..5f6acf9731c5d 100644 --- a/util/ranger/testdata/ranger_suite_in.json +++ b/util/ranger/testdata/ranger_suite_in.json @@ -5,5 +5,17 @@ "explain select t.e in (select count(*) from t s use index(idx), t t1 where s.b = 1 and s.c in (1, 2) and s.d = t.a and s.a = t1.a) from t", "select t.e in (select count(*) from t s use index(idx), t t1 where s.b = 1 and s.c in (1, 2) and s.d = t.a and s.a = t1.a) from t" ] + }, + { + "name": "TestIndexStringIsTrueRange", + "cases": [ + "explain select * from t0 where c0", + "explain select * from t0 where c0 and c0 > '123'", + "explain select * from t0 where c0 and c0 <> '123'", + "explain select * from t0 where c0 is true", + "explain select * from t0 where c0 is false", + "explain select * from t0 where c0 and c0 in ('123','456','789')", + "explain SELECT * FROM t0 WHERE ('a' != t0.c0) AND t0.c0;" + ] } ] diff --git a/util/ranger/testdata/ranger_suite_out.json b/util/ranger/testdata/ranger_suite_out.json index 85e77fdf38c1a..c45605d68ce4e 100644 --- a/util/ranger/testdata/ranger_suite_out.json +++ b/util/ranger/testdata/ranger_suite_out.json @@ -25,5 +25,66 @@ ] } ] + }, + { + "Name": "TestIndexStringIsTrueRange", + "Cases": [ + { + "SQL": "explain select * from t0 where c0", + "Result": [ + "TableReader_7 0.80 root data:Selection_6", + "└─Selection_6 0.80 cop[tikv] test.t0.c0", + " └─TableFullScan_5 1.00 cop[tikv] table:t0 keep order:false" + ] + }, + { + "SQL": "explain select * from t0 where c0 and c0 > '123'", + "Result": [ + "IndexReader_7 1.00 root index:Selection_6", + "└─Selection_6 1.00 cop[tikv] test.t0.c0", + " └─IndexRangeScan_5 1.00 cop[tikv] table:t0, index:i0(c0) range:(\"123\",+inf], keep order:false" + ] + }, + { + "SQL": "explain select * from t0 where c0 and c0 <> '123'", + "Result": [ + "IndexReader_7 1.00 root index:Selection_6", + "└─Selection_6 1.00 cop[tikv] test.t0.c0", + " └─IndexRangeScan_5 1.00 cop[tikv] table:t0, index:i0(c0) range:[-inf,\"123\"), (\"123\",+inf], keep order:false" + ] + }, + { + "SQL": "explain select * from t0 where c0 is true", + "Result": [ + "TableReader_7 0.80 root data:Selection_6", + "└─Selection_6 0.80 cop[tikv] istrue(cast(test.t0.c0))", + " └─TableFullScan_5 1.00 cop[tikv] table:t0 keep order:false" + ] + }, + { + "SQL": "explain select * from t0 where c0 is false", + "Result": [ + "TableReader_7 0.80 root data:Selection_6", + "└─Selection_6 0.80 cop[tikv] isfalse(cast(test.t0.c0))", + " └─TableFullScan_5 1.00 cop[tikv] table:t0 keep order:false" + ] + }, + { + "SQL": "explain select * from t0 where c0 and c0 in ('123','456','789')", + "Result": [ + "IndexReader_7 1.00 root index:Selection_6", + "└─Selection_6 1.00 cop[tikv] test.t0.c0", + " └─IndexRangeScan_5 1.00 cop[tikv] table:t0, index:i0(c0) range:[\"123\",\"123\"], [\"456\",\"456\"], [\"789\",\"789\"], keep order:false" + ] + }, + { + "SQL": "explain SELECT * FROM t0 WHERE ('a' != t0.c0) AND t0.c0;", + "Result": [ + "IndexReader_7 1.00 root index:Selection_6", + "└─Selection_6 1.00 cop[tikv] test.t0.c0", + " └─IndexRangeScan_5 1.00 cop[tikv] table:t0, index:i0(c0) range:[-inf,\"a\"), (\"a\",+inf], keep order:false" + ] + } + ] } ] diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index 2c058cac68525..aa32fd395e0ad 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -77,7 +78,7 @@ func NewRowDecoder(tbl table.Table, decodeColMap map[int64]Column) *RowDecoder { } // DecodeAndEvalRowWithMap decodes a byte slice into datums and evaluates the generated column value. -func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle int64, b []byte, decodeLoc, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { +func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv.Handle, b []byte, decodeLoc, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { var err error if rowcodec.IsNewFormat(b) { row, err = tablecodec.DecodeRowWithMapNew(b, rd.colTypes, decodeLoc, row) @@ -98,19 +99,13 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle int rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) continue } - + if rd.tryDecodeFromHandle(dCol, handle) { + continue + } // Get the default value of the column in the generated column expression. - if dCol.Col.IsPKHandleColumn(rd.tbl.Meta()) { - if mysql.HasUnsignedFlag(colInfo.Flag) { - val.SetUint64(uint64(handle)) - } else { - val.SetInt64(handle) - } - } else { - val, err = tables.GetColDefaultValue(ctx, dCol.Col, rd.defaultVals) - if err != nil { - return nil, err - } + val, err = tables.GetColDefaultValue(ctx, dCol.Col, rd.defaultVals) + if err != nil { + return nil, err } rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) } @@ -143,6 +138,22 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle int return row, nil } +func (rd *RowDecoder) tryDecodeFromHandle(dCol Column, handle kv.Handle) bool { + if handle == nil { + return false + } + colInfo := dCol.Col.ColumnInfo + if dCol.Col.IsPKHandleColumn(rd.tbl.Meta()) { + if mysql.HasUnsignedFlag(colInfo.Flag) { + rd.mutRow.SetValue(colInfo.Offset, uint64(handle.IntValue())) + } else { + rd.mutRow.SetValue(colInfo.Offset, handle.IntValue()) + } + return true + } + return false +} + // BuildFullDecodeColMap build a map that contains [columnID -> struct{*table.Column, expression.Expression}] from // indexed columns and all of its depending columns. `genExprProducer` is used to produce a generated expression based on a table.Column. func BuildFullDecodeColMap(indexedCols []*table.Column, t table.Table, genExprProducer func(*table.Column) (expression.Expression, error)) (map[int64]Column, error) { diff --git a/util/rowDecoder/decoder_test.go b/util/rowDecoder/decoder_test.go index caee27eefb86d..bd59619d2123d 100644 --- a/util/rowDecoder/decoder_test.go +++ b/util/rowDecoder/decoder_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -127,7 +128,7 @@ func (s *testDecoderSuite) TestRowDecoder(c *C) { c.Assert(err, IsNil) c.Assert(bs, NotNil) - r, err := de.DecodeAndEvalRowWithMap(ctx, int64(i), bs, time.UTC, timeZoneIn8, nil) + r, err := de.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil) c.Assert(err, IsNil) // Last column is primary-key column, and the table primary-key is handle, then the primary-key value won't be // stored in raw data, but store in the raw key. @@ -145,7 +146,7 @@ func (s *testDecoderSuite) TestRowDecoder(c *C) { } } // test decode with no generated column. - r2, err := deWithNoGenCols.DecodeAndEvalRowWithMap(ctx, int64(i), bs, time.UTC, timeZoneIn8, nil) + r2, err := deWithNoGenCols.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil) c.Assert(err, IsNil) for k, v := range r2 { v1, ok := r[k] diff --git a/util/rowcodec/bench_test.go b/util/rowcodec/bench_test.go index 94c0ad496ac3a..349d1b7ac4895 100644 --- a/util/rowcodec/bench_test.go +++ b/util/rowcodec/bench_test.go @@ -18,6 +18,7 @@ import ( "time" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -83,11 +84,11 @@ func BenchmarkDecode(b *testing.B) { Collate: tp.Collate, } } - decoder := rowcodec.NewChunkDecoder(cols, -1, nil, time.Local) + decoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, time.Local) chk := chunk.NewChunkWithCapacity(tps, 1) for i := 0; i < b.N; i++ { chk.Reset() - err = decoder.DecodeToChunk(xRowData, 1, chk) + err = decoder.DecodeToChunk(xRowData, kv.IntHandle(1), chk) if err != nil { b.Fatal(err) } diff --git a/util/rowcodec/common.go b/util/rowcodec/common.go index ef1a20b035e42..c939fa7e79594 100644 --- a/util/rowcodec/common.go +++ b/util/rowcodec/common.go @@ -129,7 +129,7 @@ func encodeUint(buf []byte, uVal uint64) []byte { binary.LittleEndian.PutUint32(tmp[:], uint32(uVal)) buf = append(buf, tmp[:4]...) } else { - binary.LittleEndian.PutUint64(tmp[:], uint64(uVal)) + binary.LittleEndian.PutUint64(tmp[:], uVal) buf = append(buf, tmp[:8]...) } return buf diff --git a/util/rowcodec/decoder.go b/util/rowcodec/decoder.go index 8352fc8fa2efc..2596934807bb9 100644 --- a/util/rowcodec/decoder.go +++ b/util/rowcodec/decoder.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" @@ -29,17 +30,17 @@ import ( // decoder contains base util for decode row. type decoder struct { row - columns []ColInfo - handleColID int64 - loc *time.Location + columns []ColInfo + handleColIDs []int64 + loc *time.Location } // NewDecoder creates a decoder. -func NewDecoder(columns []ColInfo, handleColID int64, loc *time.Location) *decoder { +func NewDecoder(columns []ColInfo, handleColIDs []int64, loc *time.Location) *decoder { return &decoder{ - columns: columns, - handleColID: handleColID, - loc: loc, + columns: columns, + handleColIDs: handleColIDs, + loc: loc, } } @@ -63,16 +64,16 @@ type DatumMapDecoder struct { } // NewDatumMapDecoder creates a DatumMapDecoder. -func NewDatumMapDecoder(columns []ColInfo, handleColID int64, loc *time.Location) *DatumMapDecoder { +func NewDatumMapDecoder(columns []ColInfo, handleColIDs []int64, loc *time.Location) *DatumMapDecoder { return &DatumMapDecoder{decoder{ - columns: columns, - handleColID: handleColID, - loc: loc, + columns: columns, + handleColIDs: handleColIDs, + loc: loc, }} } // DecodeToDatumMap decodes byte slices to datum map. -func (decoder *DatumMapDecoder) DecodeToDatumMap(rowData []byte, handle int64, row map[int64]types.Datum) (map[int64]types.Datum, error) { +func (decoder *DatumMapDecoder) DecodeToDatumMap(rowData []byte, handle kv.Handle, row map[int64]types.Datum) (map[int64]types.Datum, error) { if row == nil { row = make(map[int64]types.Datum, len(decoder.columns)) } @@ -80,19 +81,15 @@ func (decoder *DatumMapDecoder) DecodeToDatumMap(rowData []byte, handle int64, r if err != nil { return nil, err } - for _, col := range decoder.columns { - if col.ID == decoder.handleColID { - if mysql.HasUnsignedFlag(uint(col.Flag)) { - row[col.ID] = types.NewUintDatum(uint64(handle)) - } else { - row[col.ID] = types.NewIntDatum(handle) - } + for i := range decoder.columns { + col := &decoder.columns[i] + if decoder.tryDecodeHandleColumn(col, handle, row) { continue } idx, isNil, notFound := decoder.row.findColID(col.ID) if !notFound && !isNil { colData := decoder.getData(idx) - d, err := decoder.decodeColDatum(&col, colData) + d, err := decoder.decodeColDatum(col, colData) if err != nil { return nil, err } @@ -110,6 +107,21 @@ func (decoder *DatumMapDecoder) DecodeToDatumMap(rowData []byte, handle int64, r return row, nil } +func (decoder *DatumMapDecoder) tryDecodeHandleColumn(col *ColInfo, handle kv.Handle, row map[int64]types.Datum) bool { + if handle == nil { + return false + } + if col.ID == decoder.handleColIDs[0] { + if mysql.HasUnsignedFlag(uint(col.Flag)) { + row[col.ID] = types.NewUintDatum(uint64(handle.IntValue())) + } else { + row[col.ID] = types.NewIntDatum(handle.IntValue()) + } + return true + } + return false +} + func (decoder *DatumMapDecoder) decodeColDatum(col *ColInfo, colData []byte) (types.Datum, error) { var d types.Datum switch byte(col.Tp) { @@ -199,27 +211,27 @@ type ChunkDecoder struct { } // NewChunkDecoder creates a NewChunkDecoder. -func NewChunkDecoder(columns []ColInfo, handleColID int64, defDatum func(i int, chk *chunk.Chunk) error, loc *time.Location) *ChunkDecoder { +func NewChunkDecoder(columns []ColInfo, handleColIDs []int64, defDatum func(i int, chk *chunk.Chunk) error, loc *time.Location) *ChunkDecoder { return &ChunkDecoder{ decoder: decoder{ - columns: columns, - handleColID: handleColID, - loc: loc, + columns: columns, + handleColIDs: handleColIDs, + loc: loc, }, defDatum: defDatum, } } // DecodeToChunk decodes a row to chunk. -func (decoder *ChunkDecoder) DecodeToChunk(rowData []byte, handle int64, chk *chunk.Chunk) error { +func (decoder *ChunkDecoder) DecodeToChunk(rowData []byte, handle kv.Handle, chk *chunk.Chunk) error { err := decoder.fromBytes(rowData) if err != nil { return err } - for colIdx, col := range decoder.columns { - if col.ID == decoder.handleColID { - chk.AppendInt64(colIdx, handle) + for colIdx := range decoder.columns { + col := &decoder.columns[colIdx] + if decoder.tryAppendHandleColumn(colIdx, col, handle, chk) { continue } // fill the virtual column value after row calculation @@ -231,7 +243,7 @@ func (decoder *ChunkDecoder) DecodeToChunk(rowData []byte, handle int64, chk *ch idx, isNil, notFound := decoder.row.findColID(col.ID) if !notFound && !isNil { colData := decoder.getData(idx) - err := decoder.decodeColToChunk(colIdx, &col, colData, chk) + err := decoder.decodeColToChunk(colIdx, col, colData, chk) if err != nil { return err } @@ -256,6 +268,17 @@ func (decoder *ChunkDecoder) DecodeToChunk(rowData []byte, handle int64, chk *ch return nil } +func (decoder *ChunkDecoder) tryAppendHandleColumn(colIdx int, col *ColInfo, handle kv.Handle, chk *chunk.Chunk) bool { + if handle == nil { + return false + } + if col.ID == decoder.handleColIDs[0] { + chk.AppendInt64(colIdx, handle.IntValue()) + return true + } + return false +} + func (decoder *ChunkDecoder) decodeColToChunk(colIdx int, col *ColInfo, colData []byte, chk *chunk.Chunk) error { switch byte(col.Tp) { case mysql.TypeLonglong, mysql.TypeLong, mysql.TypeInt24, mysql.TypeShort, mysql.TypeTiny: @@ -350,38 +373,30 @@ type BytesDecoder struct { // NewByteDecoder creates a BytesDecoder. // defBytes: provided default value bytes in old datum format(flag+colData). -func NewByteDecoder(columns []ColInfo, handleColID int64, defBytes func(i int) ([]byte, error), loc *time.Location) *BytesDecoder { +func NewByteDecoder(columns []ColInfo, handleColIDs []int64, defBytes func(i int) ([]byte, error), loc *time.Location) *BytesDecoder { return &BytesDecoder{ decoder: decoder{ - columns: columns, - handleColID: handleColID, - loc: loc, + columns: columns, + handleColIDs: handleColIDs, + loc: loc, }, defBytes: defBytes, } } -func (decoder *BytesDecoder) decodeToBytesInternal(outputOffset map[int64]int, handle int64, value []byte, cacheBytes []byte, useHandle bool) ([][]byte, error) { +func (decoder *BytesDecoder) decodeToBytesInternal(outputOffset map[int64]int, handle kv.Handle, value []byte, cacheBytes []byte) ([][]byte, error) { var r row err := r.fromBytes(value) if err != nil { return nil, err } values := make([][]byte, len(outputOffset)) - for i, col := range decoder.columns { + for i := range decoder.columns { + col := &decoder.columns[i] tp := fieldType2Flag(byte(col.Tp), uint(col.Flag)&mysql.UnsignedFlag == 0) colID := col.ID offset := outputOffset[colID] - if useHandle && (col.IsPKHandle || colID == model.ExtraHandleID) { - handleData := cacheBytes - if mysql.HasUnsignedFlag(uint(col.Flag)) { - handleData = append(handleData, UintFlag) - handleData = codec.EncodeUint(handleData, uint64(handle)) - } else { - handleData = append(handleData, IntFlag) - handleData = codec.EncodeInt(handleData, handle) - } - values[offset] = handleData + if decoder.tryDecodeHandle(values, offset, col, handle, cacheBytes) { continue } @@ -413,14 +428,34 @@ func (decoder *BytesDecoder) decodeToBytesInternal(outputOffset map[int64]int, h return values, nil } +func (decoder *BytesDecoder) tryDecodeHandle(values [][]byte, offset int, col *ColInfo, + handle kv.Handle, cacheBytes []byte) bool { + if handle == nil { + return false + } + if col.IsPKHandle || col.ID == model.ExtraHandleID { + handleData := cacheBytes + if mysql.HasUnsignedFlag(uint(col.Flag)) { + handleData = append(handleData, UintFlag) + handleData = codec.EncodeUint(handleData, uint64(handle.IntValue())) + } else { + handleData = append(handleData, IntFlag) + handleData = codec.EncodeInt(handleData, handle.IntValue()) + } + values[offset] = handleData + return true + } + return false +} + // DecodeToBytesNoHandle decodes raw byte slice to row dat without handle. func (decoder *BytesDecoder) DecodeToBytesNoHandle(outputOffset map[int64]int, value []byte) ([][]byte, error) { - return decoder.decodeToBytesInternal(outputOffset, 0, value, nil, false) + return decoder.decodeToBytesInternal(outputOffset, nil, value, nil) } // DecodeToBytes decodes raw byte slice to row data. -func (decoder *BytesDecoder) DecodeToBytes(outputOffset map[int64]int, handle int64, value []byte, cacheBytes []byte) ([][]byte, error) { - return decoder.decodeToBytesInternal(outputOffset, handle, value, cacheBytes, true) +func (decoder *BytesDecoder) DecodeToBytes(outputOffset map[int64]int, handle kv.Handle, value []byte, cacheBytes []byte) ([][]byte, error) { + return decoder.decodeToBytesInternal(outputOffset, handle, value, cacheBytes) } func (decoder *BytesDecoder) encodeOldDatum(tp byte, val []byte) []byte { diff --git a/util/rowcodec/encoder.go b/util/rowcodec/encoder.go index 355f7e57440e9..8e865adf253f5 100644 --- a/util/rowcodec/encoder.go +++ b/util/rowcodec/encoder.go @@ -130,7 +130,7 @@ func (encoder *Encoder) encodeRowCols(sc *stmtctx.StatementContext, numCols, not for i := 0; i < notNullIdx; i++ { d := encoder.values[i] var err error - r.data, err = EncodeValueDatum(sc, d, r.data) + r.data, err = encodeValueDatum(sc, d, r.data) if err != nil { return err } @@ -155,9 +155,9 @@ func (encoder *Encoder) encodeRowCols(sc *stmtctx.StatementContext, numCols, not return nil } -// EncodeValueDatum encodes one row datum entry into bytes. +// encodeValueDatum encodes one row datum entry into bytes. // due to encode as value, this method will flatten value type like tablecodec.flatten -func EncodeValueDatum(sc *stmtctx.StatementContext, d types.Datum, buffer []byte) (nBuffer []byte, err error) { +func encodeValueDatum(sc *stmtctx.StatementContext, d types.Datum, buffer []byte) (nBuffer []byte, err error) { switch d.Kind() { case types.KindInt64: buffer = encodeInt(buffer, d.GetInt64()) @@ -209,9 +209,6 @@ func EncodeValueDatum(sc *stmtctx.StatementContext, d types.Datum, buffer []byte j := d.GetMysqlJSON() buffer = append(buffer, j.TypeCode) buffer = append(buffer, j.Value...) - case types.KindNull: - case types.KindMinNotNull: - case types.KindMaxValue: default: err = errors.Errorf("unsupport encode type %d", d.Kind()) } diff --git a/util/rowcodec/rowcodec_test.go b/util/rowcodec/rowcodec_test.go index 04b1c3455f5e1..1a3995060b50f 100644 --- a/util/rowcodec/rowcodec_test.go +++ b/util/rowcodec/rowcodec_test.go @@ -21,6 +21,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -67,8 +68,8 @@ func (s *testSuite) TestEncodeLargeSmallReuseBug(c *C) { Decimal: colFt.Decimal, Elems: colFt.Elems, }, - }, -1, nil) - m, err := bDecoder.DecodeToDatumMap(b, -1, nil) + }, []int64{-1}, nil) + m, err := bDecoder.DecodeToDatumMap(b, kv.IntHandle(-1), nil) c.Assert(err, IsNil) v := m[largeColID] @@ -87,8 +88,8 @@ func (s *testSuite) TestEncodeLargeSmallReuseBug(c *C) { Decimal: colFt.Decimal, Elems: colFt.Elems, }, - }, -1, nil) - m, err = bDecoder.DecodeToDatumMap(b, -1, nil) + }, []int64{-1}, nil) + m, err = bDecoder.DecodeToDatumMap(b, kv.IntHandle(-1), nil) c.Assert(err, IsNil) v = m[smallColID] c.Assert(v.GetInt64(), Equals, int64(2)) @@ -131,8 +132,8 @@ func (s *testSuite) TestDecodeRowWithHandle(c *C) { c.Assert(err, IsNil) // decode to datum map. - mDecoder := rowcodec.NewDatumMapDecoder(cols, -1, sc.TimeZone) - dm, err := mDecoder.DecodeToDatumMap(newRow, handleValue, nil) + mDecoder := rowcodec.NewDatumMapDecoder(cols, []int64{-1}, sc.TimeZone) + dm, err := mDecoder.DecodeToDatumMap(newRow, kv.IntHandle(handleValue), nil) c.Assert(err, IsNil) for _, t := range testData { d, exists := dm[t.id] @@ -141,9 +142,9 @@ func (s *testSuite) TestDecodeRowWithHandle(c *C) { } // decode to chunk. - cDecoder := rowcodec.NewChunkDecoder(cols, -1, nil, sc.TimeZone) + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) chk := chunk.New(fts, 1, 1) - err = cDecoder.DecodeToChunk(newRow, handleValue, chk) + err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(handleValue), chk) c.Assert(err, IsNil) chkRow := chk.GetRow(0) cdt := chkRow.GetDatumRow(fts) @@ -161,8 +162,8 @@ func (s *testSuite) TestDecodeRowWithHandle(c *C) { for i, t := range testData { colOffset[t.id] = i } - bDecoder := rowcodec.NewByteDecoder(cols, -1, nil, nil) - oldRow, err := bDecoder.DecodeToBytes(colOffset, handleValue, newRow, nil) + bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, nil) + oldRow, err := bDecoder.DecodeToBytes(colOffset, kv.IntHandle(handleValue), newRow, nil) c.Assert(err, IsNil) for i, t := range testData { remain, d, err := codec.DecodeOne(oldRow[i]) @@ -219,6 +220,88 @@ func (s *testSuite) TestDecodeRowWithHandle(c *C) { encodeAndDecodeHandle(c, testDataUnsigned) } +func (s *testSuite) TestEncodeKindNullDatum(c *C) { + var encoder rowcodec.Encoder + sc := new(stmtctx.StatementContext) + sc.TimeZone = time.UTC + colIDs := []int64{ + 1, + 2, + } + var nilDt types.Datum + nilDt.SetNull() + dts := []types.Datum{nilDt, types.NewIntDatum(2)} + ft := types.NewFieldType(mysql.TypeLonglong) + fts := []*types.FieldType{ft, ft} + newRow, err := encoder.Encode(sc, colIDs, dts, nil) + c.Assert(err, IsNil) + + cols := []rowcodec.ColInfo{{ + ID: 1, + Tp: int32(ft.Tp), + Flag: int32(ft.Flag), + Flen: ft.Flen, + Decimal: ft.Decimal, + Elems: ft.Elems, + Collate: ft.Collate, + }, + { + ID: 2, + Tp: int32(ft.Tp), + Flag: int32(ft.Flag), + Flen: ft.Flen, + Decimal: ft.Decimal, + Elems: ft.Elems, + Collate: ft.Collate, + }} + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) + chk := chunk.New(fts, 1, 1) + err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) + c.Assert(err, IsNil) + chkRow := chk.GetRow(0) + cdt := chkRow.GetDatumRow(fts) + c.Assert(cdt[0].IsNull(), Equals, true) + c.Assert(cdt[1].GetInt64(), Equals, int64(2)) +} + +func (s *testSuite) TestDecodeDecimalFspNotMatch(c *C) { + var encoder rowcodec.Encoder + sc := new(stmtctx.StatementContext) + sc.TimeZone = time.UTC + colIDs := []int64{ + 1, + } + dec := withFrac(4)(withLen(6)(types.NewDecimalDatum(types.NewDecFromStringForTest("11.9900")))) + dts := []types.Datum{dec} + ft := types.NewFieldType(mysql.TypeNewDecimal) + ft.Decimal = 4 + fts := []*types.FieldType{ft} + newRow, err := encoder.Encode(sc, colIDs, dts, nil) + c.Assert(err, IsNil) + + // decode to chunk. + ft = types.NewFieldType(mysql.TypeNewDecimal) + ft.Decimal = 3 + cols := make([]rowcodec.ColInfo, 0) + cols = append(cols, rowcodec.ColInfo{ + ID: 1, + Tp: int32(ft.Tp), + Flag: int32(ft.Flag), + Flen: ft.Flen, + Decimal: ft.Decimal, + Elems: ft.Elems, + Collate: ft.Collate, + }) + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) + chk := chunk.New(fts, 1, 1) + err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) + c.Assert(err, IsNil) + chkRow := chk.GetRow(0) + cdt := chkRow.GetDatumRow(fts) + dec = withFrac(3)(withLen(6)(types.NewDecimalDatum(types.NewDecFromStringForTest("11.990")))) + c.Assert(cdt[0].GetMysqlDecimal().String(), DeepEquals, dec.GetMysqlDecimal().String()) +} + func (s *testSuite) TestTypesNewRowCodec(c *C) { getJSONDatum := func(value string) types.Datum { j, err := json.ParseBinaryFromString(value) @@ -238,6 +321,7 @@ func (s *testSuite) TestTypesNewRowCodec(c *C) { return t } + var encoder rowcodec.Encoder encodeAndDecode := func(c *C, testData []testData) { // transform test data into input. colIDs := make([]int64, 0, len(testData)) @@ -262,15 +346,14 @@ func (s *testSuite) TestTypesNewRowCodec(c *C) { } // test encode input. - var encoder rowcodec.Encoder sc := new(stmtctx.StatementContext) sc.TimeZone = time.UTC newRow, err := encoder.Encode(sc, colIDs, dts, nil) c.Assert(err, IsNil) // decode to datum map. - mDecoder := rowcodec.NewDatumMapDecoder(cols, -1, sc.TimeZone) - dm, err := mDecoder.DecodeToDatumMap(newRow, -1, nil) + mDecoder := rowcodec.NewDatumMapDecoder(cols, []int64{-1}, sc.TimeZone) + dm, err := mDecoder.DecodeToDatumMap(newRow, kv.IntHandle(-1), nil) c.Assert(err, IsNil) for _, t := range testData { d, exists := dm[t.id] @@ -279,9 +362,9 @@ func (s *testSuite) TestTypesNewRowCodec(c *C) { } // decode to chunk. - cDecoder := rowcodec.NewChunkDecoder(cols, -1, nil, sc.TimeZone) + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) chk := chunk.New(fts, 1, 1) - err = cDecoder.DecodeToChunk(newRow, -1, chk) + err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) c.Assert(err, IsNil) chkRow := chk.GetRow(0) cdt := chkRow.GetDatumRow(fts) @@ -299,8 +382,8 @@ func (s *testSuite) TestTypesNewRowCodec(c *C) { for i, t := range testData { colOffset[t.id] = i } - bDecoder := rowcodec.NewByteDecoder(cols, -1, nil, nil) - oldRow, err := bDecoder.DecodeToBytes(colOffset, -1, newRow, nil) + bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, nil) + oldRow, err := bDecoder.DecodeToBytes(colOffset, kv.IntHandle(-1), newRow, nil) c.Assert(err, IsNil) for i, t := range testData { remain, d, err := codec.DecodeOne(oldRow[i]) @@ -524,8 +607,8 @@ func (s *testSuite) TestNilAndDefault(c *C) { c.Assert(err, IsNil) // decode to datum map. - mDecoder := rowcodec.NewDatumMapDecoder(cols, -1, sc.TimeZone) - dm, err := mDecoder.DecodeToDatumMap(newRow, -1, nil) + mDecoder := rowcodec.NewDatumMapDecoder(cols, []int64{-1}, sc.TimeZone) + dm, err := mDecoder.DecodeToDatumMap(newRow, kv.IntHandle(-1), nil) c.Assert(err, IsNil) for _, t := range testData { d, exists := dm[t.id] @@ -540,8 +623,8 @@ func (s *testSuite) TestNilAndDefault(c *C) { //decode to chunk. chk := chunk.New(fts, 1, 1) - cDecoder := rowcodec.NewChunkDecoder(cols, -1, ddf, sc.TimeZone) - err = cDecoder.DecodeToChunk(newRow, -1, chk) + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, ddf, sc.TimeZone) + err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) c.Assert(err, IsNil) chkRow := chk.GetRow(0) cdt := chkRow.GetDatumRow(fts) @@ -554,13 +637,27 @@ func (s *testSuite) TestNilAndDefault(c *C) { } } + chk = chunk.New(fts, 1, 1) + cDecoder = rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) + err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) + c.Assert(err, IsNil) + chkRow = chk.GetRow(0) + cdt = chkRow.GetDatumRow(fts) + for i := range testData { + if i == 0 { + continue + } + d := cdt[i] + c.Assert(d.IsNull(), Equals, true) + } + // decode to old row bytes. colOffset := make(map[int64]int) for i, t := range testData { colOffset[t.id] = i } - bDecoder := rowcodec.NewByteDecoder(cols, -1, bdf, sc.TimeZone) - oldRow, err := bDecoder.DecodeToBytes(colOffset, -1, newRow, nil) + bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, bdf, sc.TimeZone) + oldRow, err := bDecoder.DecodeToBytes(colOffset, kv.IntHandle(-1), newRow, nil) c.Assert(err, IsNil) for i, t := range testData { remain, d, err := codec.DecodeOne(oldRow[i]) @@ -624,13 +721,13 @@ func (s *testSuite) TestVarintCompatibility(c *C) { sc.TimeZone = time.UTC newRow, err := encoder.Encode(sc, colIDs, dts, nil) c.Assert(err, IsNil) - decoder := rowcodec.NewByteDecoder(cols, -1, nil, sc.TimeZone) + decoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, sc.TimeZone) // decode to old row bytes. colOffset := make(map[int64]int) for i, t := range testData { colOffset[t.id] = i } - oldRow, err := decoder.DecodeToBytes(colOffset, 1, newRow, nil) + oldRow, err := decoder.DecodeToBytes(colOffset, kv.IntHandle(1), newRow, nil) c.Assert(err, IsNil) for i, t := range testData { oldVarint, err := tablecodec.EncodeValue(nil, nil, t.bt) // tablecodec will encode as varint/varuint @@ -693,7 +790,7 @@ func (s *testSuite) TestCodecUtil(c *C) { Collate: ft.Collate, }) } - d := rowcodec.NewDecoder(cols, -1, nil) + d := rowcodec.NewDecoder(cols, []int64{-1}, nil) // test ColumnIsNull isNil, err := d.ColumnIsNull(newRow, 4, nil) @@ -743,9 +840,9 @@ func (s *testSuite) TestOldRowCodec(c *C) { Collate: tp.Collate, } } - rd := rowcodec.NewChunkDecoder(cols, 0, nil, time.Local) + rd := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, time.Local) chk := chunk.NewChunkWithCapacity(tps, 1) - err = rd.DecodeToChunk(newRow, -1, chk) + err = rd.DecodeToChunk(newRow, kv.IntHandle(-1), chk) c.Assert(err, IsNil) row := chk.GetRow(0) for i := 0; i < 3; i++ { @@ -769,8 +866,8 @@ func (s *testSuite) Test65535Bug(c *C) { Tp: int32(tps[0].Tp), Flag: int32(tps[0].Flag), } - dc := rowcodec.NewDatumMapDecoder(cols, -1, nil) - result, err := dc.DecodeToDatumMap(bd, -1, nil) + dc := rowcodec.NewDatumMapDecoder(cols, []int64{-1}, nil) + result, err := dc.DecodeToDatumMap(bd, kv.IntHandle(-1), nil) c.Check(err, IsNil) rs := result[1] c.Check(rs.GetString(), Equals, text65535) diff --git a/util/stmtsummary/statement_summary.go b/util/stmtsummary/statement_summary.go index 01ef85d013dc0..12558adaeb5b0 100644 --- a/util/stmtsummary/statement_summary.go +++ b/util/stmtsummary/statement_summary.go @@ -35,9 +35,6 @@ import ( "go.uber.org/zap" ) -// There're many types of statement summary tables in MySQL, but we have -// only implemented events_statements_summary_by_digest for now. - // stmtSummaryByDigestKey defines key for stmtSummaryByDigestMap.summaryMap. type stmtSummaryByDigestKey struct { // Same statements may appear in different schema, but they refer to different tables. @@ -106,11 +103,13 @@ type stmtSummaryByDigestElement struct { beginTime int64 endTime int64 // basic - sampleSQL string - prevSQL string - samplePlan string - indexNames []string - execCount int64 + sampleSQL string + prevSQL string + samplePlan string + indexNames []string + execCount int64 + sumErrors int + sumWarnings int // latency sumLatency time.Duration maxLatency time.Duration @@ -120,11 +119,9 @@ type stmtSummaryByDigestElement struct { sumCompileLatency time.Duration maxCompileLatency time.Duration // coprocessor - numCopTasks int64 - sumCopProcessTime int64 + sumNumCopTasks int64 maxCopProcessTime time.Duration maxCopProcessAddress string - sumCopWaitTime int64 maxCopWaitTime time.Duration maxCopWaitAddress string // TiKV @@ -194,6 +191,7 @@ type StmtExecInfo struct { MemMax int64 StartTime time.Time IsInternal bool + Succeed bool } // newStmtSummaryByDigestMap creates an empty stmtSummaryByDigestMap. @@ -612,6 +610,10 @@ func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeco // refreshInterval may change anytime, update endTime ASAP. ssElement.endTime = ssElement.beginTime + intervalSeconds ssElement.execCount++ + if !sei.Succeed { + ssElement.sumErrors += 1 + } + ssElement.sumWarnings += int(sei.StmtCtx.WarningCount()) // latency ssElement.sumLatency += sei.TotalLatency @@ -632,13 +634,11 @@ func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeco // coprocessor numCopTasks := int64(sei.CopTasks.NumCopTasks) - ssElement.numCopTasks += numCopTasks - ssElement.sumCopProcessTime += sei.CopTasks.AvgProcessTime.Nanoseconds() * numCopTasks + ssElement.sumNumCopTasks += numCopTasks if sei.CopTasks.MaxProcessTime > ssElement.maxCopProcessTime { ssElement.maxCopProcessTime = sei.CopTasks.MaxProcessTime ssElement.maxCopProcessAddress = sei.CopTasks.MaxProcessAddress } - ssElement.sumCopWaitTime += sei.CopTasks.AvgWaitTime.Nanoseconds() * numCopTasks if sei.CopTasks.MaxWaitTime > ssElement.maxCopWaitTime { ssElement.maxCopWaitTime = sei.CopTasks.MaxWaitTime ssElement.maxCopWaitAddress = sei.CopTasks.MaxWaitAddress @@ -763,6 +763,8 @@ func (ssElement *stmtSummaryByDigestElement) toDatum(ssbd *stmtSummaryByDigest) convertEmptyToNil(strings.Join(ssElement.indexNames, ",")), convertEmptyToNil(sampleUser), ssElement.execCount, + ssElement.sumErrors, + ssElement.sumWarnings, int64(ssElement.sumLatency), int64(ssElement.maxLatency), int64(ssElement.minLatency), @@ -771,11 +773,9 @@ func (ssElement *stmtSummaryByDigestElement) toDatum(ssbd *stmtSummaryByDigest) int64(ssElement.maxParseLatency), avgInt(int64(ssElement.sumCompileLatency), ssElement.execCount), int64(ssElement.maxCompileLatency), - ssElement.numCopTasks, - avgInt(ssElement.sumCopProcessTime, ssElement.numCopTasks), + ssElement.sumNumCopTasks, int64(ssElement.maxCopProcessTime), convertEmptyToNil(ssElement.maxCopProcessAddress), - avgInt(ssElement.sumCopWaitTime, ssElement.numCopTasks), int64(ssElement.maxCopWaitTime), convertEmptyToNil(ssElement.maxCopWaitAddress), avgInt(int64(ssElement.sumProcessTime), ssElement.execCount), diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index c9ea94c619963..1f08ecd9e16e6 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -89,11 +89,9 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { maxParseLatency: stmtExecInfo1.ParseLatency, sumCompileLatency: stmtExecInfo1.CompileLatency, maxCompileLatency: stmtExecInfo1.CompileLatency, - numCopTasks: int64(stmtExecInfo1.CopTasks.NumCopTasks), - sumCopProcessTime: int64(stmtExecInfo1.CopTasks.AvgProcessTime) * int64(stmtExecInfo1.CopTasks.NumCopTasks), + sumNumCopTasks: int64(stmtExecInfo1.CopTasks.NumCopTasks), maxCopProcessTime: stmtExecInfo1.CopTasks.MaxProcessTime, maxCopProcessAddress: stmtExecInfo1.CopTasks.MaxProcessAddress, - sumCopWaitTime: int64(stmtExecInfo1.CopTasks.AvgWaitTime) * int64(stmtExecInfo1.CopTasks.NumCopTasks), maxCopWaitTime: stmtExecInfo1.CopTasks.MaxWaitTime, maxCopWaitAddress: stmtExecInfo1.CopTasks.MaxWaitAddress, sumProcessTime: stmtExecInfo1.ExecDetail.ProcessTime, @@ -207,6 +205,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { }, MemMax: 20000, StartTime: time.Date(2019, 1, 1, 10, 10, 20, 10, time.UTC), + Succeed: true, } stmtExecInfo2.StmtCtx.AddAffectedRows(200) expectedSummaryElement.execCount++ @@ -216,11 +215,9 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { expectedSummaryElement.maxParseLatency = stmtExecInfo2.ParseLatency expectedSummaryElement.sumCompileLatency += stmtExecInfo2.CompileLatency expectedSummaryElement.maxCompileLatency = stmtExecInfo2.CompileLatency - expectedSummaryElement.numCopTasks += int64(stmtExecInfo2.CopTasks.NumCopTasks) - expectedSummaryElement.sumCopProcessTime += int64(stmtExecInfo2.CopTasks.AvgProcessTime) * int64(stmtExecInfo2.CopTasks.NumCopTasks) + expectedSummaryElement.sumNumCopTasks += int64(stmtExecInfo2.CopTasks.NumCopTasks) expectedSummaryElement.maxCopProcessTime = stmtExecInfo2.CopTasks.MaxProcessTime expectedSummaryElement.maxCopProcessAddress = stmtExecInfo2.CopTasks.MaxProcessAddress - expectedSummaryElement.sumCopWaitTime += int64(stmtExecInfo2.CopTasks.AvgWaitTime) * int64(stmtExecInfo2.CopTasks.NumCopTasks) expectedSummaryElement.maxCopWaitTime = stmtExecInfo2.CopTasks.MaxWaitTime expectedSummaryElement.maxCopWaitAddress = stmtExecInfo2.CopTasks.MaxWaitAddress expectedSummaryElement.sumProcessTime += stmtExecInfo2.ExecDetail.ProcessTime @@ -323,6 +320,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { }, MemMax: 200, StartTime: time.Date(2019, 1, 1, 10, 10, 0, 10, time.UTC), + Succeed: true, } stmtExecInfo3.StmtCtx.AddAffectedRows(20000) expectedSummaryElement.execCount++ @@ -330,9 +328,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { expectedSummaryElement.minLatency = stmtExecInfo3.TotalLatency expectedSummaryElement.sumParseLatency += stmtExecInfo3.ParseLatency expectedSummaryElement.sumCompileLatency += stmtExecInfo3.CompileLatency - expectedSummaryElement.numCopTasks += int64(stmtExecInfo3.CopTasks.NumCopTasks) - expectedSummaryElement.sumCopProcessTime += int64(stmtExecInfo3.CopTasks.AvgProcessTime) * int64(stmtExecInfo3.CopTasks.NumCopTasks) - expectedSummaryElement.sumCopWaitTime += int64(stmtExecInfo3.CopTasks.AvgWaitTime) * int64(stmtExecInfo3.CopTasks.NumCopTasks) + expectedSummaryElement.sumNumCopTasks += int64(stmtExecInfo3.CopTasks.NumCopTasks) expectedSummaryElement.sumProcessTime += stmtExecInfo3.ExecDetail.ProcessTime expectedSummaryElement.sumWaitTime += stmtExecInfo3.ExecDetail.WaitTime expectedSummaryElement.sumBackoffTime += stmtExecInfo3.ExecDetail.BackoffTime @@ -426,6 +422,8 @@ func matchStmtSummaryByDigest(first, second *stmtSummaryByDigest) bool { ssElement1.samplePlan != ssElement2.samplePlan || ssElement1.prevSQL != ssElement2.prevSQL || ssElement1.execCount != ssElement2.execCount || + ssElement1.sumErrors != ssElement2.sumErrors || + ssElement1.sumWarnings != ssElement2.sumWarnings || ssElement1.sumLatency != ssElement2.sumLatency || ssElement1.maxLatency != ssElement2.maxLatency || ssElement1.minLatency != ssElement2.minLatency || @@ -433,11 +431,9 @@ func matchStmtSummaryByDigest(first, second *stmtSummaryByDigest) bool { ssElement1.maxParseLatency != ssElement2.maxParseLatency || ssElement1.sumCompileLatency != ssElement2.sumCompileLatency || ssElement1.maxCompileLatency != ssElement2.maxCompileLatency || - ssElement1.numCopTasks != ssElement2.numCopTasks || - ssElement1.sumCopProcessTime != ssElement2.sumCopProcessTime || + ssElement1.sumNumCopTasks != ssElement2.sumNumCopTasks || ssElement1.maxCopProcessTime != ssElement2.maxCopProcessTime || ssElement1.maxCopProcessAddress != ssElement2.maxCopProcessAddress || - ssElement1.sumCopWaitTime != ssElement2.sumCopWaitTime || ssElement1.maxCopWaitTime != ssElement2.maxCopWaitTime || ssElement1.maxCopWaitAddress != ssElement2.maxCopWaitAddress || ssElement1.sumProcessTime != ssElement2.sumProcessTime || @@ -569,6 +565,7 @@ func generateAnyExecInfo() *StmtExecInfo { }, MemMax: 10000, StartTime: time.Date(2019, 1, 1, 10, 10, 10, 10, time.UTC), + Succeed: true, } stmtExecInfo.StmtCtx.AddAffectedRows(10000) return stmtExecInfo @@ -589,12 +586,11 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) { e := types.NewTime(types.FromGoTime(time.Unix(s.ssMap.beginTimeForCurInterval+1800, 0)), mysql.TypeTimestamp, types.DefaultFsp) t := types.NewTime(types.FromGoTime(stmtExecInfo1.StartTime), mysql.TypeTimestamp, types.DefaultFsp) expectedDatum := []interface{}{n, e, "select", stmtExecInfo1.SchemaName, stmtExecInfo1.Digest, stmtExecInfo1.NormalizedSQL, - "db1.tb1,db2.tb2", "a", stmtExecInfo1.User, 1, int64(stmtExecInfo1.TotalLatency), + "db1.tb1,db2.tb2", "a", stmtExecInfo1.User, 1, 0, 0, int64(stmtExecInfo1.TotalLatency), int64(stmtExecInfo1.TotalLatency), int64(stmtExecInfo1.TotalLatency), int64(stmtExecInfo1.TotalLatency), int64(stmtExecInfo1.ParseLatency), int64(stmtExecInfo1.ParseLatency), int64(stmtExecInfo1.CompileLatency), - int64(stmtExecInfo1.CompileLatency), stmtExecInfo1.CopTasks.NumCopTasks, int64(stmtExecInfo1.CopTasks.AvgProcessTime), - int64(stmtExecInfo1.CopTasks.MaxProcessTime), stmtExecInfo1.CopTasks.MaxProcessAddress, - int64(stmtExecInfo1.CopTasks.AvgWaitTime), int64(stmtExecInfo1.CopTasks.MaxWaitTime), + int64(stmtExecInfo1.CompileLatency), stmtExecInfo1.CopTasks.NumCopTasks, int64(stmtExecInfo1.CopTasks.MaxProcessTime), + stmtExecInfo1.CopTasks.MaxProcessAddress, int64(stmtExecInfo1.CopTasks.MaxWaitTime), stmtExecInfo1.CopTasks.MaxWaitAddress, int64(stmtExecInfo1.ExecDetail.ProcessTime), int64(stmtExecInfo1.ExecDetail.ProcessTime), int64(stmtExecInfo1.ExecDetail.WaitTime), int64(stmtExecInfo1.ExecDetail.WaitTime), int64(stmtExecInfo1.ExecDetail.BackoffTime), int64(stmtExecInfo1.ExecDetail.BackoffTime), stmtExecInfo1.ExecDetail.TotalKeys, stmtExecInfo1.ExecDetail.TotalKeys, diff --git a/util/sys/linux/sys_linux.go b/util/sys/linux/sys_linux.go index e336122b2c879..9768159562408 100644 --- a/util/sys/linux/sys_linux.go +++ b/util/sys/linux/sys_linux.go @@ -15,8 +15,9 @@ package linux import ( - "golang.org/x/sys/unix" "syscall" + + "golang.org/x/sys/unix" ) // OSVersion returns version info of operation system. diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 87edb0a020ecd..cd7c389385437 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -224,7 +224,7 @@ func (tk *TestKit) MustTableDual(sql string, args ...interface{}) *Result { func (tk *TestKit) MustPointGet(sql string, args ...interface{}) *Result { rs := tk.MustQuery("explain "+sql, args...) tk.c.Assert(len(rs.rows), check.Equals, 1) - tk.c.Assert(strings.Contains(rs.rows[0][0], "Point_Get"), check.IsTrue) + tk.c.Assert(strings.Contains(rs.rows[0][0], "Point_Get"), check.IsTrue, check.Commentf("plan %v", rs.rows[0][0])) return tk.MustQuery(sql, args...) } diff --git a/util/testutil/testutil.go b/util/testutil/testutil.go index 3ee8439248172..b36578e9dee19 100644 --- a/util/testutil/testutil.go +++ b/util/testutil/testutil.go @@ -342,7 +342,7 @@ func (a *autoRandom) RestoreAutoRandomTestConfig() { globalCfg.AlterPrimaryKey = a.originAlterPrimaryKey } -// MaskSortHandles masks highest shard_bits numbers of table handles and sort it. +// MaskSortHandles sorts the handles by lowest (fieldTypeBits - 1 - shardBitsCount) bits. func (a *autoRandom) MaskSortHandles(handles []int64, shardBitsCount int, fieldType byte) []int64 { typeBitsLength := mysql.DefaultLengthOfMysqlTypes[fieldType] * 8 const signBitCount = 1 diff --git a/util/timeutil/time.go b/util/timeutil/time.go index 80e859b56db32..8817f70acec0a 100644 --- a/util/timeutil/time.go +++ b/util/timeutil/time.go @@ -91,11 +91,11 @@ func inferTZNameFromFileName(path string) (string, error) { substrMojave := "zoneinfo.default" if idx := strings.Index(path, substrMojave); idx != -1 { - return string(path[idx+len(substrMojave)+1:]), nil + return path[idx+len(substrMojave)+1:], nil } if idx := strings.Index(path, substr); idx != -1 { - return string(path[idx+len(substr)+1:]), nil + return path[idx+len(substr)+1:], nil } return "", fmt.Errorf("path %s is not supported", path) } From 69fd90da5a2c15761ee1c4d86535720b433276ba Mon Sep 17 00:00:00 2001 From: Fei Han Date: Sat, 9 May 2020 14:52:47 +0800 Subject: [PATCH 46/61] check session var conflict --- planner/core/integration_test.go | 1 + sessionctx/variable/varsutil.go | 10 +++++++++- sessionctx/variable/varsutil_test.go | 10 ++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 240c5484cf790..5f6273b4a947e 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -380,6 +380,7 @@ func (s *testIntegrationSerialSuite) TestBroadcastJoin(c *C) { } tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_allow_batch_cop = 1") tk.MustExec("set @@session.tidb_opt_broadcast_join = 1") var input []string var output []struct { diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 4a690486a9777..ab1246772b19a 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -415,7 +415,12 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc return "ON", nil } return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) - case TiDBSkipUTF8Check, TiDBOptAggPushDown, TiDBOptDistinctAggPushDown, TiDBOptBCJ, + case TiDBOptBCJ: + if (strings.EqualFold(value, "ON") || value == "1") && vars.AllowBatchCop == 0 { + return value, ErrWrongValueForVar.GenWithStackByArgs("Can't set BCJ to 1 but tidb_allow_batch_cop is 0, please active batch cop at first.") + } + return value, nil + case TiDBSkipUTF8Check, TiDBOptAggPushDown, TiDBOptDistinctAggPushDown, TiDBOptInSubqToJoinAndAgg, TiDBEnableFastAnalyze, TiDBBatchInsert, TiDBDisableTxnAutoRetry, TiDBEnableStreaming, TiDBEnableChunkRPC, TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction, TiDBPProfSQLCPU, @@ -526,6 +531,9 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc if err != nil { return value, ErrWrongTypeForVar.GenWithStackByArgs(name) } + if v == 0 && vars.AllowBCJ { + return value, ErrWrongValueForVar.GenWithStackByArgs("Can't set batch cop 0 but tidb_opt_broadcast_join is 1, please set bcj 0 at first") + } if v < 0 || v > 2 { return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) } diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 22dfb3d69a1ab..145ff229890b4 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -65,6 +65,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.IndexLookupJoinConcurrency, Equals, DefIndexLookupJoinConcurrency) c.Assert(vars.HashJoinConcurrency, Equals, DefTiDBHashJoinConcurrency) c.Assert(vars.AllowBatchCop, Equals, DefTiDBAllowBatchCop) + c.Assert(vars.AllowBCJ, Equals, DefOptBCJ) c.Assert(vars.ProjectionConcurrency, Equals, int64(DefTiDBProjectionConcurrency)) c.Assert(vars.HashAggPartialConcurrency, Equals, DefTiDBHashAggPartialConcurrency) c.Assert(vars.HashAggFinalConcurrency, Equals, DefTiDBHashAggFinalConcurrency) @@ -193,6 +194,15 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.SQLMode, Equals, mode) } + err = SetSessionSystemVar(v, "tidb_opt_broadcast_join", types.NewStringDatum("1")) + c.Assert(terror.ErrorEqual(err, ErrWrongValueForVar), IsTrue) + err = SetSessionSystemVar(v, "tidb_allow_batch_cop", types.NewStringDatum("1")) + c.Assert(err, IsNil) + err = SetSessionSystemVar(v, "tidb_opt_broadcast_join", types.NewStringDatum("1")) + c.Assert(err, IsNil) + err = SetSessionSystemVar(v, "tidb_allow_batch_cop", types.NewStringDatum("0")) + c.Assert(terror.ErrorEqual(err, ErrWrongValueForVar), IsTrue) + // Combined sql_mode SetSessionSystemVar(v, "sql_mode", types.NewStringDatum("REAL_AS_FLOAT,ANSI_QUOTES")) c.Assert(v.SQLMode, Equals, mysql.ModeRealAsFloat|mysql.ModeANSIQuotes) From 694fa00da5ec59ea57df2b60aeda342dc2c22625 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=99=8E?= Date: Mon, 11 May 2020 15:36:29 +0800 Subject: [PATCH 47/61] Add perfer local hint for broadcast join (#12) * update * remove useless code * remove useless code * update parser * add test for prefer local join * use bcj_local * update go.mod --- go.mod | 2 +- go.sum | 12 +++++ planner/core/exhaust_physical_plans.go | 8 ++- planner/core/logical_plan_builder.go | 49 +++++++++++++------ planner/core/planbuilder.go | 35 +++++++++---- .../testdata/integration_serial_suite_in.json | 4 +- .../integration_serial_suite_out.json | 32 ++++++++++++ 7 files changed, 114 insertions(+), 28 deletions(-) diff --git a/go.mod b/go.mod index 5c77d26ac06f4..ec8240c79a414 100644 --- a/go.mod +++ b/go.mod @@ -2,7 +2,7 @@ module github.com/pingcap/tidb replace github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee => github.com/hanfei1991/tipb v0.0.0-20200506100205-b17edac4757f -replace github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657 => github.com/hanfei1991/parser v0.0.0-20200509083937-95b513acb77d +replace github.com/pingcap/parser => github.com/hanfei1991/parser v0.0.0-20200511041744-9942139265ca require ( github.com/BurntSushi/toml v0.3.1 diff --git a/go.sum b/go.sum index 89c90b62eccc8..3da44e05b6850 100644 --- a/go.sum +++ b/go.sum @@ -227,6 +227,10 @@ github.com/hanfei1991/parser v0.0.0-20200506102604-dde82b00618d h1:m2LqNBUncEXeU github.com/hanfei1991/parser v0.0.0-20200506102604-dde82b00618d/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/parser v0.0.0-20200509083937-95b513acb77d h1:28vZByxR413Q0JIqNu/cPQVkj+yfqEYnTdo1bg21y6U= github.com/hanfei1991/parser v0.0.0-20200509083937-95b513acb77d/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/hanfei1991/parser v0.0.0-20200511041337-73e0eaaee304 h1:LhbAIpiJiKgXXkyqgFLftzFPZTGYLPYcw/Bn5fc2DRs= +github.com/hanfei1991/parser v0.0.0-20200511041337-73e0eaaee304/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/hanfei1991/parser v0.0.0-20200511041744-9942139265ca h1:tk247eW129q6SXiwvDp8aZS+ZgEUhSLTSVjitNafqtM= +github.com/hanfei1991/parser v0.0.0-20200511041744-9942139265ca/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/tipb v0.0.0-20200506100205-b17edac4757f h1:zO5fnD8Wmce+3D9ukaP/2dUL7pgGNVrfiLWwx6kdMYs= github.com/hanfei1991/tipb v0.0.0-20200506100205-b17edac4757f/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= @@ -234,6 +238,14 @@ github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/ichn-hu/parser v0.0.0-20200508082827-d61002d0e55e h1:nOHi37tRE20bNcJlGHFcDEY9hSHNmfrUCd6wfz5q3Gs= +github.com/ichn-hu/parser v0.0.0-20200508082827-d61002d0e55e/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/ichn-hu/parser v0.0.0-20200509094229-22ef1fc04ea4 h1:u3DZVvdncsK1VRPMBAUzlZNcBNmDj46MXL2e+9++ORo= +github.com/ichn-hu/parser v0.0.0-20200509094229-22ef1fc04ea4/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/ichn-hu/parser v0.0.0-20200510024028-6a681e8c9110 h1:ijs3GcuNutc7pAU9LwQ0ctYyMh0oApeK3AqlXIsFjXQ= +github.com/ichn-hu/parser v0.0.0-20200510024028-6a681e8c9110/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/ichn-hu/parser v0.0.0-20200511041337-73e0eaaee304 h1:JRc6f6u7gkevPFKFQEjsjMgsZDMCLDN22siag4WE/pU= +github.com/ichn-hu/parser v0.0.0-20200511041337-73e0eaaee304/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/jinzhu/gorm v1.9.12/go.mod h1:vhTjlKSJUTWNtcbQtrMBFCxy7eXTzeCAzfL5fBZT/Qs= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 20f88a1fbe3e8..822affc6a6f38 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1509,8 +1509,12 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop *property.PhysicalProperty) []P preferredBuildIndex = 1 } preferredGlobalIndex := preferredBuildIndex - if prop.TaskTp != property.CopTiFlashGlobalReadTaskType && getAllDataSourceTotalRowSize(p.children[preferredGlobalIndex]) > getAllDataSourceTotalRowSize(p.children[1-preferredGlobalIndex]) { - preferredGlobalIndex = 1 - preferredGlobalIndex + if prop.TaskTp != property.CopTiFlashGlobalReadTaskType { + if hasPrefer, idx := p.getPreferredBCJLocalIndex(); hasPrefer { + preferredGlobalIndex = 1 - idx + } else if getAllDataSourceTotalRowSize(p.children[preferredGlobalIndex]) > getAllDataSourceTotalRowSize(p.children[1-preferredGlobalIndex]) { + preferredGlobalIndex = 1 - preferredGlobalIndex + } } // todo: currently, build side is the one has less rowcount and global read side // is the one has less datasource row size(which mean less remote read), need diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 77861605bf501..ce0a82bc068b5 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -65,6 +65,8 @@ const ( // HintBCJ indicates applying broadcast join by force. HintBCJ = "bc_join" + // HintBCJPreferLocal specifies the preferred local read table + HintBCJPreferLocal = "bcj_local" // TiDBIndexNestedLoopJoin is hint enforce index nested loop join. TiDBIndexNestedLoopJoin = "tidb_inlj" @@ -467,6 +469,19 @@ func extractTableAlias(p Plan, parentOffset int) *hintTableInfo { return nil } +func (p *LogicalJoin) getPreferredBCJLocalIndex() (hasPrefer bool, prefer int) { + if p.hintInfo == nil { + return + } + if p.hintInfo.ifPreferAsLocalInBCJoin(p.children[0], p.blockOffset) { + return true, 0 + } + if p.hintInfo.ifPreferAsLocalInBCJoin(p.children[1], p.blockOffset) { + return true, 1 + } + return false, 0 +} + func (p *LogicalJoin) setPreferredJoinType(hintInfo *tableHintInfo) { if hintInfo == nil { return @@ -2272,11 +2287,11 @@ func (b *PlanBuilder) pushHintWithoutTableWarning(hint *ast.TableOptimizerHint) func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, nodeType utilhint.NodeType, currentLevel int) { hints = b.hintProcessor.GetCurrentStmtHints(hints, nodeType, currentLevel) var ( - sortMergeTables, INLJTables, INLHJTables, INLMJTables, hashJoinTables, BCTables []hintTableInfo - indexHintList, indexMergeHintList []indexHintInfo - tiflashTables, tikvTables []hintTableInfo - aggHints aggHintInfo - timeRangeHint ast.HintTimeRange + sortMergeTables, INLJTables, INLHJTables, INLMJTables, hashJoinTables, BCTables, BCJPreferLocalTables []hintTableInfo + indexHintList, indexMergeHintList []indexHintInfo + tiflashTables, tikvTables []hintTableInfo + aggHints aggHintInfo + timeRangeHint ast.HintTimeRange ) for _, hint := range hints { // Set warning for the hint that requires the table name. @@ -2294,6 +2309,8 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, nodeType u sortMergeTables = append(sortMergeTables, tableNames2HintTableInfo(b.ctx, hint.Tables, b.hintProcessor, nodeType, currentLevel)...) case TiDBBroadCastJoin, HintBCJ: BCTables = append(BCTables, tableNames2HintTableInfo(b.ctx, hint.Tables, b.hintProcessor, nodeType, currentLevel)...) + case HintBCJPreferLocal: + BCJPreferLocalTables = append(BCJPreferLocalTables, tableNames2HintTableInfo(b.ctx, hint.Tables, b.hintProcessor, nodeType, currentLevel)...) case TiDBIndexNestedLoopJoin, HintINLJ: INLJTables = append(INLJTables, tableNames2HintTableInfo(b.ctx, hint.Tables, b.hintProcessor, nodeType, currentLevel)...) case HintINLHJ: @@ -2364,16 +2381,17 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, nodeType u } } b.tableHintInfo = append(b.tableHintInfo, tableHintInfo{ - sortMergeJoinTables: sortMergeTables, - broadcastJoinTables: BCTables, - indexNestedLoopJoinTables: indexNestedLoopJoinTables{INLJTables, INLHJTables, INLMJTables}, - hashJoinTables: hashJoinTables, - indexHintList: indexHintList, - tiflashTables: tiflashTables, - tikvTables: tikvTables, - aggHints: aggHints, - indexMergeHintList: indexMergeHintList, - timeRangeHint: timeRangeHint, + sortMergeJoinTables: sortMergeTables, + broadcastJoinTables: BCTables, + broadcastJoinPreferredLocal: BCJPreferLocalTables, + indexNestedLoopJoinTables: indexNestedLoopJoinTables{INLJTables, INLHJTables, INLMJTables}, + hashJoinTables: hashJoinTables, + indexHintList: indexHintList, + tiflashTables: tiflashTables, + tikvTables: tikvTables, + aggHints: aggHints, + indexMergeHintList: indexMergeHintList, + timeRangeHint: timeRangeHint, }) } @@ -2386,6 +2404,7 @@ func (b *PlanBuilder) popTableHints() { b.appendUnmatchedJoinHintWarning(HintINLMJ, "", hintInfo.indexNestedLoopJoinTables.inlmjTables) b.appendUnmatchedJoinHintWarning(HintSMJ, TiDBMergeJoin, hintInfo.sortMergeJoinTables) b.appendUnmatchedJoinHintWarning(HintBCJ, TiDBBroadCastJoin, hintInfo.broadcastJoinTables) + b.appendUnmatchedJoinHintWarning(HintBCJPreferLocal, "", hintInfo.broadcastJoinPreferredLocal) b.appendUnmatchedJoinHintWarning(HintHJ, TiDBHashJoin, hintInfo.hashJoinTables) b.appendUnmatchedStorageHintWarning(hintInfo.tiflashTables, hintInfo.tikvTables) b.tableHintInfo = b.tableHintInfo[:len(b.tableHintInfo)-1] diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 40241a2ef5f6f..db0496a070ccc 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -71,15 +71,16 @@ type indexNestedLoopJoinTables struct { type tableHintInfo struct { indexNestedLoopJoinTables - sortMergeJoinTables []hintTableInfo - broadcastJoinTables []hintTableInfo - hashJoinTables []hintTableInfo - indexHintList []indexHintInfo - tiflashTables []hintTableInfo - tikvTables []hintTableInfo - aggHints aggHintInfo - indexMergeHintList []indexHintInfo - timeRangeHint ast.HintTimeRange + sortMergeJoinTables []hintTableInfo + broadcastJoinTables []hintTableInfo + broadcastJoinPreferredLocal []hintTableInfo + hashJoinTables []hintTableInfo + indexHintList []indexHintInfo + tiflashTables []hintTableInfo + tikvTables []hintTableInfo + aggHints aggHintInfo + indexMergeHintList []indexHintInfo + timeRangeHint ast.HintTimeRange } type hintTableInfo struct { @@ -157,6 +158,22 @@ func tableNames2HintTableInfo(ctx sessionctx.Context, hintTables []ast.HintTable return hintTableInfos } +// ifPreferAsLocalInBCJoin checks if there is a data source specified as local read by hint +func (info *tableHintInfo) ifPreferAsLocalInBCJoin(p LogicalPlan, blockOffset int) bool { + alias := extractTableAlias(p, blockOffset) + if alias != nil { + tableNames := make([]*hintTableInfo, 1) + tableNames[0] = alias + return info.matchTableName(tableNames, info.broadcastJoinPreferredLocal) + } + for _, c := range p.Children() { + if info.ifPreferAsLocalInBCJoin(c, blockOffset) { + return true + } + } + return false +} + func (info *tableHintInfo) ifPreferMergeJoin(tableNames ...*hintTableInfo) bool { return info.matchTableName(tableNames, info.sortMergeJoinTables) } diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 0f949e23690af..ab63804168776 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -10,7 +10,9 @@ "name": "TestBroadcastJoin", "cases": [ "explain select /*+ tidb_bcj(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k" + "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", + "explain select /*+ tidb_bcj(fact_t,d1_t), bcj_local(d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t), bcj_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k" ] }, { diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 2428560505ad6..864fae552070c 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -54,6 +54,38 @@ " └─Selection_22(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", " └─TableFullScan_21 8.00 cop[tiflash] table:fact_t keep order:false" ] + }, + { + "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t), bcj_local(d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "Plan": [ + "StreamAgg_25 1.00 root funcs:count(Column#14)->Column#11", + "└─TableReader_26 1.00 root data:StreamAgg_13", + " └─StreamAgg_13 1.00 cop[tiflash] funcs:count(1)->Column#14", + " └─TypeBroadcastJoin_24 8.00 cop[tiflash] ", + " ├─Selection_18(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan_17 2.00 cop[tiflash] table:d1_t keep order:false", + " └─Selection_16(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan_15 8.00 cop[tiflash] table:fact_t keep order:false, global read" + ] + }, + { + "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t), bcj_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", + "Plan": [ + "StreamAgg_35 1.00 root funcs:count(Column#20)->Column#17", + "└─TableReader_36 1.00 root data:StreamAgg_17", + " └─StreamAgg_17 1.00 cop[tiflash] funcs:count(1)->Column#20", + " └─TypeBroadcastJoin_34 8.00 cop[tiflash] ", + " ├─Selection_28(Build) 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan_27 2.00 cop[tiflash] table:d3_t keep order:false, global read", + " └─TypeBroadcastJoin_19(Probe) 8.00 cop[tiflash] ", + " ├─Selection_26(Build) 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan_25 2.00 cop[tiflash] table:d2_t keep order:false", + " └─TypeBroadcastJoin_20(Probe) 8.00 cop[tiflash] ", + " ├─Selection_24(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan_23 2.00 cop[tiflash] table:d1_t keep order:false, global read", + " └─Selection_22(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan_21 8.00 cop[tiflash] table:fact_t keep order:false, global read" + ] } ] }, From 00f2da1619319a8ca6f1f9d6fa0165b6dee7ef57 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Wed, 13 May 2020 16:43:53 +0800 Subject: [PATCH 48/61] refine planner --- planner/core/exhaust_physical_plans.go | 34 +++------- planner/core/task.go | 6 +- .../integration_serial_suite_out.json | 62 +++++++++---------- sessionctx/variable/session.go | 5 ++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 3 + sessionctx/variable/varsutil.go | 1 + sessionctx/variable/varsutil_test.go | 9 +++ 8 files changed, 61 insertions(+), 60 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 822affc6a6f38..b4c777a20c638 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1469,19 +1469,6 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P return joins, true } -func getAllDataSourceTotalRowSize(plan LogicalPlan) float64 { - if ds, ok := plan.(*DataSource); ok { - rowCount := ds.statsInfo().Count() - rowSize := ds.TblColHists.GetTableAvgRowSize(ds.ctx, ds.schema.Columns, kv.StoreType(ds.preferStoreType), ds.handleCol != nil) - return float64(rowCount) * rowSize - } - ret := float64(0) - for _, child := range plan.Children() { - ret += getAllDataSourceTotalRowSize(child) - } - return ret -} - func (p *LogicalJoin) tryToGetBroadCastJoin(prop *property.PhysicalProperty) []PhysicalPlan { if !prop.IsEmpty() { return nil @@ -1494,6 +1481,15 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop *property.PhysicalProperty) []P return nil } + if hasPrefer, idx := p.getPreferredBCJLocalIndex(); hasPrefer { + return p.tryToGetBroadCastJoinByPreferGlobalIdx(prop, 1-idx) + } + results := p.tryToGetBroadCastJoinByPreferGlobalIdx(prop, 0) + results = append(results, p.tryToGetBroadCastJoinByPreferGlobalIdx(prop, 1)...) + return results +} + +func (p *LogicalJoin) tryToGetBroadCastJoinByPreferGlobalIdx(prop *property.PhysicalProperty, preferredGlobalIndex int) []PhysicalPlan { lkeys, rkeys := p.GetJoinKeys() baseJoin := basePhysicalJoin{ JoinType: p.JoinType, @@ -1508,18 +1504,6 @@ func (p *LogicalJoin) tryToGetBroadCastJoin(prop *property.PhysicalProperty) []P if p.children[0].statsInfo().Count() > p.children[1].statsInfo().Count() { preferredBuildIndex = 1 } - preferredGlobalIndex := preferredBuildIndex - if prop.TaskTp != property.CopTiFlashGlobalReadTaskType { - if hasPrefer, idx := p.getPreferredBCJLocalIndex(); hasPrefer { - preferredGlobalIndex = 1 - idx - } else if getAllDataSourceTotalRowSize(p.children[preferredGlobalIndex]) > getAllDataSourceTotalRowSize(p.children[1-preferredGlobalIndex]) { - preferredGlobalIndex = 1 - preferredGlobalIndex - } - } - // todo: currently, build side is the one has less rowcount and global read side - // is the one has less datasource row size(which mean less remote read), need - // to use cbo to decide the build side and global read side if preferred build index - // is not equal to preferred global index baseJoin.InnerChildIdx = preferredBuildIndex childrenReqProps := make([]*property.PhysicalProperty, 2) childrenReqProps[preferredGlobalIndex] = &property.PhysicalProperty{TaskTp: property.CopTiFlashGlobalReadTaskType, ExpectedCnt: math.MaxFloat64} diff --git a/planner/core/task.go b/planner/core/task.go index 27b3b249d6380..6b86157fedc5c 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -552,10 +552,8 @@ func (p *PhysicalBroadCastJoin) GetCost(lCnt, rCnt float64) float64 { } numPairs := helper.estimate() probeCost := numPairs * sessVars.CopCPUFactor - // should divided by the cop concurrency, which is decide by TiFlash, but TiDB - // can not get the information from TiFlash, so just use `sessVars.HashJoinConcurrency` - // as a workaround - probeCost /= float64(sessVars.HashJoinConcurrency) + // should divided by the concurrency in tiflash, which should be the number of core in tiflash nodes. + probeCost /= float64(sessVars.CopTiFlashConcurrencyFactor) cpuCost += probeCost // todo since TiFlash join is significant faster than TiDB join, maybe diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 864fae552070c..1878ec443b07e 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -26,33 +26,33 @@ { "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ - "StreamAgg_25 1.00 root funcs:count(Column#14)->Column#11", - "└─TableReader_26 1.00 root data:StreamAgg_13", + "StreamAgg_32 1.00 root funcs:count(Column#14)->Column#11", + "└─TableReader_33 1.00 root data:StreamAgg_13", " └─StreamAgg_13 1.00 cop[tiflash] funcs:count(1)->Column#14", - " └─TypeBroadcastJoin_24 8.00 cop[tiflash] ", - " ├─Selection_18(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan_17 2.00 cop[tiflash] table:d1_t keep order:false, global read", - " └─Selection_16(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan_15 8.00 cop[tiflash] table:fact_t keep order:false" + " └─TypeBroadcastJoin_31 8.00 cop[tiflash] ", + " ├─Selection_23(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan_22 2.00 cop[tiflash] table:d1_t keep order:false, global read", + " └─Selection_21(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan_20 8.00 cop[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", "Plan": [ - "StreamAgg_35 1.00 root funcs:count(Column#20)->Column#17", - "└─TableReader_36 1.00 root data:StreamAgg_17", + "StreamAgg_52 1.00 root funcs:count(Column#20)->Column#17", + "└─TableReader_53 1.00 root data:StreamAgg_17", " └─StreamAgg_17 1.00 cop[tiflash] funcs:count(1)->Column#20", - " └─TypeBroadcastJoin_34 8.00 cop[tiflash] ", - " ├─Selection_28(Build) 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan_27 2.00 cop[tiflash] table:d3_t keep order:false, global read", - " └─TypeBroadcastJoin_19(Probe) 8.00 cop[tiflash] ", - " ├─Selection_26(Build) 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan_25 2.00 cop[tiflash] table:d2_t keep order:false, global read", - " └─TypeBroadcastJoin_20(Probe) 8.00 cop[tiflash] ", - " ├─Selection_24(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan_23 2.00 cop[tiflash] table:d1_t keep order:false, global read", - " └─Selection_22(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan_21 8.00 cop[tiflash] table:fact_t keep order:false" + " └─TypeBroadcastJoin_51 8.00 cop[tiflash] ", + " ├─Selection_43(Build) 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan_42 2.00 cop[tiflash] table:d3_t keep order:false, global read", + " └─TypeBroadcastJoin_33(Probe) 8.00 cop[tiflash] ", + " ├─Selection_29(Build) 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan_28 2.00 cop[tiflash] table:d2_t keep order:false, global read", + " └─TypeBroadcastJoin_37(Probe) 8.00 cop[tiflash] ", + " ├─Selection_27(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan_26 2.00 cop[tiflash] table:d1_t keep order:false, global read", + " └─Selection_41(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan_40 8.00 cop[tiflash] table:fact_t keep order:false" ] }, { @@ -71,20 +71,20 @@ { "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t), bcj_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", "Plan": [ - "StreamAgg_35 1.00 root funcs:count(Column#20)->Column#17", - "└─TableReader_36 1.00 root data:StreamAgg_17", + "StreamAgg_36 1.00 root funcs:count(Column#20)->Column#17", + "└─TableReader_37 1.00 root data:StreamAgg_17", " └─StreamAgg_17 1.00 cop[tiflash] funcs:count(1)->Column#20", - " └─TypeBroadcastJoin_34 8.00 cop[tiflash] ", - " ├─Selection_28(Build) 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan_27 2.00 cop[tiflash] table:d3_t keep order:false, global read", + " └─TypeBroadcastJoin_35 8.00 cop[tiflash] ", + " ├─Selection_29(Build) 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan_28 2.00 cop[tiflash] table:d3_t keep order:false, global read", " └─TypeBroadcastJoin_19(Probe) 8.00 cop[tiflash] ", - " ├─Selection_26(Build) 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan_25 2.00 cop[tiflash] table:d2_t keep order:false", + " ├─Selection_27(Build) 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan_26 2.00 cop[tiflash] table:d2_t keep order:false", " └─TypeBroadcastJoin_20(Probe) 8.00 cop[tiflash] ", - " ├─Selection_24(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan_23 2.00 cop[tiflash] table:d1_t keep order:false, global read", - " └─Selection_22(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan_21 8.00 cop[tiflash] table:fact_t keep order:false, global read" + " ├─Selection_25(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan_24 2.00 cop[tiflash] table:d1_t keep order:false, global read", + " └─Selection_23(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan_22 8.00 cop[tiflash] table:fact_t keep order:false, global read" ] } ] diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 187393d11e553..639c0efa08e98 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -407,6 +407,8 @@ type SessionVars struct { CPUFactor float64 // CopCPUFactor is the CPU cost of processing one expression for one row in coprocessor. CopCPUFactor float64 + // CopTiFlashConcurrencyFactor is the concurrency number of computation in tiflash coprocessor. + CopTiFlashConcurrencyFactor float64 // NetworkFactor is the network cost of transferring 1 byte data. NetworkFactor float64 // ScanFactor is the IO cost of scanning 1 byte data on TiKV and TiFlash. @@ -663,6 +665,7 @@ func NewSessionVars() *SessionVars { CorrelationExpFactor: DefOptCorrelationExpFactor, CPUFactor: DefOptCPUFactor, CopCPUFactor: DefOptCopCPUFactor, + CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, NetworkFactor: DefOptNetworkFactor, ScanFactor: DefOptScanFactor, DescScanFactor: DefOptDescScanFactor, @@ -1077,6 +1080,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.CPUFactor = tidbOptFloat64(val, DefOptCPUFactor) case TiDBOptCopCPUFactor: s.CopCPUFactor = tidbOptFloat64(val, DefOptCopCPUFactor) + case TiDBOptTiFlashConcurrencyFactor: + s.CopTiFlashConcurrencyFactor = tidbOptFloat64(val, DefOptTiFlashConcurrencyFactor) case TiDBOptNetworkFactor: s.NetworkFactor = tidbOptFloat64(val, DefOptNetworkFactor) case TiDBOptScanFactor: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index bf9bf549f84d1..2686e09b46503 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -623,6 +623,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal | ScopeSession, TiDBOptCorrelationThreshold, strconv.FormatFloat(DefOptCorrelationThreshold, 'f', -1, 64)}, {ScopeGlobal | ScopeSession, TiDBOptCorrelationExpFactor, strconv.Itoa(DefOptCorrelationExpFactor)}, {ScopeGlobal | ScopeSession, TiDBOptCPUFactor, strconv.FormatFloat(DefOptCPUFactor, 'f', -1, 64)}, + {ScopeGlobal | ScopeSession, TiDBOptTiFlashConcurrencyFactor, strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64)}, {ScopeGlobal | ScopeSession, TiDBOptCopCPUFactor, strconv.FormatFloat(DefOptCopCPUFactor, 'f', -1, 64)}, {ScopeGlobal | ScopeSession, TiDBOptNetworkFactor, strconv.FormatFloat(DefOptNetworkFactor, 'f', -1, 64)}, {ScopeGlobal | ScopeSession, TiDBOptScanFactor, strconv.FormatFloat(DefOptScanFactor, 'f', -1, 64)}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 1b4a67f563b4c..9024c3e8b8f13 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -199,6 +199,8 @@ const ( TiDBOptCPUFactor = "tidb_opt_cpu_factor" // tidb_opt_copcpu_factor is the CPU cost of processing one expression for one row in coprocessor. TiDBOptCopCPUFactor = "tidb_opt_copcpu_factor" + // tidb_opt_copcpu_factor is the CPU cost of processing one expression for one row in coprocessor. + TiDBOptTiFlashConcurrencyFactor = "tidb_opt_tiflash_concurrency_factor" // tidb_opt_network_factor is the network cost of transferring 1 byte data. TiDBOptNetworkFactor = "tidb_opt_network_factor" // tidb_opt_scan_factor is the IO cost of scanning 1 byte data on TiKV. @@ -420,6 +422,7 @@ const ( DefOptCorrelationExpFactor = 1 DefOptCPUFactor = 3.0 DefOptCopCPUFactor = 3.0 + DefOptTiFlashConcurrencyFactor = 24.0 DefOptNetworkFactor = 1.0 DefOptScanFactor = 1.5 DefOptDescScanFactor = 3.0 diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 582f60e4f07a0..fdde558b2eeff 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -546,6 +546,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc } return value, nil case TiDBOptCPUFactor, + TiDBOptTiFlashConcurrencyFactor, TiDBOptCopCPUFactor, TiDBOptNetworkFactor, TiDBOptScanFactor, diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 145ff229890b4..71a628dc43f54 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -325,6 +325,14 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(val, Equals, "5.0") c.Assert(v.CopCPUFactor, Equals, 5.0) + c.Assert(v.CopTiFlashConcurrencyFactor, Equals, 24.0) + err = SetSessionSystemVar(v, TiDBOptTiFlashConcurrencyFactor, types.NewStringDatum("5.0")) + c.Assert(err, IsNil) + val, err = GetSessionSystemVar(v, TiDBOptTiFlashConcurrencyFactor) + c.Assert(err, IsNil) + c.Assert(val, Equals, "5.0") + c.Assert(v.CopCPUFactor, Equals, 5.0) + c.Assert(v.NetworkFactor, Equals, 1.0) err = SetSessionSystemVar(v, TiDBOptNetworkFactor, types.NewStringDatum("3.0")) c.Assert(err, IsNil) @@ -515,6 +523,7 @@ func (s *testVarsutilSuite) TestValidate(c *C) { {TiDBOptCorrelationThreshold, "-2", true}, {TiDBOptCPUFactor, "a", true}, {TiDBOptCPUFactor, "-2", true}, + {TiDBOptTiFlashConcurrencyFactor, "-2", true}, {TiDBOptCopCPUFactor, "a", true}, {TiDBOptCopCPUFactor, "-2", true}, {TiDBOptNetworkFactor, "a", true}, From 1ce94eeff3588d9b6c36f79492b68672ebea23e8 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Wed, 13 May 2020 16:48:14 +0800 Subject: [PATCH 49/61] refine comments --- sessionctx/variable/tidb_vars.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 9024c3e8b8f13..6b401f2647f52 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -199,7 +199,7 @@ const ( TiDBOptCPUFactor = "tidb_opt_cpu_factor" // tidb_opt_copcpu_factor is the CPU cost of processing one expression for one row in coprocessor. TiDBOptCopCPUFactor = "tidb_opt_copcpu_factor" - // tidb_opt_copcpu_factor is the CPU cost of processing one expression for one row in coprocessor. + // tidb_opt_tiflash_concurrency_factor is concurrency number of tiflash computation. TiDBOptTiFlashConcurrencyFactor = "tidb_opt_tiflash_concurrency_factor" // tidb_opt_network_factor is the network cost of transferring 1 byte data. TiDBOptNetworkFactor = "tidb_opt_network_factor" From 94ece030212b6ef7846fc69713a832f602f04c53 Mon Sep 17 00:00:00 2001 From: Fei Han Date: Sun, 17 May 2020 14:48:38 +0800 Subject: [PATCH 50/61] fix make dev --- go.mod | 1 + go.sum | 26 +++---------------------- planner/core/plan_to_pb.go | 40 +++++++++++++++++++------------------- 3 files changed, 24 insertions(+), 43 deletions(-) diff --git a/go.mod b/go.mod index 46e76e67f0053..5fbc858346638 100644 --- a/go.mod +++ b/go.mod @@ -61,6 +61,7 @@ require ( golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1 google.golang.org/grpc v1.25.1 gopkg.in/natefinch/lumberjack.v2 v2.0.0 + honnef.co/go/tools v0.0.1-2020.1.4 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) diff --git a/go.sum b/go.sum index 7490db8b84632..7b2bc0cc0934f 100644 --- a/go.sum +++ b/go.sum @@ -219,16 +219,6 @@ github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqC github.com/grpc-ecosystem/grpc-gateway v1.14.3 h1:OCJlWkOUoTnl0neNGlf4fUm3TmbEtguw7vR+nGtnDjY= github.com/grpc-ecosystem/grpc-gateway v1.14.3/go.mod h1:6CwZWGDSPRJidgKAtJVvND6soZe6fT7iteq8wDPdhb0= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= -github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c h1:akJYnz7Fe6AXtCTcZai3gf+HrZxuoVbnaGXxEmtxvpo= -github.com/hanfei1991/parser v0.0.0-20200420101937-63e37f36ae0c/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/hanfei1991/parser v0.0.0-20200506100705-497ab5ecbc16 h1:zlhQFwi+d+tkwFvVr0Wma3mpXVizCOP1e6F04DFVHiA= -github.com/hanfei1991/parser v0.0.0-20200506100705-497ab5ecbc16/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/hanfei1991/parser v0.0.0-20200506102604-dde82b00618d h1:m2LqNBUncEXeUo44rcIlrKv0v5fbvBUihcidWDDdg2E= -github.com/hanfei1991/parser v0.0.0-20200506102604-dde82b00618d/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/hanfei1991/parser v0.0.0-20200509083937-95b513acb77d h1:28vZByxR413Q0JIqNu/cPQVkj+yfqEYnTdo1bg21y6U= -github.com/hanfei1991/parser v0.0.0-20200509083937-95b513acb77d/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/hanfei1991/parser v0.0.0-20200511041337-73e0eaaee304 h1:LhbAIpiJiKgXXkyqgFLftzFPZTGYLPYcw/Bn5fc2DRs= -github.com/hanfei1991/parser v0.0.0-20200511041337-73e0eaaee304/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/parser v0.0.0-20200511041744-9942139265ca h1:tk247eW129q6SXiwvDp8aZS+ZgEUhSLTSVjitNafqtM= github.com/hanfei1991/parser v0.0.0-20200511041744-9942139265ca/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/tipb v0.0.0-20200506100205-b17edac4757f h1:zO5fnD8Wmce+3D9ukaP/2dUL7pgGNVrfiLWwx6kdMYs= @@ -238,14 +228,6 @@ github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= -github.com/ichn-hu/parser v0.0.0-20200508082827-d61002d0e55e h1:nOHi37tRE20bNcJlGHFcDEY9hSHNmfrUCd6wfz5q3Gs= -github.com/ichn-hu/parser v0.0.0-20200508082827-d61002d0e55e/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/ichn-hu/parser v0.0.0-20200509094229-22ef1fc04ea4 h1:u3DZVvdncsK1VRPMBAUzlZNcBNmDj46MXL2e+9++ORo= -github.com/ichn-hu/parser v0.0.0-20200509094229-22ef1fc04ea4/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/ichn-hu/parser v0.0.0-20200510024028-6a681e8c9110 h1:ijs3GcuNutc7pAU9LwQ0ctYyMh0oApeK3AqlXIsFjXQ= -github.com/ichn-hu/parser v0.0.0-20200510024028-6a681e8c9110/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/ichn-hu/parser v0.0.0-20200511041337-73e0eaaee304 h1:JRc6f6u7gkevPFKFQEjsjMgsZDMCLDN22siag4WE/pU= -github.com/ichn-hu/parser v0.0.0-20200511041337-73e0eaaee304/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/jinzhu/gorm v1.9.12/go.mod h1:vhTjlKSJUTWNtcbQtrMBFCxy7eXTzeCAzfL5fBZT/Qs= @@ -388,9 +370,6 @@ github.com/pingcap/kvproto v0.0.0-20200428135407-0f5ffe459677/go.mod h1:IOdRDPLy github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd h1:CV3VsP3Z02MVtdpTMfEgRJ4T9NGgGTxdHpJerent7rM= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20200424075042-8222d8b724a4/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657 h1:2ceTso30kmgMeddZ4iZ6zrK8N9eFF8zmCa1hSSE1tXc= -github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 h1:JTzYYukREvxVSKW/ncrzNjFitd8snoQ/Xz32pw8i+s8= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= @@ -401,8 +380,6 @@ github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incomp github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible h1:+K5bqDYG5HT+GqLdx4GH5VmS84+xHgpHbGg6Xt6qQec= github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= -github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee h1:XJQ6/LGzOSc/jo33AD8t7jtc4GohxcyODsYnb+kZXJM= -github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -430,6 +407,7 @@ github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= +github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v2.19.10+incompatible h1:lA4Pi29JEVIQIgATSeftHSY0rMGI9CLrl2ZvDLiahto= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= @@ -747,6 +725,8 @@ honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3 h1:sXmLre5bzIR6ypkjXCDI3jHPssRhc8KD/Ome589sc3U= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.0.1-2020.1.4 h1:UoveltGrhghAA7ePc+e+QYDHXrBps2PqFZiHkGR/xK8= +honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 7eaa161652ba2..1898cfc5370f2 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -49,16 +49,16 @@ func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) ( for _, aggFunc := range p.AggFuncs { aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc)) } - executorId := "" + executorID := "" if storeType == kv.TiFlash { var err error aggExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } - executorId = p.ExplainID().String() + executorID = p.ExplainID().String() } - return &tipb.Executor{Tp: tipb.ExecType_TypeAggregation, Aggregation: aggExec, ExecutorId: &executorId}, nil + return &tipb.Executor{Tp: tipb.ExecType_TypeAggregation, Aggregation: aggExec, ExecutorId: &executorID}, nil } // ToPB implements PhysicalPlan ToPB interface. @@ -75,16 +75,16 @@ func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context, storeType kv.StoreType) for _, aggFunc := range p.AggFuncs { aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc)) } - executorId := "" + executorID := "" if storeType == kv.TiFlash { var err error aggExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } - executorId = p.ExplainID().String() + executorID = p.ExplainID().String() } - return &tipb.Executor{Tp: tipb.ExecType_TypeStreamAgg, Aggregation: aggExec, ExecutorId: &executorId}, nil + return &tipb.Executor{Tp: tipb.ExecType_TypeStreamAgg, Aggregation: aggExec, ExecutorId: &executorID}, nil } // ToPB implements PhysicalPlan ToPB interface. @@ -98,16 +98,16 @@ func (p *PhysicalSelection) ToPB(ctx sessionctx.Context, storeType kv.StoreType) selExec := &tipb.Selection{ Conditions: conditions, } - executorId := "" + executorID := "" if storeType == kv.TiFlash { var err error selExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } - executorId = p.ExplainID().String() + executorID = p.ExplainID().String() } - return &tipb.Executor{Tp: tipb.ExecType_TypeSelection, Selection: selExec, ExecutorId: &executorId}, nil + return &tipb.Executor{Tp: tipb.ExecType_TypeSelection, Selection: selExec, ExecutorId: &executorID}, nil } // ToPB implements PhysicalPlan ToPB interface. @@ -120,16 +120,16 @@ func (p *PhysicalTopN) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*ti for _, item := range p.ByItems { topNExec.OrderBy = append(topNExec.OrderBy, expression.SortByItemToPB(sc, client, item.Expr, item.Desc)) } - executorId := "" + executorID := "" if storeType == kv.TiFlash { var err error topNExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } - executorId = p.ExplainID().String() + executorID = p.ExplainID().String() } - return &tipb.Executor{Tp: tipb.ExecType_TypeTopN, TopN: topNExec, ExecutorId: &executorId}, nil + return &tipb.Executor{Tp: tipb.ExecType_TypeTopN, TopN: topNExec, ExecutorId: &executorID}, nil } // ToPB implements PhysicalPlan ToPB interface. @@ -137,16 +137,16 @@ func (p *PhysicalLimit) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*t limitExec := &tipb.Limit{ Limit: p.Count, } - executorId := "" + executorID := "" if storeType == kv.TiFlash { var err error limitExec.Child, err = p.children[0].ToPB(ctx, storeType) if err != nil { return nil, errors.Trace(err) } - executorId = p.ExplainID().String() + executorID = p.ExplainID().String() } - return &tipb.Executor{Tp: tipb.ExecType_TypeLimit, Limit: limitExec, ExecutorId: &executorId}, nil + return &tipb.Executor{Tp: tipb.ExecType_TypeLimit, Limit: limitExec, ExecutorId: &executorID}, nil } // ToPB implements PhysicalPlan ToPB interface. @@ -159,7 +159,7 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) if p.isPartition { tsExec.TableId = p.physicalTableID } - executorId := "" + executorID := "" if storeType == kv.TiFlash && p.IsGlobalRead { tsExec.NextReadEngine = tipb.EngineType_TiFlash ranges := distsql.TableRangesToKVRanges(tsExec.TableId, p.Ranges, nil) @@ -169,10 +169,10 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) logutil.BgLogger().Info("make range for table.") } if storeType == kv.TiFlash { - executorId = p.ExplainID().String() + executorID = p.ExplainID().String() } err := SetPBColumnsDefaultValue(ctx, tsExec.Columns, p.Columns) - return &tipb.Executor{Tp: tipb.ExecType_TypeTableScan, TblScan: tsExec, ExecutorId: &executorId}, err + return &tipb.Executor{Tp: tipb.ExecType_TypeTableScan, TblScan: tsExec, ExecutorId: &executorID}, err } // checkCoverIndex checks whether we can pass unique info to TiKV. We should push it if and only if the length of @@ -262,8 +262,8 @@ func (p *PhysicalBroadCastJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreT Children: []*tipb.Executor{lChildren, rChildren}, } - executorId := p.ExplainID().String() - return &tipb.Executor{Tp: tipb.ExecType_TypeJoin, Join: join, ExecutorId: &executorId}, nil + executorID := p.ExplainID().String() + return &tipb.Executor{Tp: tipb.ExecType_TypeJoin, Join: join, ExecutorId: &executorID}, nil } // SetPBColumnsDefaultValue sets the default values of tipb.ColumnInfos. From 691acc52e1118772f53bba89c837b41831874356 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 15 Jun 2020 15:28:38 +0800 Subject: [PATCH 51/61] fix make dev --- go.sum | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/go.sum b/go.sum index b92a6970bc351..9bdc29da2f1e9 100644 --- a/go.sum +++ b/go.sum @@ -260,14 +260,8 @@ github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqC github.com/grpc-ecosystem/grpc-gateway v1.14.3 h1:OCJlWkOUoTnl0neNGlf4fUm3TmbEtguw7vR+nGtnDjY= github.com/grpc-ecosystem/grpc-gateway v1.14.3/go.mod h1:6CwZWGDSPRJidgKAtJVvND6soZe6fT7iteq8wDPdhb0= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= -github.com/hanfei1991/parser v0.0.0-20200511041744-9942139265ca h1:tk247eW129q6SXiwvDp8aZS+ZgEUhSLTSVjitNafqtM= -github.com/hanfei1991/parser v0.0.0-20200511041744-9942139265ca/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= github.com/hanfei1991/parser v0.0.0-20200615062953-980888fe15f7 h1:RdGUdtR0kzQjnozAalqbZ8DBqKybHzgIvPoOBw1oxlM= github.com/hanfei1991/parser v0.0.0-20200615062953-980888fe15f7/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/hanfei1991/tipb v0.0.0-20200506100205-b17edac4757f h1:zO5fnD8Wmce+3D9ukaP/2dUL7pgGNVrfiLWwx6kdMYs= -github.com/hanfei1991/tipb v0.0.0-20200506100205-b17edac4757f/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= -github.com/hanfei1991/tipb v0.0.0-20200528072723-b2224724d610 h1:xpw6HwOSbyc7zM51P5Nmhp1VnFuoR0RG7BNCKkm4lSk= -github.com/hanfei1991/tipb v0.0.0-20200528072723-b2224724d610/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hanfei1991/tipb v0.0.0-20200615064645-bb7c184dfad7 h1:v3AV42scaKrnFDpS8Tl2BKlb2KPe7+qnBiYA3CMIbIo= github.com/hanfei1991/tipb v0.0.0-20200615064645-bb7c184dfad7/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= @@ -461,12 +455,6 @@ github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad h1:SveG82rmu/GFxYanffxsSF503SiQV+2JLnWEiGiF+Tc= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20200424075042-8222d8b724a4/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= -github.com/pingcap/parser v0.0.0-20200518090819-ec1e13b948b1/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200522094936-3b720a0512a6/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/pingcap/parser v0.0.0-20200609110328-c65941b9fbb3 h1:nPE21utCCSq0c0QjoeU1IvMarBvLOfigeaHggx3B07M= -github.com/pingcap/parser v0.0.0-20200609110328-c65941b9fbb3/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 h1:JTzYYukREvxVSKW/ncrzNjFitd8snoQ/Xz32pw8i+s8= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181 h1:FM+PzdoR3fmWAJx3ug+p5aOgs5aZYwFkoDL7Potdsz0= @@ -485,9 +473,6 @@ github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompat github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible h1:/JKsYjsa5Ug8v5CN4zIbJGIqsvgBUkGwaP/rEScVvWM= github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= -github.com/pingcap/tipb v0.0.0-20200610045017-b69a98cfcf6b h1:9t3egBBwrKPrSeJeLTbcSpa5R/UE/lg0VwDe8C180Co= -github.com/pingcap/tipb v0.0.0-20200610045017-b69a98cfcf6b/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= From a750bdca02a1b8ebc4593ee547ff0c768baa759b Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 17 Jun 2020 16:47:55 +0800 Subject: [PATCH 52/61] update parser --- go.mod | 2 +- go.sum | 2 ++ planner/core/integration_test.go | 6 +++--- planner/core/logical_plan_builder.go | 4 ++-- planner/core/testdata/integration_serial_suite_in.json | 8 ++++---- planner/core/testdata/integration_serial_suite_out.json | 8 ++++---- 6 files changed, 16 insertions(+), 14 deletions(-) diff --git a/go.mod b/go.mod index d057bae88ebca..bfc6e897b67be 100644 --- a/go.mod +++ b/go.mod @@ -2,7 +2,7 @@ module github.com/pingcap/tidb replace github.com/pingcap/tipb => github.com/hanfei1991/tipb v0.0.0-20200615064645-bb7c184dfad7 -replace github.com/pingcap/parser => github.com/hanfei1991/parser v0.0.0-20200615062953-980888fe15f7 +replace github.com/pingcap/parser => github.com/hanfei1991/parser v0.0.0-20200617082422-444a49f9c68e require ( github.com/BurntSushi/toml v0.3.1 diff --git a/go.sum b/go.sum index 9bdc29da2f1e9..53be46b7d2865 100644 --- a/go.sum +++ b/go.sum @@ -262,6 +262,8 @@ github.com/grpc-ecosystem/grpc-gateway v1.14.3/go.mod h1:6CwZWGDSPRJidgKAtJVvND6 github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= github.com/hanfei1991/parser v0.0.0-20200615062953-980888fe15f7 h1:RdGUdtR0kzQjnozAalqbZ8DBqKybHzgIvPoOBw1oxlM= github.com/hanfei1991/parser v0.0.0-20200615062953-980888fe15f7/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= +github.com/hanfei1991/parser v0.0.0-20200617082422-444a49f9c68e h1:C5xRGrh4bFK5FRMg/NO06i7+BH31Gcso/tjEcQ5Tg2c= +github.com/hanfei1991/parser v0.0.0-20200617082422-444a49f9c68e/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/hanfei1991/tipb v0.0.0-20200615064645-bb7c184dfad7 h1:v3AV42scaKrnFDpS8Tl2BKlb2KPe7+qnBiYA3CMIbIo= github.com/hanfei1991/tipb v0.0.0-20200615064645-bb7c184dfad7/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index e2c23f9f86bfa..d6434a4055c71 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -400,15 +400,15 @@ func (s *testIntegrationSerialSuite) TestBroadcastJoin(c *C) { } // out join not supported - _, err := tk.Exec("explain select /*+ tidb_bcj(fact_t, d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k") + _, err := tk.Exec("explain select /*+ broadcast_join(fact_t, d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k") c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") // join with non-equal condition not supported - _, err = tk.Exec("explain select /*+ tidb_bcj(fact_t, d1_t) */ count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value") + _, err = tk.Exec("explain select /*+ broadcast_join(fact_t, d1_t) */ count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value") c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") // cartsian join not supported - _, err = tk.Exec("explain select /*+ tidb_bcj(fact_t, d1_t) */ count(*) from fact_t join d1_t") + _, err = tk.Exec("explain select /*+ broadcast_join(fact_t, d1_t) */ count(*) from fact_t join d1_t") c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 5b904769c8c0d..5e7ef309bf4ae 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -64,9 +64,9 @@ const ( TiDBBroadCastJoin = "tidb_bcj" // HintBCJ indicates applying broadcast join by force. - HintBCJ = "bc_join" + HintBCJ = "broadcast_join" // HintBCJPreferLocal specifies the preferred local read table - HintBCJPreferLocal = "bcj_local" + HintBCJPreferLocal = "broadcast_join_local" // TiDBIndexNestedLoopJoin is hint enforce index nested loop join. TiDBIndexNestedLoopJoin = "tidb_inlj" diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 4015dffe22af9..eb17fe5e54308 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -9,10 +9,10 @@ { "name": "TestBroadcastJoin", "cases": [ - "explain select /*+ tidb_bcj(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "explain select /*+ tidb_bcj(fact_t,d1_t), bcj_local(d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t), bcj_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k" + "explain select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "explain select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", + "explain select /*+ broadcast_join(fact_t,d1_t), broadcast_join_local(d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "explain select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t), broadcast_join_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k" ] }, { diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 58c6e427d4433..17b70d52e44b5 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -24,7 +24,7 @@ "Name": "TestBroadcastJoin", "Cases": [ { - "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "SQL": "explain select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ "StreamAgg_32 1.00 root funcs:count(Column#14)->Column#11", "└─TableReader_33 1.00 root data:StreamAgg_13", @@ -37,7 +37,7 @@ ] }, { - "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", + "SQL": "explain select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", "Plan": [ "StreamAgg_52 1.00 root funcs:count(Column#20)->Column#17", "└─TableReader_53 1.00 root data:StreamAgg_17", @@ -56,7 +56,7 @@ ] }, { - "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t), bcj_local(d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "SQL": "explain select /*+ broadcast_join(fact_t,d1_t), broadcast_join_local(d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ "StreamAgg_25 1.00 root funcs:count(Column#14)->Column#11", "└─TableReader_26 1.00 root data:StreamAgg_13", @@ -69,7 +69,7 @@ ] }, { - "SQL": "explain select /*+ tidb_bcj(fact_t,d1_t,d2_t,d3_t), bcj_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", + "SQL": "explain select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t), broadcast_join_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", "Plan": [ "StreamAgg_36 1.00 root funcs:count(Column#20)->Column#17", "└─TableReader_37 1.00 root data:StreamAgg_17", From 0a6b1ca81418e3838bd5d69cd8787e7662ce2f0d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 17 Jun 2020 18:13:59 +0800 Subject: [PATCH 53/61] address comments --- planner/core/plan_to_pb.go | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index ff24bb6d25ce9..17ec08b946707 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" ) @@ -175,7 +174,6 @@ func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) for _, keyRange := range ranges { tsExec.Ranges = append(tsExec.Ranges, tipb.KeyRange{Low: keyRange.StartKey, High: keyRange.EndKey}) } - logutil.BgLogger().Info("make range for table.") } if storeType == kv.TiFlash { executorID = p.ExplainID().String() @@ -262,13 +260,13 @@ func (p *PhysicalBroadCastJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreT return nil, errors.Trace(err) } - left, leftErr := expression.ExpressionsToPBList(sc, leftJoinKeys, client) - if leftErr != nil { - return nil, leftErr + left, err := expression.ExpressionsToPBList(sc, leftJoinKeys, client) + if err != nil { + return nil, err } - right, rightErr := expression.ExpressionsToPBList(sc, rightJoinKeys, client) - if rightErr != nil { - return nil, rightErr + right, err := expression.ExpressionsToPBList(sc, rightJoinKeys, client) + if err != nil { + return nil, err } join := &tipb.Join{ JoinType: tipb.JoinType_TypeInnerJoin, From 635ce4e4beee3a3a456fae0c50bb809cb6ced1b4 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 18 Jun 2020 18:27:05 +0800 Subject: [PATCH 54/61] fix make dev --- go.mod | 8 ++------ go.sum | 16 ++++++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/go.mod b/go.mod index bfc6e897b67be..0963d113f8696 100644 --- a/go.mod +++ b/go.mod @@ -1,9 +1,5 @@ module github.com/pingcap/tidb -replace github.com/pingcap/tipb => github.com/hanfei1991/tipb v0.0.0-20200615064645-bb7c184dfad7 - -replace github.com/pingcap/parser => github.com/hanfei1991/parser v0.0.0-20200617082422-444a49f9c68e - require ( github.com/BurntSushi/toml v0.3.1 github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d @@ -37,11 +33,11 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20200518112156-d4aeb467de29 github.com/pingcap/log v0.0.0-20200511115504-543df19646ad - github.com/pingcap/parser v0.0.0-20200609110328-c65941b9fbb3 + github.com/pingcap/parser v0.0.0-20200617145328-e63134b60279 github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181 github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible - github.com/pingcap/tipb v0.0.0-20200610045017-b69a98cfcf6b + github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3 github.com/prometheus/client_golang v1.5.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.9.1 diff --git a/go.sum b/go.sum index 53be46b7d2865..2f72479dce2ad 100644 --- a/go.sum +++ b/go.sum @@ -260,12 +260,6 @@ github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqC github.com/grpc-ecosystem/grpc-gateway v1.14.3 h1:OCJlWkOUoTnl0neNGlf4fUm3TmbEtguw7vR+nGtnDjY= github.com/grpc-ecosystem/grpc-gateway v1.14.3/go.mod h1:6CwZWGDSPRJidgKAtJVvND6soZe6fT7iteq8wDPdhb0= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= -github.com/hanfei1991/parser v0.0.0-20200615062953-980888fe15f7 h1:RdGUdtR0kzQjnozAalqbZ8DBqKybHzgIvPoOBw1oxlM= -github.com/hanfei1991/parser v0.0.0-20200615062953-980888fe15f7/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/hanfei1991/parser v0.0.0-20200617082422-444a49f9c68e h1:C5xRGrh4bFK5FRMg/NO06i7+BH31Gcso/tjEcQ5Tg2c= -github.com/hanfei1991/parser v0.0.0-20200617082422-444a49f9c68e/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= -github.com/hanfei1991/tipb v0.0.0-20200615064645-bb7c184dfad7 h1:v3AV42scaKrnFDpS8Tl2BKlb2KPe7+qnBiYA3CMIbIo= -github.com/hanfei1991/tipb v0.0.0-20200615064645-bb7c184dfad7/go.mod h1:nPMpOLbGpW71jAalpqgYYjMAZfCGk+DdSPhoIVVizkU= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= @@ -457,6 +451,12 @@ github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad h1:SveG82rmu/GFxYanffxsSF503SiQV+2JLnWEiGiF+Tc= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= +github.com/pingcap/parser v0.0.0-20200424075042-8222d8b724a4/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/pingcap/parser v0.0.0-20200507022230-f3bf29096657/go.mod h1:9v0Edh8IbgjGYW2ArJr19E+bvL8zKahsFp+ixWeId+4= +github.com/pingcap/parser v0.0.0-20200518090819-ec1e13b948b1/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= +github.com/pingcap/parser v0.0.0-20200522094936-3b720a0512a6/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= +github.com/pingcap/parser v0.0.0-20200617145328-e63134b60279 h1:vFUmWKPZl8M48s2o2MMfmB1+aj/oxwyDMZyhctPspLk= +github.com/pingcap/parser v0.0.0-20200617145328-e63134b60279/go.mod h1:vQdbJqobJAgFyiRNNtXahpMoGWwPEuWciVEK5A20NS0= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2 h1:JTzYYukREvxVSKW/ncrzNjFitd8snoQ/Xz32pw8i+s8= github.com/pingcap/pd/v4 v4.0.0-rc.1.0.20200422143320-428acd53eba2/go.mod h1:s+utZtXDznOiL24VK0qGmtoHjjXNsscJx3m1n8cC56s= github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181 h1:FM+PzdoR3fmWAJx3ug+p5aOgs5aZYwFkoDL7Potdsz0= @@ -475,6 +475,10 @@ github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompat github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible h1:/JKsYjsa5Ug8v5CN4zIbJGIqsvgBUkGwaP/rEScVvWM= github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= +github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= +github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3 h1:ESL3eIt1kUt8IMvR1011ejZlAyDcOzw89ARvVHvpD5k= +github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= From b0748f96e3acb40980e83fa54748c304593bff2a Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 23 Jun 2020 13:19:29 +0800 Subject: [PATCH 55/61] disable broadcast join when new collation is enabled --- planner/core/exhaust_physical_plans.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 55857885ec89c..c97e64cd7650e 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/ranger" @@ -1479,6 +1480,10 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } func (p *LogicalJoin) tryToGetBroadCastJoin(prop *property.PhysicalProperty) []PhysicalPlan { + /// todo remove this restriction after join on new collation is supported in TiFlash + if collate.NewCollationEnabled() { + return nil + } if !prop.IsEmpty() { return nil } From 82ce64126a4f9b94bdd677c138304592e37d0b03 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 24 Jun 2020 01:12:20 +0800 Subject: [PATCH 56/61] Update planner/core/exhaust_physical_plans.go Co-authored-by: Zhuomin(Charming) Liu --- planner/core/exhaust_physical_plans.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index c97e64cd7650e..cc54addd0e81b 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1535,9 +1535,7 @@ func (p *LogicalJoin) tryToGetBroadCastJoinByPreferGlobalIdx(prop *property.Phys basePhysicalJoin: baseJoin, globalChildIndex: preferredGlobalIndex, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, childrenReqProps...) - results := make([]PhysicalPlan, 0, 1) - results = append(results, join) - return results + return []PhysicalPlan{join} } // TryToGetChildProp will check if this sort property can be pushed or not. From b760e78f9e26e51359fb703482116dcc9a4805af Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 24 Jun 2020 14:12:06 +0800 Subject: [PATCH 57/61] address comments --- executor/builder.go | 2 +- planner/core/exhaust_physical_plans.go | 4 ++-- planner/core/task.go | 23 ----------------------- planner/property/physical_property.go | 2 +- store/tikv/coprocessor.go | 10 ---------- store/tikv/region_cache.go | 18 +----------------- 6 files changed, 5 insertions(+), 54 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 8243948986eb6..f9b1ea807bfeb 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2020,7 +2020,7 @@ func markChildrenUsedCols(outputSchema *expression.Schema, childSchema ...*expre func constructDistExecForTiFlash(sctx sessionctx.Context, p plannercore.PhysicalPlan) ([]*tipb.Executor, bool, error) { execPB, err := p.ToPB(sctx, kv.TiFlash) - return []*tipb.Executor{execPB}, false, errors.Trace(err) + return []*tipb.Executor{execPB}, false, err } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index cc54addd0e81b..c85d65117b59b 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1436,7 +1436,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } }) - if prop.IsFlashOnlyProp() && ((p.preferJoinType&preferMergeJoin) > 0 || (p.preferJoinType&preferHashJoin) > 0) { + if prop.IsFlashOnlyProp() && ((p.preferJoinType&preferBCJoin) == 0 && p.preferJoinType > 0) { return nil, false } joins := make([]PhysicalPlan, 0, 5) @@ -1760,7 +1760,7 @@ func (la *LogicalAggregation) distinctArgsMeetsProperty() bool { } func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []PhysicalPlan { - // todo support CopTiFlash task type in stream agg + // TODO: support CopTiFlash task type in stream agg if prop.IsFlashOnlyProp() { return nil } diff --git a/planner/core/task.go b/planner/core/task.go index 967777e9ae788..79b66c84ed3b5 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/plancodec" @@ -565,8 +564,6 @@ func (p *PhysicalBroadCastJoin) GetCost(lCnt, rCnt float64) float64 { func (p *PhysicalBroadCastJoin) attach2Task(tasks ...task) task { lTask, lok := tasks[0].(*copTask) rTask, rok := tasks[1].(*copTask) - lGlobalRead := p.childrenReqProps[0].TaskTp == property.CopTiFlashGlobalReadTaskType - rGlobalRead := p.childrenReqProps[1].TaskTp == property.CopTiFlashGlobalReadTaskType if !lok || !rok || (lTask.getStoreType() != kv.TiFlash && rTask.getStoreType() != kv.TiFlash) { return invalidTask } @@ -581,21 +578,6 @@ func (p *PhysicalBroadCastJoin) attach2Task(tasks ...task) task { lCost := lTask.cost() rCost := rTask.cost() - if !(lGlobalRead && rGlobalRead) { - // the cost model for top level broadcast join is - // globalReadSideCost * copTaskNumber + localReadSideCost + broadcast operator cost - // because for broadcast join, the global side is executed in every copTask. - copTaskNumber := int32(1) - copClient, ok := p.ctx.GetClient().(*tikv.CopClient) - if ok { - copTaskNumber = copClient.GetBatchCopTaskNumber() - } - if lGlobalRead { - lCost = lCost * float64(copTaskNumber) - } else { - rCost = rCost * float64(copTaskNumber) - } - } task := &copTask{ tblColHists: rTask.tblColHists, @@ -724,11 +706,6 @@ func finishCopTask(ctx sessionctx.Context, task task) task { // is Min(DistSQLScanConcurrency, numRegionsInvolvedInScan), since we cannot infer // the number of regions involved, we simply use DistSQLScanConcurrency. copIterWorkers := float64(t.plan().SCtx().GetSessionVars().DistSQLScanConcurrency()) - if t.tablePlan != nil && t.tablePlan.TP() == plancodec.TypeBroadcastJoin { - if copClient, ok := ctx.GetClient().(*tikv.CopClient); ok { - copIterWorkers = math.Min(float64(copClient.GetBatchCopTaskNumber()), copIterWorkers) - } - } t.finishIndexPlan() // Network cost of transferring rows of table scan to TiDB. if t.tablePlan != nil { diff --git a/planner/property/physical_property.go b/planner/property/physical_property.go index 35a66ade47579..1753031bd0bb5 100644 --- a/planner/property/physical_property.go +++ b/planner/property/physical_property.go @@ -97,7 +97,7 @@ func (p *PhysicalProperty) GetAllPossibleChildTaskTypes() []TaskType { if p.TaskTp == RootTaskType { return wholeTaskTypes } - // todo for CopSingleReadTaskType and CopDoubleReadTaskType, this function should never be called + // TODO: For CopSingleReadTaskType and CopDoubleReadTaskType, this function should never be called return []TaskType{p.TaskTp} } diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index af5c30ad87be9..370852a61de43 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -55,16 +55,6 @@ type CopClient struct { replicaReadSeed uint32 } -// GetBatchCopTaskNumber returns the number of stores that tiflash engines have. -// This information is important for generating tiflash plan, for example, it can help calcuate the cost of broadcasting small table. -func (c *CopClient) GetBatchCopTaskNumber() (ret int32) { - ret = c.store.regionCache.storeMu.flashStoreNumber - if ret <= 0 { - ret = 1 - } - return ret -} - // Send builds the request and gets the coprocessor iterator response. func (c *CopClient) Send(ctx context.Context, req *kv.Request, vars *kv.Variables) kv.Response { if req.StoreType == kv.TiFlash && req.BatchCop { diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 61e6320354af9..7bfdfb72ca24d 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -271,8 +271,7 @@ type RegionCache struct { } storeMu struct { sync.RWMutex - stores map[uint64]*Store - flashStoreNumber int32 + stores map[uint64]*Store } notifyCheckCh chan struct{} closeCh chan struct{} @@ -286,7 +285,6 @@ func NewRegionCache(pdClient pd.Client) *RegionCache { c.mu.regions = make(map[RegionVerID]*Region) c.mu.sorted = btree.New(btreeDegree) c.storeMu.stores = make(map[uint64]*Store) - c.storeMu.flashStoreNumber = 0 c.notifyCheckCh = make(chan struct{}, 1) c.closeCh = make(chan struct{}) go c.asyncCheckAndResolveLoop() @@ -1079,13 +1077,6 @@ func (c *RegionCache) getStoreAddr(bo *Backoffer, region *Region, store *Store, return case unresolved: addr, err = store.initResolve(bo, c) - if store.storeType == kv.TiFlash { - c.storeMu.Lock() - if _, exists := c.storeMu.stores[store.storeID]; exists { - c.storeMu.flashStoreNumber++ - } - c.storeMu.Unlock() - } return case deleted: addr = c.changeToActiveStore(region, store, storeIdx) @@ -1498,14 +1489,7 @@ func (s *Store) reResolve(c *RegionCache) { newStore := &Store{storeID: s.storeID, addr: addr, saddr: store.GetStatusAddress(), storeType: storeType} newStore.state = *(*uint64)(&state) c.storeMu.Lock() - orgStore, exists := c.storeMu.stores[newStore.storeID] - if exists && orgStore.storeType == kv.TiFlash { - c.storeMu.flashStoreNumber-- - } c.storeMu.stores[newStore.storeID] = newStore - if newStore.storeType == kv.TiFlash { - c.storeMu.flashStoreNumber++ - } c.storeMu.Unlock() retryMarkDel: // all region used those From f69bb5bea3e06a667d30bc60531342b201352a48 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 24 Jun 2020 14:50:47 +0800 Subject: [PATCH 58/61] fix --- go.sum | 2 -- 1 file changed, 2 deletions(-) diff --git a/go.sum b/go.sum index 565fba8e884d9..f2c76caa89deb 100644 --- a/go.sum +++ b/go.sum @@ -486,8 +486,6 @@ github.com/pingcap/tidb-tools v4.0.0+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnw github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20200604070248-508f03b0b342/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= -github.com/pingcap/tipb v0.0.0-20200615034523-dcfcea0b5965 h1:a0kZ+iaj/sbzJa5mt5310t1XJSpY+wmmIauAkrr7gU4= -github.com/pingcap/tipb v0.0.0-20200615034523-dcfcea0b5965/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3 h1:ESL3eIt1kUt8IMvR1011ejZlAyDcOzw89ARvVHvpD5k= github.com/pingcap/tipb v0.0.0-20200618092958-4fad48b4c8c3/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= From 499f7b8fd6170e74d213b81e0b4fcb105c9c0c54 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 7 Jul 2020 19:10:53 +0800 Subject: [PATCH 59/61] address comments --- planner/core/exhaust_physical_plans.go | 1 - 1 file changed, 1 deletion(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b18b8ddd21c75..19fc365a3fe4c 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1481,7 +1481,6 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P if p.ctx.GetSessionVars().AllowBCJ { broadCastJoins := p.tryToGetBroadCastJoin(prop) if (p.preferJoinType & preferBCJoin) > 0 { - logutil.BgLogger().Info("prefer bc join", zap.Int("bc count", len(broadCastJoins))) return broadCastJoins, true } joins = append(joins, broadCastJoins...) From adc2d0483e3006f9718e57b38e8f16e905633dc3 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 17 Jul 2020 14:35:44 +0800 Subject: [PATCH 60/61] fix tests --- planner/core/exhaust_physical_plans.go | 2 +- store/mockstore/mocktikv/cop_handler_dag.go | 40 +++++++++++++++++---- store/mockstore/mocktikv/rpc.go | 2 +- 3 files changed, 35 insertions(+), 9 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index c753061db907a..9a360c1b07541 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1477,7 +1477,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P if prop.IsFlashOnlyProp() && ((p.preferJoinType&preferBCJoin) == 0 && p.preferJoinType > 0) { return nil, false } - joins := make([]PhysicalPlan, 0, 5) + joins := make([]PhysicalPlan, 0, 8) if p.ctx.GetSessionVars().AllowBCJ { broadCastJoins := p.tryToGetBroadCastJoin(prop) if (p.preferJoinType & preferBCJoin) > 0 { diff --git a/store/mockstore/mocktikv/cop_handler_dag.go b/store/mockstore/mocktikv/cop_handler_dag.go index bb240a23e8506..bdd7ca845c633 100644 --- a/store/mockstore/mocktikv/cop_handler_dag.go +++ b/store/mockstore/mocktikv/cop_handler_dag.go @@ -60,7 +60,7 @@ func (h *rpcHandler) handleCopDAGRequest(req *coprocessor.Request) *coprocessor. resp.RegionError = err return resp } - dagCtx, e, dagReq, err := h.buildDAGExecutor(req) + dagCtx, e, dagReq, err := h.buildDAGExecutor(req, false) if err != nil { resp.OtherError = err.Error() return resp @@ -92,7 +92,7 @@ func (h *rpcHandler) handleCopDAGRequest(req *coprocessor.Request) *coprocessor. return buildResp(selResp, execDetails, err) } -func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request) (*dagContext, executor, *tipb.DAGRequest, error) { +func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request, batchCop bool) (*dagContext, executor, *tipb.DAGRequest, error) { if len(req.Ranges) == 0 { return nil, nil, nil, errors.New("request range is null") } @@ -118,7 +118,12 @@ func (h *rpcHandler) buildDAGExecutor(req *coprocessor.Request) (*dagContext, ex startTS: req.StartTs, evalCtx: &evalContext{sc: sc}, } - e, err := h.buildDAG(ctx, dagReq.Executors) + var e executor + if batchCop { + e, err = h.buildDAGForTiFlash(ctx, dagReq.RootExecutor) + } else { + e, err = h.buildDAG(ctx, dagReq.Executors) + } if err != nil { return nil, nil, nil, errors.Trace(err) } @@ -133,7 +138,7 @@ func constructTimeZone(name string, offset int) (*time.Location, error) { } func (h *rpcHandler) handleCopStream(ctx context.Context, req *coprocessor.Request) (tikvpb.Tikv_CoprocessorStreamClient, error) { - dagCtx, e, dagReq, err := h.buildDAGExecutor(req) + dagCtx, e, dagReq, err := h.buildDAGExecutor(req, false) if err != nil { return nil, errors.Trace(err) } @@ -146,9 +151,10 @@ func (h *rpcHandler) handleCopStream(ctx context.Context, req *coprocessor.Reque }, nil } -func (h *rpcHandler) buildExec(ctx *dagContext, curr *tipb.Executor) (executor, error) { +func (h *rpcHandler) buildExec(ctx *dagContext, curr *tipb.Executor) (executor, *tipb.Executor, error) { var currExec executor var err error + var childExec *tipb.Executor switch curr.GetTp() { case tipb.ExecType_TypeTableScan: currExec, err = h.buildTableScan(ctx, curr) @@ -156,26 +162,46 @@ func (h *rpcHandler) buildExec(ctx *dagContext, curr *tipb.Executor) (executor, currExec, err = h.buildIndexScan(ctx, curr) case tipb.ExecType_TypeSelection: currExec, err = h.buildSelection(ctx, curr) + childExec = curr.Selection.Child case tipb.ExecType_TypeAggregation: currExec, err = h.buildHashAgg(ctx, curr) + childExec = curr.Aggregation.Child case tipb.ExecType_TypeStreamAgg: currExec, err = h.buildStreamAgg(ctx, curr) + childExec = curr.Aggregation.Child case tipb.ExecType_TypeTopN: currExec, err = h.buildTopN(ctx, curr) + childExec = curr.TopN.Child case tipb.ExecType_TypeLimit: currExec = &limitExec{limit: curr.Limit.GetLimit(), execDetail: new(execDetail)} + childExec = curr.Limit.Child default: // TODO: Support other types. err = errors.Errorf("this exec type %v doesn't support yet.", curr.GetTp()) } - return currExec, errors.Trace(err) + return currExec, childExec, errors.Trace(err) +} + +func (h *rpcHandler) buildDAGForTiFlash(ctx *dagContext, farther *tipb.Executor) (executor, error) { + curr, child, err := h.buildExec(ctx, farther) + if err != nil { + return nil, errors.Trace(err) + } + if child != nil { + childExec, err := h.buildDAGForTiFlash(ctx, child) + if err != nil { + return nil, errors.Trace(err) + } + curr.SetSrcExec(childExec) + } + return curr, nil } func (h *rpcHandler) buildDAG(ctx *dagContext, executors []*tipb.Executor) (executor, error) { var src executor for i := 0; i < len(executors); i++ { - curr, err := h.buildExec(ctx, executors[i]) + curr, _, err := h.buildExec(ctx, executors[i]) if err != nil { return nil, errors.Trace(err) } diff --git a/store/mockstore/mocktikv/rpc.go b/store/mockstore/mocktikv/rpc.go index edd439c8f0d60..0bb162b52e9f8 100644 --- a/store/mockstore/mocktikv/rpc.go +++ b/store/mockstore/mocktikv/rpc.go @@ -689,7 +689,7 @@ func (h *rpcHandler) handleBatchCopRequest(ctx context.Context, req *coprocessor StartTs: req.StartTs, Ranges: ri.Ranges, } - _, exec, dagReq, err := h.buildDAGExecutor(&cop) + _, exec, dagReq, err := h.buildDAGExecutor(&cop, true) if err != nil { return nil, errors.Trace(err) } From 84a35397e07fb5055f9f06601627cf9cb8cc5dfe Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 21 Jul 2020 12:45:03 +0800 Subject: [PATCH 61/61] address comments --- planner/core/find_best_task.go | 8 ++++---- planner/core/planbuilder.go | 2 +- .../testdata/integration_serial_suite_out.json | 16 ++++++++-------- planner/util/path.go | 4 ++-- sessionctx/variable/varsutil.go | 4 ++-- util/plancodec/id.go | 2 +- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index a9b9d685ee377..c3728a7904811 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -537,14 +537,14 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida var currentCandidate *candidatePath if path.IsTablePath() { if path.StoreType == kv.TiFlash { - if path.IsGlobalRead && prop.TaskTp == property.CopTiFlashGlobalReadTaskType { + if path.IsTiFlashGlobalRead && prop.TaskTp == property.CopTiFlashGlobalReadTaskType { currentCandidate = ds.getTableCandidate(path, prop) } - if !path.IsGlobalRead && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { + if !path.IsTiFlashGlobalRead && prop.TaskTp != property.CopTiFlashGlobalReadTaskType { currentCandidate = ds.getTableCandidate(path, prop) } } else { - if !path.IsGlobalRead && !prop.IsFlashOnlyProp() { + if !path.IsTiFlashGlobalRead && !prop.IsFlashOnlyProp() { currentCandidate = ds.getTableCandidate(path, prop) } } @@ -1523,7 +1523,7 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper AccessCondition: path.AccessConds, filterCondition: path.TableFilters, StoreType: path.StoreType, - IsGlobalRead: path.IsGlobalRead, + IsGlobalRead: path.IsTiFlashGlobalRead, }.Init(ds.ctx, ds.blockOffset) ts.SetSchema(ds.schema.Clone()) if ts.Table.PKIsHandle { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index ddeae652b6f91..4d1a85275221d 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -801,7 +801,7 @@ func isPrimaryIndex(indexName model.CIStr) bool { } func genTiFlashPath(tblInfo *model.TableInfo, isGlobalRead bool) *util.AccessPath { - tiFlashPath := &util.AccessPath{StoreType: kv.TiFlash, IsGlobalRead: isGlobalRead} + tiFlashPath := &util.AccessPath{StoreType: kv.TiFlash, IsTiFlashGlobalRead: isGlobalRead} fillContentForTablePath(tiFlashPath, tblInfo) return tiFlashPath } diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 17b70d52e44b5..5beb702a949dd 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -29,7 +29,7 @@ "StreamAgg_32 1.00 root funcs:count(Column#14)->Column#11", "└─TableReader_33 1.00 root data:StreamAgg_13", " └─StreamAgg_13 1.00 cop[tiflash] funcs:count(1)->Column#14", - " └─TypeBroadcastJoin_31 8.00 cop[tiflash] ", + " └─BroadcastJoin_31 8.00 cop[tiflash] ", " ├─Selection_23(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", " │ └─TableFullScan_22 2.00 cop[tiflash] table:d1_t keep order:false, global read", " └─Selection_21(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", @@ -42,13 +42,13 @@ "StreamAgg_52 1.00 root funcs:count(Column#20)->Column#17", "└─TableReader_53 1.00 root data:StreamAgg_17", " └─StreamAgg_17 1.00 cop[tiflash] funcs:count(1)->Column#20", - " └─TypeBroadcastJoin_51 8.00 cop[tiflash] ", + " └─BroadcastJoin_51 8.00 cop[tiflash] ", " ├─Selection_43(Build) 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", " │ └─TableFullScan_42 2.00 cop[tiflash] table:d3_t keep order:false, global read", - " └─TypeBroadcastJoin_33(Probe) 8.00 cop[tiflash] ", + " └─BroadcastJoin_33(Probe) 8.00 cop[tiflash] ", " ├─Selection_29(Build) 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", " │ └─TableFullScan_28 2.00 cop[tiflash] table:d2_t keep order:false, global read", - " └─TypeBroadcastJoin_37(Probe) 8.00 cop[tiflash] ", + " └─BroadcastJoin_37(Probe) 8.00 cop[tiflash] ", " ├─Selection_27(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", " │ └─TableFullScan_26 2.00 cop[tiflash] table:d1_t keep order:false, global read", " └─Selection_41(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", @@ -61,7 +61,7 @@ "StreamAgg_25 1.00 root funcs:count(Column#14)->Column#11", "└─TableReader_26 1.00 root data:StreamAgg_13", " └─StreamAgg_13 1.00 cop[tiflash] funcs:count(1)->Column#14", - " └─TypeBroadcastJoin_24 8.00 cop[tiflash] ", + " └─BroadcastJoin_24 8.00 cop[tiflash] ", " ├─Selection_18(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", " │ └─TableFullScan_17 2.00 cop[tiflash] table:d1_t keep order:false", " └─Selection_16(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", @@ -74,13 +74,13 @@ "StreamAgg_36 1.00 root funcs:count(Column#20)->Column#17", "└─TableReader_37 1.00 root data:StreamAgg_17", " └─StreamAgg_17 1.00 cop[tiflash] funcs:count(1)->Column#20", - " └─TypeBroadcastJoin_35 8.00 cop[tiflash] ", + " └─BroadcastJoin_35 8.00 cop[tiflash] ", " ├─Selection_29(Build) 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", " │ └─TableFullScan_28 2.00 cop[tiflash] table:d3_t keep order:false, global read", - " └─TypeBroadcastJoin_19(Probe) 8.00 cop[tiflash] ", + " └─BroadcastJoin_19(Probe) 8.00 cop[tiflash] ", " ├─Selection_27(Build) 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", " │ └─TableFullScan_26 2.00 cop[tiflash] table:d2_t keep order:false", - " └─TypeBroadcastJoin_20(Probe) 8.00 cop[tiflash] ", + " └─BroadcastJoin_20(Probe) 8.00 cop[tiflash] ", " ├─Selection_25(Build) 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", " │ └─TableFullScan_24 2.00 cop[tiflash] table:d1_t keep order:false, global read", " └─Selection_23(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", diff --git a/planner/util/path.go b/planner/util/path.go index 60169040f24cc..eccf642677fc4 100644 --- a/planner/util/path.go +++ b/planner/util/path.go @@ -50,8 +50,8 @@ type AccessPath struct { IsDNFCond bool - // IsGlobalRead indicates whether this path is a remote read path for tiflash - IsGlobalRead bool + // IsTiFlashGlobalRead indicates whether this path is a remote read path for tiflash + IsTiFlashGlobalRead bool // IsIntHandlePath indicates whether this path is table path. IsIntHandlePath bool diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 0fa43fd4e86c5..5dd30ddb91b28 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -453,7 +453,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) case TiDBOptBCJ: if (strings.EqualFold(value, "ON") || value == "1") && vars.AllowBatchCop == 0 { - return value, ErrWrongValueForVar.GenWithStackByArgs("Can't set BCJ to 1 but tidb_allow_batch_cop is 0, please active batch cop at first.") + return value, ErrWrongValueForVar.GenWithStackByArgs("Can't set Broadcast Join to 1 but tidb_allow_batch_cop is 0, please active batch cop at first.") } return value, nil case TiDBSkipUTF8Check, TiDBSkipASCIICheck, TiDBOptAggPushDown, @@ -581,7 +581,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string, scope Sc return value, ErrWrongTypeForVar.GenWithStackByArgs(name) } if v == 0 && vars.AllowBCJ { - return value, ErrWrongValueForVar.GenWithStackByArgs("Can't set batch cop 0 but tidb_opt_broadcast_join is 1, please set bcj 0 at first") + return value, ErrWrongValueForVar.GenWithStackByArgs("Can't set batch cop 0 but tidb_opt_broadcast_join is 1, please set tidb_opt_broadcast_join 0 at first") } if v < 0 || v > 2 { return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) diff --git a/util/plancodec/id.go b/util/plancodec/id.go index f319166a88e24..a21b0286cfc02 100644 --- a/util/plancodec/id.go +++ b/util/plancodec/id.go @@ -53,7 +53,7 @@ const ( // TypeHashJoin is the type of hash join. TypeHashJoin = "HashJoin" // TypeBroadcastJoin is the type of broad cast join. - TypeBroadcastJoin = "TypeBroadcastJoin" + TypeBroadcastJoin = "BroadcastJoin" // TypeMergeJoin is the type of merge join. TypeMergeJoin = "MergeJoin" // TypeIndexJoin is the type of index look up join.