From e9d56f7f693b8f32eb516d444ce7ec249d999f53 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 29 Mar 2022 11:18:09 +0800 Subject: [PATCH 01/44] randomly dispatch req to tiflash-dev1 Signed-off-by: guo-shaoge --- store/copr/batch_coprocessor.go | 116 ++++++++++++++++++++++++++++- store/copr/batch_request_sender.go | 1 + store/copr/mpp.go | 24 +++++- 3 files changed, 139 insertions(+), 2 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 32f38a72f7576..207fcd35f3a59 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -526,7 +526,8 @@ func balanceBatchCopTask(ctx context.Context, kvStore *kvStore, originalTasks [] } func buildBatchCopTasksForNonPartitionedTable(bo *backoff.Backoffer, store *kvStore, ranges *KeyRanges, storeType kv.StoreType, mppStoreLastFailTime map[string]time.Time, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) ([]*batchCopTask, error) { - return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) + // return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) + return buildBatchCopTasksRandomly(bo, store, []*KeyRanges{ranges}, storeType) } func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType, mppStoreLastFailTime map[string]time.Time, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64, partitionIDs []int64) ([]*batchCopTask, error) { @@ -539,6 +540,119 @@ func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, store *kvStore return batchTasks, nil } +// 1. The returned batchCopTask num equals to realted TiFlash node num +// 2. The region of first copTask in one batchCopTask is located in TiFlash specified in batchCopTask.storeAddr. +// 3. But the remaining copTasks in one batchCopTask is randomly dispatched (using golang map). +func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType) (res []*batchCopTask, _ error) { + const cmdType = tikvrpc.CmdBatchCop + var rangesLen int + var retryNum int + var addrMap map[string]bool + var taskCtxMap map[*copTask]*tikv.RPCContext + var tasks []*copTask + cache := store.GetRegionCache() + + for { + retryNum++ + if retryNum >= 10 { + return nil, errors.New("too many times of retry to GetTiFlashRPCContext()") + } + + rangesLen = 0 + addrMap = make(map[string]bool) + taskCtxMap = make(map[*copTask]*tikv.RPCContext) + tasks = make([]*copTask, 0) + + for i, ranges := range rangesForEachPhysicalTable { + rangesLen += ranges.Len() + locations, err := cache.SplitKeyRangesByLocations(bo, ranges) + if err != nil { + return nil, errors.Trace(err) + } + for _, lo := range locations { + tasks = append(tasks, &copTask{ + region: lo.Location.Region, + ranges: lo.Ranges, + cmdType: cmdType, + storeType: storeType, + partitionIndex: int64(i), + }) + } + } + + for _, task := range tasks { + // todo: check this func + rpcCtx, err := cache.GetTiFlashRPCContext(bo.TiKVBackoffer(), task.region, false) + if err != nil { + return nil, errors.Trace(err) + } + if rpcCtx == nil { + // todo: looks like it happens, why + logutil.BgLogger().Info(fmt.Sprintf("GetTiFlashRPCContext failed: %v", task.region)) + err := bo.Backoff(tikv.BoTiFlashRPC(), errors.New("Cannot find region with TiFlash peer")) + if err != nil { + return nil, errors.Trace(err) + } + break + } + addrMap[rpcCtx.Addr] = true + taskCtxMap[task] = rpcCtx + } + if len(taskCtxMap) == len(tasks) { + break + } + } + + nodeNum := len(addrMap) + copTaskNumForEachNode := len(tasks)/nodeNum + 1 + var taskIdxInOneNode int + var handledNum int + for task, rpcCtx := range taskCtxMap { + allStores := cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store) + regionInfo := RegionInfo{ + Region: task.region, + Meta: rpcCtx.Meta, + Ranges: task.ranges, + AllStores: allStores, + PartitionIndex: task.partitionIndex, + Addr: rpcCtx.Addr, + } + + if taskIdxInOneNode == copTaskNumForEachNode { + taskIdxInOneNode = 0 + } + + if taskIdxInOneNode == 0 { + res = append(res, &batchCopTask{ + storeAddr: rpcCtx.Addr, + cmdType: cmdType, + ctx: rpcCtx, + regionInfos: []RegionInfo{regionInfo}, + }) + } else if taskIdxInOneNode < copTaskNumForEachNode { + lastBatchCopTask := res[len(res)-1] + lastBatchCopTask.regionInfos = append(lastBatchCopTask.regionInfos, regionInfo) + } else { + panic("cannot reach here") + } + handledNum++ + taskIdxInOneNode++ + } + + if handledNum != len(tasks) { + panic(fmt.Sprintf("handled %v tasks, but expect %v, len(taskCtxMap): %v, nodeNum: %v, copTaskNumForEachNode: %v", handledNum, len(tasks), len(taskCtxMap), nodeNum, copTaskNumForEachNode)) + } + + for _, batchTask := range res { + logMsg := fmt.Sprintf("batchCopTask Addr: %s", batchTask.storeAddr) + for i, perTaskRegionInfo := range batchTask.regionInfos { + logMsg += fmt.Sprintf(", copTask[%d]: %s", i, perTaskRegionInfo.Addr) + } + logutil.BgLogger().Info(logMsg) + } + return res, nil +} + // When `partitionIDs != nil`, it means that buildBatchCopTasksCore is constructing a batch cop tasks for PartitionTableScan. // At this time, `len(rangesForEachPhysicalTable) == len(partitionIDs)` and `rangesForEachPhysicalTable[i]` is for partition `partitionIDs[i]`. // Otherwise, `rangesForEachPhysicalTable[0]` indicates the range for the single physical table. diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index b976d26a59ab3..a1e3e210ff67a 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -35,6 +35,7 @@ type RegionInfo struct { Ranges *KeyRanges AllStores []uint64 PartitionIndex int64 // used by PartitionTableScan, indicates the n-th partition of the partition table + Addr string } func (ri *RegionInfo) toCoprocessorRegionInfo() *coprocessor.RegionInfo { diff --git a/store/copr/mpp.go b/store/copr/mpp.go index f385fedacc5b3..a2e60091dbe7d 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -16,7 +16,9 @@ package copr import ( "context" + "fmt" "io" + "math/rand" "strconv" "sync" "sync/atomic" @@ -80,6 +82,7 @@ func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasks } ranges := NewKeyRanges(req.KeyRanges) tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store, ranges, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20) + // tasks = disruptTaskAddr(tasks) } if err != nil { @@ -92,6 +95,25 @@ func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasks return mppTasks, nil } +func disruptTaskAddr(tasks []*batchCopTask) []*batchCopTask { + addrs := make([]string, 0, len(tasks)) + var logMsg string + for _, t := range tasks { + addrs = append(addrs, t.GetAddress()) + logMsg += fmt.Sprintf("%s;", t.GetAddress()) + } + logutil.BgLogger().Info("batchCopTask Addr before disrupt: " + logMsg) + + rand.Shuffle(len(addrs), func(i, j int) { addrs[i], addrs[j] = addrs[j], addrs[i] }) + logMsg = "" + for i, addr := range addrs { + tasks[i].storeAddr = addr + logMsg += fmt.Sprintf("%s;", addr) + } + logutil.BgLogger().Info("batchCopTask Addr after disrupt: " + logMsg) + return tasks +} + // mppResponse wraps mpp data packet. type mppResponse struct { pbResp *mpp.MPPDataPacket @@ -300,7 +322,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req if index < 10 || log.GetLevel() <= zap.DebugLevel { logutil.BgLogger().Info("invalid region because tiflash detected stale region", zap.String("region id", id.String())) } - m.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) + // m.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) } } failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { From 56f98ca63b7134a45eeb00e341e62b82ef9a22f0 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 29 Mar 2022 14:41:31 +0800 Subject: [PATCH 02/44] distribute task randomly Signed-off-by: guo-shaoge --- store/copr/batch_coprocessor.go | 28 +++++++++++++++++++--------- 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 207fcd35f3a59..f661f6c7b5261 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -20,6 +20,7 @@ import ( "fmt" "io" "math" + "math/rand" "sort" "strconv" "sync" @@ -547,10 +548,9 @@ func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesFor const cmdType = tikvrpc.CmdBatchCop var rangesLen int var retryNum int - var addrMap map[string]bool var taskCtxMap map[*copTask]*tikv.RPCContext var tasks []*copTask - cache := store.GetRegionCache() + cache := store.GetRegionCache() for { retryNum++ @@ -559,7 +559,6 @@ func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesFor } rangesLen = 0 - addrMap = make(map[string]bool) taskCtxMap = make(map[*copTask]*tikv.RPCContext) tasks = make([]*copTask, 0) @@ -595,7 +594,6 @@ func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesFor } break } - addrMap[rpcCtx.Addr] = true taskCtxMap[task] = rpcCtx } if len(taskCtxMap) == len(tasks) { @@ -603,17 +601,22 @@ func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesFor } } - nodeNum := len(addrMap) + allStores := cache.GetTiFlashStores() + nodeNum := len(allStores) + if nodeNum == 0 { + panic("tiflash stores num is zero") + } + copTaskNumForEachNode := len(tasks)/nodeNum + 1 var taskIdxInOneNode int var handledNum int for task, rpcCtx := range taskCtxMap { - allStores := cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store) + rpcCtx.Addr = allStores[rand.Intn(len(allStores))].GetAddr() regionInfo := RegionInfo{ Region: task.region, Meta: rpcCtx.Meta, Ranges: task.ranges, - AllStores: allStores, + AllStores: cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store), PartitionIndex: task.partitionIndex, Addr: rpcCtx.Addr, } @@ -640,13 +643,20 @@ func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesFor } if handledNum != len(tasks) { - panic(fmt.Sprintf("handled %v tasks, but expect %v, len(taskCtxMap): %v, nodeNum: %v, copTaskNumForEachNode: %v", handledNum, len(tasks), len(taskCtxMap), nodeNum, copTaskNumForEachNode)) + panic(fmt.Sprintf("handled %v tasks, but expect %v, len(taskCtxMap): %v, nodeNum: %v, copTaskNumForEachNode: %v", + handledNum, len(tasks), len(taskCtxMap), nodeNum, copTaskNumForEachNode)) + } + + var logMsg string + for i, s := range allStores { + logMsg += fmt.Sprintf("store[%d]: %s, ", i, s.GetAddr()) } + logutil.BgLogger().Info(fmt.Sprintf("nodeNum: %v. ", nodeNum) + logMsg) for _, batchTask := range res { logMsg := fmt.Sprintf("batchCopTask Addr: %s", batchTask.storeAddr) for i, perTaskRegionInfo := range batchTask.regionInfos { - logMsg += fmt.Sprintf(", copTask[%d]: %s", i, perTaskRegionInfo.Addr) + logMsg += fmt.Sprintf(", copTask[%d]: %s(regid:%v)", i, perTaskRegionInfo.Addr, perTaskRegionInfo.Region) } logutil.BgLogger().Info(logMsg) } From d84a99d5a0a6e9da1227985b59c55402f14cf7f8 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 27 Apr 2022 21:14:04 +0800 Subject: [PATCH 03/44] dispatch MPPTask to TiFlash ReadNodes using consistent hash Signed-off-by: guo-shaoge --- go.mod | 4 + go.sum | 6 +- store/copr/batch_coprocessor.go | 140 ++++++++++++++++++++++++++++++-- 3 files changed, 140 insertions(+), 10 deletions(-) diff --git a/go.mod b/go.mod index 8de1512d205f7..7516d9e5b092e 100644 --- a/go.mod +++ b/go.mod @@ -60,6 +60,7 @@ require ( github.com/soheilhy/cmux v0.1.5 github.com/spf13/cobra v1.4.0 github.com/spf13/pflag v1.0.5 + github.com/stathat/consistent v1.0.0 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 github.com/tikv/client-go/v2 v2.0.1-0.20220321123529-f4eae62b7ed5 @@ -187,9 +188,12 @@ require ( gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect sigs.k8s.io/yaml v1.2.0 // indirect + stathat.com/c/consistent v1.0.0 // indirect ) replace github.com/pingcap/tidb/parser => ./parser // fix potential security issue(CVE-2020-26160) introduced by indirect dependency. replace github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible + +replace github.com/tikv/client-go/v2 => /home/guojiangtao/work/client-go diff --git a/go.sum b/go.sum index ddc99c25aa9cf..ae3a300036464 100644 --- a/go.sum +++ b/go.sum @@ -700,6 +700,8 @@ 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/spf13/viper v1.7.0/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg= +github.com/stathat/consistent v1.0.0 h1:ZFJ1QTRn8npNBKW065raSZ8xfOqhpb8vLOkfp4CcL/U= +github.com/stathat/consistent v1.0.0/go.mod h1:uajTPbgSygZBJ+V+0mY7meZ8i0XAcZs7AQ6V121XSxw= 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/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= @@ -712,8 +714,6 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= 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/tikv/client-go/v2 v2.0.1-0.20220321123529-f4eae62b7ed5 h1:XcvL365oWoFAdprI2H23vvFQYHc2MBQ/CcWHvkvIzGk= -github.com/tikv/client-go/v2 v2.0.1-0.20220321123529-f4eae62b7ed5/go.mod h1:0scaG+seu7L56apm+Gjz9vckyO7ABIzM6T7n00mrIXs= github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710 h1:jxgmKOscXSjaFEKQGRyY5qOpK8hLqxs2irb/uDJMtwk= github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710/go.mod h1:AtvppPwkiyUgQlR1W9qSqfTB+OsOIu19jDCOxOsPkmU= github.com/tklauser/go-sysconf v0.3.9 h1:JeUVdAOWhhxVcU6Eqr/ATFHgXk/mmiItdKeJPev3vTo= @@ -1390,3 +1390,5 @@ sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 h1:ucqkfp sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 h1:e1sMhtVq9AfcEy8AXNb8eSg6gbzfdpYhoNqnPJa+GzI= sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67/go.mod h1:L5q+DGLGOQFpo1snNEkLOJT2d1YTW66rWNzatr3He1k= +stathat.com/c/consistent v1.0.0 h1:ezyc51EGcRPJUxfHGSgJjWzJdj3NiMU9pNfLNGiXV0c= +stathat.com/c/consistent v1.0.0/go.mod h1:QkzMWzcbB+yQBL2AttO6sgsQS/JSTapcDISJalmCDS0= diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index f661f6c7b5261..87febbc964341 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -20,7 +20,6 @@ import ( "fmt" "io" "math" - "math/rand" "sort" "strconv" "sync" @@ -41,6 +40,7 @@ import ( "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" "go.uber.org/zap" + "github.com/stathat/consistent" ) // batchCopTask comprises of multiple copTask that will send to same store. @@ -528,7 +528,8 @@ func balanceBatchCopTask(ctx context.Context, kvStore *kvStore, originalTasks [] func buildBatchCopTasksForNonPartitionedTable(bo *backoff.Backoffer, store *kvStore, ranges *KeyRanges, storeType kv.StoreType, mppStoreLastFailTime map[string]time.Time, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) ([]*batchCopTask, error) { // return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) - return buildBatchCopTasksRandomly(bo, store, []*KeyRanges{ranges}, storeType) + // return buildBatchCopTasksRandomly(bo, store, []*KeyRanges{ranges}, storeType) + return buildBatchCopTasksConsistentHash(bo, store, []*KeyRanges{ranges}, storeType) } func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType, mppStoreLastFailTime map[string]time.Time, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64, partitionIDs []int64) ([]*batchCopTask, error) { @@ -541,10 +542,7 @@ func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, store *kvStore return batchTasks, nil } -// 1. The returned batchCopTask num equals to realted TiFlash node num -// 2. The region of first copTask in one batchCopTask is located in TiFlash specified in batchCopTask.storeAddr. -// 3. But the remaining copTasks in one batchCopTask is randomly dispatched (using golang map). -func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType) (res []*batchCopTask, _ error) { +func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType) (res []*batchCopTask, err error) { const cmdType = tikvrpc.CmdBatchCop var rangesLen int var retryNum int @@ -601,17 +599,21 @@ func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesFor } } - allStores := cache.GetTiFlashStores() + allStores := cache.GetTiFlashMPPStores() nodeNum := len(allStores) if nodeNum == 0 { panic("tiflash stores num is zero") } + hasher := consistent.New() + for _, store := range allStores { + hasher.Add(store.GetAddr()) + } copTaskNumForEachNode := len(tasks)/nodeNum + 1 var taskIdxInOneNode int var handledNum int for task, rpcCtx := range taskCtxMap { - rpcCtx.Addr = allStores[rand.Intn(len(allStores))].GetAddr() + rpcCtx.Addr, err = hasher.Get(strconv.Itoa(int(rpcCtx.Region.GetID()))) regionInfo := RegionInfo{ Region: task.region, Meta: rpcCtx.Meta, @@ -663,6 +665,128 @@ func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesFor return res, nil } +// // 1. The returned batchCopTask num equals to realted TiFlash node num +// // 2. The region of first copTask in one batchCopTask is located in TiFlash specified in batchCopTask.storeAddr. +// // 3. But the remaining copTasks in one batchCopTask is randomly dispatched (using golang map). +// func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType) (res []*batchCopTask, _ error) { +// const cmdType = tikvrpc.CmdBatchCop +// var rangesLen int +// var retryNum int +// var taskCtxMap map[*copTask]*tikv.RPCContext +// var tasks []*copTask +// cache := store.GetRegionCache() +// +// for { +// retryNum++ +// if retryNum >= 10 { +// return nil, errors.New("too many times of retry to GetTiFlashRPCContext()") +// } +// +// rangesLen = 0 +// taskCtxMap = make(map[*copTask]*tikv.RPCContext) +// tasks = make([]*copTask, 0) +// +// for i, ranges := range rangesForEachPhysicalTable { +// rangesLen += ranges.Len() +// locations, err := cache.SplitKeyRangesByLocations(bo, ranges) +// if err != nil { +// return nil, errors.Trace(err) +// } +// for _, lo := range locations { +// tasks = append(tasks, &copTask{ +// region: lo.Location.Region, +// ranges: lo.Ranges, +// cmdType: cmdType, +// storeType: storeType, +// partitionIndex: int64(i), +// }) +// } +// } +// +// for _, task := range tasks { +// // todo: check this func +// rpcCtx, err := cache.GetTiFlashRPCContext(bo.TiKVBackoffer(), task.region, false) +// if err != nil { +// return nil, errors.Trace(err) +// } +// if rpcCtx == nil { +// // todo: looks like it happens, why +// logutil.BgLogger().Info(fmt.Sprintf("GetTiFlashRPCContext failed: %v", task.region)) +// err := bo.Backoff(tikv.BoTiFlashRPC(), errors.New("Cannot find region with TiFlash peer")) +// if err != nil { +// return nil, errors.Trace(err) +// } +// break +// } +// taskCtxMap[task] = rpcCtx +// } +// if len(taskCtxMap) == len(tasks) { +// break +// } +// } +// +// allStores := cache.GetTiFlashStores() +// nodeNum := len(allStores) +// if nodeNum == 0 { +// panic("tiflash stores num is zero") +// } +// +// copTaskNumForEachNode := len(tasks)/nodeNum + 1 +// var taskIdxInOneNode int +// var handledNum int +// for task, rpcCtx := range taskCtxMap { +// rpcCtx.Addr = allStores[rand.Intn(len(allStores))].GetAddr() +// regionInfo := RegionInfo{ +// Region: task.region, +// Meta: rpcCtx.Meta, +// Ranges: task.ranges, +// AllStores: cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store), +// PartitionIndex: task.partitionIndex, +// Addr: rpcCtx.Addr, +// } +// +// if taskIdxInOneNode == copTaskNumForEachNode { +// taskIdxInOneNode = 0 +// } +// +// if taskIdxInOneNode == 0 { +// res = append(res, &batchCopTask{ +// storeAddr: rpcCtx.Addr, +// cmdType: cmdType, +// ctx: rpcCtx, +// regionInfos: []RegionInfo{regionInfo}, +// }) +// } else if taskIdxInOneNode < copTaskNumForEachNode { +// lastBatchCopTask := res[len(res)-1] +// lastBatchCopTask.regionInfos = append(lastBatchCopTask.regionInfos, regionInfo) +// } else { +// panic("cannot reach here") +// } +// handledNum++ +// taskIdxInOneNode++ +// } +// +// if handledNum != len(tasks) { +// panic(fmt.Sprintf("handled %v tasks, but expect %v, len(taskCtxMap): %v, nodeNum: %v, copTaskNumForEachNode: %v", +// handledNum, len(tasks), len(taskCtxMap), nodeNum, copTaskNumForEachNode)) +// } +// +// var logMsg string +// for i, s := range allStores { +// logMsg += fmt.Sprintf("store[%d]: %s, ", i, s.GetAddr()) +// } +// logutil.BgLogger().Info(fmt.Sprintf("nodeNum: %v. ", nodeNum) + logMsg) +// +// for _, batchTask := range res { +// logMsg := fmt.Sprintf("batchCopTask Addr: %s", batchTask.storeAddr) +// for i, perTaskRegionInfo := range batchTask.regionInfos { +// logMsg += fmt.Sprintf(", copTask[%d]: %s(regid:%v)", i, perTaskRegionInfo.Addr, perTaskRegionInfo.Region) +// } +// logutil.BgLogger().Info(logMsg) +// } +// return res, nil +// } + // When `partitionIDs != nil`, it means that buildBatchCopTasksCore is constructing a batch cop tasks for PartitionTableScan. // At this time, `len(rangesForEachPhysicalTable) == len(partitionIDs)` and `rangesForEachPhysicalTable[i]` is for partition `partitionIDs[i]`. // Otherwise, `rangesForEachPhysicalTable[0]` indicates the range for the single physical table. From f7f05788c86bb976a0480aa94ed13c14eb9ff2b6 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 27 Apr 2022 21:16:10 +0800 Subject: [PATCH 04/44] fix fmt Signed-off-by: guo-shaoge --- store/copr/batch_coprocessor.go | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 87febbc964341..dd69fa76f9ff1 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -36,11 +36,11 @@ import ( "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/util/logutil" + "github.com/stathat/consistent" "github.com/tikv/client-go/v2/metrics" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" "go.uber.org/zap" - "github.com/stathat/consistent" ) // batchCopTask comprises of multiple copTask that will send to same store. @@ -605,10 +605,10 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran panic("tiflash stores num is zero") } - hasher := consistent.New() - for _, store := range allStores { - hasher.Add(store.GetAddr()) - } + hasher := consistent.New() + for _, store := range allStores { + hasher.Add(store.GetAddr()) + } copTaskNumForEachNode := len(tasks)/nodeNum + 1 var taskIdxInOneNode int var handledNum int @@ -675,17 +675,17 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran // var taskCtxMap map[*copTask]*tikv.RPCContext // var tasks []*copTask // cache := store.GetRegionCache() -// +// // for { // retryNum++ // if retryNum >= 10 { // return nil, errors.New("too many times of retry to GetTiFlashRPCContext()") // } -// +// // rangesLen = 0 // taskCtxMap = make(map[*copTask]*tikv.RPCContext) // tasks = make([]*copTask, 0) -// +// // for i, ranges := range rangesForEachPhysicalTable { // rangesLen += ranges.Len() // locations, err := cache.SplitKeyRangesByLocations(bo, ranges) @@ -702,7 +702,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran // }) // } // } -// +// // for _, task := range tasks { // // todo: check this func // rpcCtx, err := cache.GetTiFlashRPCContext(bo.TiKVBackoffer(), task.region, false) @@ -724,13 +724,13 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran // break // } // } -// +// // allStores := cache.GetTiFlashStores() // nodeNum := len(allStores) // if nodeNum == 0 { // panic("tiflash stores num is zero") // } -// +// // copTaskNumForEachNode := len(tasks)/nodeNum + 1 // var taskIdxInOneNode int // var handledNum int @@ -744,11 +744,11 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran // PartitionIndex: task.partitionIndex, // Addr: rpcCtx.Addr, // } -// +// // if taskIdxInOneNode == copTaskNumForEachNode { // taskIdxInOneNode = 0 // } -// +// // if taskIdxInOneNode == 0 { // res = append(res, &batchCopTask{ // storeAddr: rpcCtx.Addr, @@ -765,18 +765,18 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran // handledNum++ // taskIdxInOneNode++ // } -// +// // if handledNum != len(tasks) { // panic(fmt.Sprintf("handled %v tasks, but expect %v, len(taskCtxMap): %v, nodeNum: %v, copTaskNumForEachNode: %v", // handledNum, len(tasks), len(taskCtxMap), nodeNum, copTaskNumForEachNode)) // } -// +// // var logMsg string // for i, s := range allStores { // logMsg += fmt.Sprintf("store[%d]: %s, ", i, s.GetAddr()) // } // logutil.BgLogger().Info(fmt.Sprintf("nodeNum: %v. ", nodeNum) + logMsg) -// +// // for _, batchTask := range res { // logMsg := fmt.Sprintf("batchCopTask Addr: %s", batchTask.storeAddr) // for i, perTaskRegionInfo := range batchTask.regionInfos { From 259e14fdfd71db41e0a570d33e3d76ad16850123 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 28 Apr 2022 21:30:23 +0800 Subject: [PATCH 05/44] change usage of cacheRegion.GetTiFlashMPPStores() Signed-off-by: guo-shaoge --- store/copr/batch_coprocessor.go | 23 +++++++++++++---------- store/copr/mpp.go | 3 ++- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index dd69fa76f9ff1..bf25b6d8bfaee 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -599,14 +599,23 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran } } - allStores := cache.GetTiFlashMPPStores() - nodeNum := len(allStores) + mppStores, err := cache.GetTiFlashMPPStores(bo.TiKVBackoffer()) + if err != nil { + return nil, err + } + nodeNum := len(mppStores) if nodeNum == 0 { - panic("tiflash stores num is zero") + return nil, errors.New("Number of tiflash_mpp node is zero") } + var logMsg string + for i, s := range mppStores { + logMsg += fmt.Sprintf("store[%d]: %s, ", i, s.GetAddr()) + } + logutil.BgLogger().Info(fmt.Sprintf("nodeNum: %v. ", nodeNum) + logMsg) + hasher := consistent.New() - for _, store := range allStores { + for _, store := range mppStores { hasher.Add(store.GetAddr()) } copTaskNumForEachNode := len(tasks)/nodeNum + 1 @@ -649,12 +658,6 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran handledNum, len(tasks), len(taskCtxMap), nodeNum, copTaskNumForEachNode)) } - var logMsg string - for i, s := range allStores { - logMsg += fmt.Sprintf("store[%d]: %s, ", i, s.GetAddr()) - } - logutil.BgLogger().Info(fmt.Sprintf("nodeNum: %v. ", nodeNum) + logMsg) - for _, batchTask := range res { logMsg := fmt.Sprintf("batchCopTask Addr: %s", batchTask.storeAddr) for i, perTaskRegionInfo := range batchTask.regionInfos { diff --git a/store/copr/mpp.go b/store/copr/mpp.go index a2e60091dbe7d..45fc755a643c2 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -322,7 +322,8 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req if index < 10 || log.GetLevel() <= zap.DebugLevel { logutil.BgLogger().Info("invalid region because tiflash detected stale region", zap.String("region id", id.String())) } - // m.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) + m.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) + m.store.GetRegionCache().InvalidateTiFlashMPPStores() } } failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { From a6d4681a3ef929fd3259347888db5a5da93b8e3c Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Sat, 30 Apr 2022 19:58:34 +0800 Subject: [PATCH 06/44] clean buildBatchCopTasksConsistentHash() and delete useless code Signed-off-by: guo-shaoge --- store/copr/batch_coprocessor.go | 258 ++++++++------------------------ store/copr/mpp.go | 22 --- 2 files changed, 61 insertions(+), 219 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index bf25b6d8bfaee..89436d6f616e8 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -542,23 +542,23 @@ func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, store *kvStore return batchTasks, nil } +// 1. Split range by region location to build copTasks. +// 2. Get all mpp nodes, using hash(copTask.reginoID) to decide which mpp node this copTask will be dispatched to. +// 3. copTasks that is sent to the same node will be put into one batchCopTask. func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType) (res []*batchCopTask, err error) { const cmdType = tikvrpc.CmdBatchCop - var rangesLen int var retryNum int - var taskCtxMap map[*copTask]*tikv.RPCContext - var tasks []*copTask cache := store.GetRegionCache() for { + logutil.BgLogger().Info(fmt.Sprintf("buildBatchCopTasksConsistentHash retryNum: %v. ", retryNum)) retryNum++ if retryNum >= 10 { return nil, errors.New("too many times of retry to GetTiFlashRPCContext()") } - rangesLen = 0 - taskCtxMap = make(map[*copTask]*tikv.RPCContext) - tasks = make([]*copTask, 0) + var rangesLen int + tasks := make([]*copTask, 0) for i, ranges := range rangesForEachPhysicalTable { rangesLen += ranges.Len() @@ -577,219 +577,83 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran } } + mppStores, err := cache.GetTiFlashMPPStores(bo.TiKVBackoffer()) + if err != nil { + return nil, err + } + nodeNum := len(mppStores) + if nodeNum == 0 { + return nil, errors.New("Number of tiflash_mpp node is zero") + } + + var logMsg string + for i, s := range mppStores { + logMsg += fmt.Sprintf("store[%d]: %s, ", i, s.GetAddr()) + } + logutil.BgLogger().Info(fmt.Sprintf("nodeNum: %v. ", nodeNum) + logMsg) + + hasher := consistent.New() + for _, store := range mppStores { + hasher.Add(store.GetAddr()) + } + + taskMap := make(map[string][]*copTask) for _, task := range tasks { - // todo: check this func - rpcCtx, err := cache.GetTiFlashRPCContext(bo.TiKVBackoffer(), task.region, false) + addr, err := hasher.Get(strconv.Itoa(int(task.region.GetID()))) + if err != nil { + return nil, errors.Trace(err) + } + if tasks, ok := taskMap[addr]; !ok { + taskMap[addr] = []*copTask{task} + } else { + tasks = append(tasks, task) + taskMap[addr] = tasks + } + } + + var needRetry bool + res = res[:0] + for addr, tasks := range taskMap { + rpcCtx, err := cache.GetTiFlashRPCContext(bo.TiKVBackoffer(), tasks[0].region, false) if err != nil { return nil, errors.Trace(err) } if rpcCtx == nil { - // todo: looks like it happens, why - logutil.BgLogger().Info(fmt.Sprintf("GetTiFlashRPCContext failed: %v", task.region)) + // todo: looks like it happens. + logutil.BgLogger().Info(fmt.Sprintf("GetTiFlashRPCContext failed: %v", tasks[0].region)) err := bo.Backoff(tikv.BoTiFlashRPC(), errors.New("Cannot find region with TiFlash peer")) if err != nil { return nil, errors.Trace(err) } + needRetry = true break } - taskCtxMap[task] = rpcCtx - } - if len(taskCtxMap) == len(tasks) { - break - } - } - - mppStores, err := cache.GetTiFlashMPPStores(bo.TiKVBackoffer()) - if err != nil { - return nil, err - } - nodeNum := len(mppStores) - if nodeNum == 0 { - return nil, errors.New("Number of tiflash_mpp node is zero") - } - - var logMsg string - for i, s := range mppStores { - logMsg += fmt.Sprintf("store[%d]: %s, ", i, s.GetAddr()) - } - logutil.BgLogger().Info(fmt.Sprintf("nodeNum: %v. ", nodeNum) + logMsg) - - hasher := consistent.New() - for _, store := range mppStores { - hasher.Add(store.GetAddr()) - } - copTaskNumForEachNode := len(tasks)/nodeNum + 1 - var taskIdxInOneNode int - var handledNum int - for task, rpcCtx := range taskCtxMap { - rpcCtx.Addr, err = hasher.Get(strconv.Itoa(int(rpcCtx.Region.GetID()))) - regionInfo := RegionInfo{ - Region: task.region, - Meta: rpcCtx.Meta, - Ranges: task.ranges, - AllStores: cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store), - PartitionIndex: task.partitionIndex, - Addr: rpcCtx.Addr, - } - - if taskIdxInOneNode == copTaskNumForEachNode { - taskIdxInOneNode = 0 - } - - if taskIdxInOneNode == 0 { + rpcCtx.Addr = addr + regionInfos := make([]RegionInfo, 0, len(tasks)) + for _, task := range tasks { + regionInfos = append(regionInfos, RegionInfo{ + Region: task.region, + Meta: rpcCtx.Meta, + Ranges: task.ranges, + AllStores: cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store), + PartitionIndex: task.partitionIndex, + Addr: rpcCtx.Addr, + }) + } res = append(res, &batchCopTask{ storeAddr: rpcCtx.Addr, cmdType: cmdType, ctx: rpcCtx, - regionInfos: []RegionInfo{regionInfo}, + regionInfos: regionInfos, }) - } else if taskIdxInOneNode < copTaskNumForEachNode { - lastBatchCopTask := res[len(res)-1] - lastBatchCopTask.regionInfos = append(lastBatchCopTask.regionInfos, regionInfo) - } else { - panic("cannot reach here") } - handledNum++ - taskIdxInOneNode++ - } - - if handledNum != len(tasks) { - panic(fmt.Sprintf("handled %v tasks, but expect %v, len(taskCtxMap): %v, nodeNum: %v, copTaskNumForEachNode: %v", - handledNum, len(tasks), len(taskCtxMap), nodeNum, copTaskNumForEachNode)) - } - - for _, batchTask := range res { - logMsg := fmt.Sprintf("batchCopTask Addr: %s", batchTask.storeAddr) - for i, perTaskRegionInfo := range batchTask.regionInfos { - logMsg += fmt.Sprintf(", copTask[%d]: %s(regid:%v)", i, perTaskRegionInfo.Addr, perTaskRegionInfo.Region) + if !needRetry { + break } - logutil.BgLogger().Info(logMsg) } return res, nil } -// // 1. The returned batchCopTask num equals to realted TiFlash node num -// // 2. The region of first copTask in one batchCopTask is located in TiFlash specified in batchCopTask.storeAddr. -// // 3. But the remaining copTasks in one batchCopTask is randomly dispatched (using golang map). -// func buildBatchCopTasksRandomly(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType) (res []*batchCopTask, _ error) { -// const cmdType = tikvrpc.CmdBatchCop -// var rangesLen int -// var retryNum int -// var taskCtxMap map[*copTask]*tikv.RPCContext -// var tasks []*copTask -// cache := store.GetRegionCache() -// -// for { -// retryNum++ -// if retryNum >= 10 { -// return nil, errors.New("too many times of retry to GetTiFlashRPCContext()") -// } -// -// rangesLen = 0 -// taskCtxMap = make(map[*copTask]*tikv.RPCContext) -// tasks = make([]*copTask, 0) -// -// for i, ranges := range rangesForEachPhysicalTable { -// rangesLen += ranges.Len() -// locations, err := cache.SplitKeyRangesByLocations(bo, ranges) -// if err != nil { -// return nil, errors.Trace(err) -// } -// for _, lo := range locations { -// tasks = append(tasks, &copTask{ -// region: lo.Location.Region, -// ranges: lo.Ranges, -// cmdType: cmdType, -// storeType: storeType, -// partitionIndex: int64(i), -// }) -// } -// } -// -// for _, task := range tasks { -// // todo: check this func -// rpcCtx, err := cache.GetTiFlashRPCContext(bo.TiKVBackoffer(), task.region, false) -// if err != nil { -// return nil, errors.Trace(err) -// } -// if rpcCtx == nil { -// // todo: looks like it happens, why -// logutil.BgLogger().Info(fmt.Sprintf("GetTiFlashRPCContext failed: %v", task.region)) -// err := bo.Backoff(tikv.BoTiFlashRPC(), errors.New("Cannot find region with TiFlash peer")) -// if err != nil { -// return nil, errors.Trace(err) -// } -// break -// } -// taskCtxMap[task] = rpcCtx -// } -// if len(taskCtxMap) == len(tasks) { -// break -// } -// } -// -// allStores := cache.GetTiFlashStores() -// nodeNum := len(allStores) -// if nodeNum == 0 { -// panic("tiflash stores num is zero") -// } -// -// copTaskNumForEachNode := len(tasks)/nodeNum + 1 -// var taskIdxInOneNode int -// var handledNum int -// for task, rpcCtx := range taskCtxMap { -// rpcCtx.Addr = allStores[rand.Intn(len(allStores))].GetAddr() -// regionInfo := RegionInfo{ -// Region: task.region, -// Meta: rpcCtx.Meta, -// Ranges: task.ranges, -// AllStores: cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store), -// PartitionIndex: task.partitionIndex, -// Addr: rpcCtx.Addr, -// } -// -// if taskIdxInOneNode == copTaskNumForEachNode { -// taskIdxInOneNode = 0 -// } -// -// if taskIdxInOneNode == 0 { -// res = append(res, &batchCopTask{ -// storeAddr: rpcCtx.Addr, -// cmdType: cmdType, -// ctx: rpcCtx, -// regionInfos: []RegionInfo{regionInfo}, -// }) -// } else if taskIdxInOneNode < copTaskNumForEachNode { -// lastBatchCopTask := res[len(res)-1] -// lastBatchCopTask.regionInfos = append(lastBatchCopTask.regionInfos, regionInfo) -// } else { -// panic("cannot reach here") -// } -// handledNum++ -// taskIdxInOneNode++ -// } -// -// if handledNum != len(tasks) { -// panic(fmt.Sprintf("handled %v tasks, but expect %v, len(taskCtxMap): %v, nodeNum: %v, copTaskNumForEachNode: %v", -// handledNum, len(tasks), len(taskCtxMap), nodeNum, copTaskNumForEachNode)) -// } -// -// var logMsg string -// for i, s := range allStores { -// logMsg += fmt.Sprintf("store[%d]: %s, ", i, s.GetAddr()) -// } -// logutil.BgLogger().Info(fmt.Sprintf("nodeNum: %v. ", nodeNum) + logMsg) -// -// for _, batchTask := range res { -// logMsg := fmt.Sprintf("batchCopTask Addr: %s", batchTask.storeAddr) -// for i, perTaskRegionInfo := range batchTask.regionInfos { -// logMsg += fmt.Sprintf(", copTask[%d]: %s(regid:%v)", i, perTaskRegionInfo.Addr, perTaskRegionInfo.Region) -// } -// logutil.BgLogger().Info(logMsg) -// } -// return res, nil -// } - // When `partitionIDs != nil`, it means that buildBatchCopTasksCore is constructing a batch cop tasks for PartitionTableScan. // At this time, `len(rangesForEachPhysicalTable) == len(partitionIDs)` and `rangesForEachPhysicalTable[i]` is for partition `partitionIDs[i]`. // Otherwise, `rangesForEachPhysicalTable[0]` indicates the range for the single physical table. diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 45fc755a643c2..15ba16138bb25 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -16,9 +16,7 @@ package copr import ( "context" - "fmt" "io" - "math/rand" "strconv" "sync" "sync/atomic" @@ -82,7 +80,6 @@ func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasks } ranges := NewKeyRanges(req.KeyRanges) tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store, ranges, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20) - // tasks = disruptTaskAddr(tasks) } if err != nil { @@ -95,25 +92,6 @@ func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasks return mppTasks, nil } -func disruptTaskAddr(tasks []*batchCopTask) []*batchCopTask { - addrs := make([]string, 0, len(tasks)) - var logMsg string - for _, t := range tasks { - addrs = append(addrs, t.GetAddress()) - logMsg += fmt.Sprintf("%s;", t.GetAddress()) - } - logutil.BgLogger().Info("batchCopTask Addr before disrupt: " + logMsg) - - rand.Shuffle(len(addrs), func(i, j int) { addrs[i], addrs[j] = addrs[j], addrs[i] }) - logMsg = "" - for i, addr := range addrs { - tasks[i].storeAddr = addr - logMsg += fmt.Sprintf("%s;", addr) - } - logutil.BgLogger().Info("batchCopTask Addr after disrupt: " + logMsg) - return tasks -} - // mppResponse wraps mpp data packet. type mppResponse struct { pbResp *mpp.MPPDataPacket From a4708e01943d34f6961bc6d49e737546b1a1641f Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 16 May 2022 21:19:12 +0800 Subject: [PATCH 07/44] fix Signed-off-by: guo-shaoge --- executor/builder.go | 1 + executor/mpp_gather.go | 5 +- kv/kv.go | 4 + kv/mpp.go | 2 +- planner/core/fragment.go | 29 +++++--- planner/core/task.go | 9 ++- sessionctx/variable/sysvar.go | 11 +++ sessionctx/variable/varsutil.go | 22 ++++++ store/copr/batch_coprocessor.go | 114 +++++++++++------------------ store/copr/batch_request_sender.go | 32 ++++---- store/copr/coprocessor.go | 4 +- store/copr/mpp.go | 10 ++- store/copr/store.go | 2 + 13 files changed, 138 insertions(+), 107 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index f5a4a629c3660..43dc5e09f24a7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3186,6 +3186,7 @@ func (b *executorBuilder) buildMPPGather(v *plannercore.PhysicalTableReader) Exe is: b.is, originalPlan: v.GetTablePlan(), startTS: startTs, + storeType: v.StoreType, } return gather } diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 3a8ce366742b6..24695beb04e35 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -48,7 +48,8 @@ type MPPGather struct { mppReqs []*kv.MPPDispatchRequest - respIter distsql.SelectResult + respIter distsql.SelectResult + storeType kv.StoreType } func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { @@ -107,7 +108,7 @@ func (e *MPPGather) Open(ctx context.Context) (err error) { // TODO: Move the construct tasks logic to planner, so we can see the explain results. sender := e.originalPlan.(*plannercore.PhysicalExchangeSender) planIDs := collectPlanIDS(e.originalPlan, nil) - frags, err := plannercore.GenerateRootMPPTasks(e.ctx, e.startTS, sender, e.is) + frags, err := plannercore.GenerateRootMPPTasks(e.ctx, e.startTS, sender, e.is, e.storeType) if err != nil { return errors.Trace(err) } diff --git a/kv/kv.go b/kv/kv.go index 58aecb5195891..4891c2662d571 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -289,6 +289,8 @@ const ( TiFlash // TiDB means the type of a store is TiDB. TiDB + // TiFlashMPP means tiflash ReadNodes. + TiFlashMPP // UnSpecified means the store type is unknown UnSpecified = 255 ) @@ -301,6 +303,8 @@ func (t StoreType) Name() string { return "tidb" } else if t == TiKV { return "tikv" + } else if t == TiFlashMPP { + return "tiflash_mpp" } return "unspecified" } diff --git a/kv/mpp.go b/kv/mpp.go index 012a182aacd1a..033a9952f55e8 100644 --- a/kv/mpp.go +++ b/kv/mpp.go @@ -80,7 +80,7 @@ type MPPDispatchRequest struct { type MPPClient interface { // ConstructMPPTasks schedules task for a plan fragment. // TODO:: This interface will be refined after we support more executors. - ConstructMPPTasks(context.Context, *MPPBuildTasksRequest, map[string]time.Time, time.Duration) ([]MPPTaskMeta, error) + ConstructMPPTasks(context.Context, *MPPBuildTasksRequest, map[string]time.Time, time.Duration, StoreType) ([]MPPTaskMeta, error) // DispatchMPPTasks dispatches ALL mpp requests at once, and returns an iterator that transfers the data. DispatchMPPTasks(ctx context.Context, vars interface{}, reqs []*MPPDispatchRequest, needTriggerFallback bool, startTs uint64) Response diff --git a/planner/core/fragment.go b/planner/core/fragment.go index ff013fa241e78..1755cdd6759fa 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -55,20 +55,26 @@ type tasksAndFrags struct { } type mppTaskGenerator struct { - ctx sessionctx.Context - startTS uint64 - is infoschema.InfoSchema - frags []*Fragment - cache map[int]tasksAndFrags + ctx sessionctx.Context + startTS uint64 + is infoschema.InfoSchema + frags []*Fragment + cache map[int]tasksAndFrags + storeType kv.StoreType } // GenerateRootMPPTasks generate all mpp tasks and return root ones. -func GenerateRootMPPTasks(ctx sessionctx.Context, startTs uint64, sender *PhysicalExchangeSender, is infoschema.InfoSchema) ([]*Fragment, error) { +func GenerateRootMPPTasks(ctx sessionctx.Context, + startTs uint64, + sender *PhysicalExchangeSender, + is infoschema.InfoSchema, + storeType kv.StoreType) ([]*Fragment, error) { g := &mppTaskGenerator{ - ctx: ctx, - startTS: startTs, - is: is, - cache: make(map[int]tasksAndFrags), + ctx: ctx, + startTS: startTs, + is: is, + cache: make(map[int]tasksAndFrags), + storeType: storeType, } return g.generateMPPTasks(sender) } @@ -344,7 +350,8 @@ func (e *mppTaskGenerator) constructMPPTasksImpl(ctx context.Context, ts *Physic logutil.BgLogger().Warn("MPP store fail ttl is invalid", zap.Error(err)) ttl = 30 * time.Second } - metas, err := e.ctx.GetMPPClient().ConstructMPPTasks(ctx, req, e.ctx.GetSessionVars().MPPStoreLastFailTime, ttl) + + metas, err := e.ctx.GetMPPClient().ConstructMPPTasks(ctx, req, e.ctx.GetSessionVars().MPPStoreLastFailTime, ttl, e.storeType) if err != nil { return nil, errors.Trace(err) } diff --git a/planner/core/task.go b/planner/core/task.go index c5bcfdc61f3bd..7f9c2a13ffd61 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -2386,9 +2387,15 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { sender.SetChildren(t.p) sender.cost = t.cost() + eng, err := variable.GetTiFlashEngine(ctx.GetSessionVars().GetIsolationReadEngines()) + if err != nil { + // This may never happen, because we have already checked if it's valid when user set this sys var. + logutil.BgLogger().Error("unexpect tiflash engine", zap.Error(err)) + return nil + } p := PhysicalTableReader{ tablePlan: sender, - StoreType: kv.TiFlash, + StoreType: eng, }.Init(ctx, t.p.SelectBlockOffset()) p.stats = t.p.statsInfo() collectPartitionInfosFromMPPPlan(p, t.p) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 420e994f5485f..9c35a11e3a486 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -278,6 +278,8 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TiDBIsolationReadEngines, Value: strings.Join(config.GetGlobalConfig().IsolationRead.Engines, ","), Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { engines := strings.Split(normalizedValue, ",") var formatVal string + var hasTiFlash bool + var hasTiFlashMPP bool for i, engine := range engines { engine = strings.TrimSpace(engine) if i != 0 { @@ -288,12 +290,19 @@ var defaultSysVars = []*SysVar{ formatVal += kv.TiKV.Name() case strings.EqualFold(engine, kv.TiFlash.Name()): formatVal += kv.TiFlash.Name() + hasTiFlash = true case strings.EqualFold(engine, kv.TiDB.Name()): formatVal += kv.TiDB.Name() + case strings.EqualFold(engine, kv.TiFlashMPP.Name()): + formatVal += kv.TiFlashMPP.Name() + hasTiFlashMPP = true default: return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBIsolationReadEngines, normalizedValue) } } + if hasTiFlash && hasTiFlashMPP { + return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBIsolationReadEngines, "tiflash when tiflash_mpp exists.") + } return formatVal, nil }, SetSession: func(s *SessionVars, val string) error { s.IsolationReadEngines = make(map[kv.StoreType]struct{}) @@ -305,6 +314,8 @@ var defaultSysVars = []*SysVar{ s.IsolationReadEngines[kv.TiFlash] = struct{}{} case kv.TiDB.Name(): s.IsolationReadEngines[kv.TiDB] = struct{}{} + case kv.TiFlashMPP.Name(): + s.IsolationReadEngines[kv.TiFlashMPP] = struct{}{} } } return nil diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 81ca3502e54d5..83ea969807e94 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -23,6 +23,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" @@ -507,3 +508,24 @@ var GAFunction4ExpressionIndex = map[string]struct{}{ ast.VitessHash: {}, ast.TiDBShard: {}, } + +// GetTiFlashEngine retrun kv.TiFlash or kv.TiFlashMPP +// Assume tiflash/tiflash_mpp is included in tidb_isolation_read_engines, otherwise will give error. +// tiflash_mpp and tiflash cannot exists at the same time, otherwise will give error. +func GetTiFlashEngine(readEngines map[kv.StoreType]struct{}) (res kv.StoreType, _ error) { + _, hasTiFlash := readEngines[kv.TiFlash] + _, hasTiFlashMPP := readEngines[kv.TiFlashMPP] + + if hasTiFlash && hasTiFlashMPP { + return kv.UnSpecified, errors.New("tiflash and tiflash_mpp cannot exists at the same time in readEngines") + } + if !hasTiFlash && !hasTiFlashMPP { + return kv.UnSpecified, errors.New("cannot get tiflash engine, nor tiflash or tiflash_mpp exists in readEngines") + } + if hasTiFlash { + res = kv.TiFlash + } else { + res = kv.TiFlashMPP + } + return res, nil +} diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 89436d6f616e8..4532757d6e6d7 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/util/logutil" - "github.com/stathat/consistent" "github.com/tikv/client-go/v2/metrics" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" @@ -526,10 +525,22 @@ func balanceBatchCopTask(ctx context.Context, kvStore *kvStore, originalTasks [] return ret } -func buildBatchCopTasksForNonPartitionedTable(bo *backoff.Backoffer, store *kvStore, ranges *KeyRanges, storeType kv.StoreType, mppStoreLastFailTime map[string]time.Time, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) ([]*batchCopTask, error) { - // return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) - // return buildBatchCopTasksRandomly(bo, store, []*KeyRanges{ranges}, storeType) - return buildBatchCopTasksConsistentHash(bo, store, []*KeyRanges{ranges}, storeType) +func buildBatchCopTasksForNonPartitionedTable(bo *backoff.Backoffer, + store *kvStore, + ranges *KeyRanges, + storeType kv.StoreType, + mppStoreLastFailTime map[string]time.Time, + ttl time.Duration, + balanceWithContinuity bool, + balanceContinuousRegionCount int64, + engine kv.StoreType) ([]*batchCopTask, error) { + if engine == kv.TiFlash { + return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) + } else if engine == kv.TiFlashMPP { + return buildBatchCopTasksConsistentHash(bo, store, []*KeyRanges{ranges}, storeType) + } else { + return nil, errors.New(fmt.Sprint("unexpected engine type for tiflash: %v", engine)) + } } func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType, mppStoreLastFailTime map[string]time.Time, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64, partitionIDs []int64) ([]*batchCopTask, error) { @@ -559,6 +570,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran var rangesLen int tasks := make([]*copTask, 0) + regionIDs := make([]tikv.RegionVerID, 0) for i, ranges := range rangesForEachPhysicalTable { rangesLen += ranges.Len() @@ -574,82 +586,34 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran storeType: storeType, partitionIndex: int64(i), }) + regionIDs = append(regionIDs, lo.Location.Region) } } - mppStores, err := cache.GetTiFlashMPPStores(bo.TiKVBackoffer()) + rpcCtxs, err := cache.GetTiFlashMPPRPCContextByConsistentHash(bo.TiKVBackoffer(), regionIDs) if err != nil { return nil, err } - nodeNum := len(mppStores) - if nodeNum == 0 { - return nil, errors.New("Number of tiflash_mpp node is zero") - } - - var logMsg string - for i, s := range mppStores { - logMsg += fmt.Sprintf("store[%d]: %s, ", i, s.GetAddr()) - } - logutil.BgLogger().Info(fmt.Sprintf("nodeNum: %v. ", nodeNum) + logMsg) - - hasher := consistent.New() - for _, store := range mppStores { - hasher.Add(store.GetAddr()) + if rpcCtxs == nil { + // retry + continue } - - taskMap := make(map[string][]*copTask) - for _, task := range tasks { - addr, err := hasher.Get(strconv.Itoa(int(task.region.GetID()))) - if err != nil { - return nil, errors.Trace(err) - } - if tasks, ok := taskMap[addr]; !ok { - taskMap[addr] = []*copTask{task} - } else { - tasks = append(tasks, task) - taskMap[addr] = tasks - } + if len(rpcCtxs) != len(tasks) { + return nil, errors.New(fmt.Sprintf("unexpected length of rpcCtxs, expect: %v, got: %v", len(tasks), len(rpcCtxs))) } - - var needRetry bool - res = res[:0] - for addr, tasks := range taskMap { - rpcCtx, err := cache.GetTiFlashRPCContext(bo.TiKVBackoffer(), tasks[0].region, false) - if err != nil { - return nil, errors.Trace(err) - } - if rpcCtx == nil { - // todo: looks like it happens. - logutil.BgLogger().Info(fmt.Sprintf("GetTiFlashRPCContext failed: %v", tasks[0].region)) - err := bo.Backoff(tikv.BoTiFlashRPC(), errors.New("Cannot find region with TiFlash peer")) - if err != nil { - return nil, errors.Trace(err) - } - needRetry = true - break - } - rpcCtx.Addr = addr - regionInfos := make([]RegionInfo, 0, len(tasks)) - for _, task := range tasks { - regionInfos = append(regionInfos, RegionInfo{ - Region: task.region, + taskMap := make(map[string]*batchCopTask) + for i, rpcCtx := range rpcCtxs { + if batchCopTask, ok := taskMap[rpcCtx.Addr]; ok { + batchCopTask.regionInfos = append(batchCopTask.regionInfos, RegionInfo{ + Region: tasks[i].region, Meta: rpcCtx.Meta, - Ranges: task.ranges, - AllStores: cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store), - PartitionIndex: task.partitionIndex, - Addr: rpcCtx.Addr, + Ranges: tasks[i].ranges, + AllStores: []uint64{rpcCtx.Store.StoreID()}, + PartitionIndex: tasks[i].partitionIndex, }) } - res = append(res, &batchCopTask{ - storeAddr: rpcCtx.Addr, - cmdType: cmdType, - ctx: rpcCtx, - regionInfos: regionInfos, - }) - } - if !needRetry { - break } + break } return res, nil } @@ -787,7 +751,7 @@ func convertRegionInfosToPartitionTableRegions(batchTasks []*batchCopTask, parti } } -func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.Variables, option *kv.ClientSendOption) kv.Response { +func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.Variables, option *kv.ClientSendOption, storeType kv.StoreType) kv.Response { if req.KeepOrder || req.Desc { return copErrorResponse{errors.New("batch coprocessor cannot prove keep order or desc property")} } @@ -807,7 +771,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store.kvStore, keyRanges, req.StoreType, nil, 0, false, 0, partitionIDs) } else { ranges := NewKeyRanges(req.KeyRanges) - tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store.kvStore, ranges, req.StoreType, nil, 0, false, 0) + tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store.kvStore, ranges, req.StoreType, nil, 0, false, 0, storeType) } if err != nil { @@ -820,6 +784,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V vars: vars, rpcCancel: tikv.NewRPCanceller(), enableCollectExecutionInfo: option.EnableCollectExecutionInfo, + storeType: storeType, } ctx = context.WithValue(ctx, tikv.RPCCancellerCtxKey{}, it.rpcCancel) it.tasks = tasks @@ -849,6 +814,9 @@ type batchCopIterator struct { closed uint32 enableCollectExecutionInfo bool + + // For tiflash_mpp, will use consistent hashsing to dispatch batchCopTask. + storeType kv.StoreType } func (b *batchCopIterator) run(ctx context.Context) { @@ -954,7 +922,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoff.Ba ranges = append(ranges, *ran) }) } - ret, err := buildBatchCopTasksForNonPartitionedTable(bo, b.store, NewKeyRanges(ranges), b.req.StoreType, nil, 0, false, 0) + ret, err := buildBatchCopTasksForNonPartitionedTable(bo, b.store, NewKeyRanges(ranges), b.req.StoreType, nil, 0, false, 0, b.storeType) return ret, err } // Retry Partition Table Scan @@ -1006,7 +974,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *backoff.Backo if b.req.ResourceGroupTagger != nil { b.req.ResourceGroupTagger(req) } - req.StoreTp = tikvrpc.TiFlash + req.StoreTp = getEndPointType(b.storeType) logutil.BgLogger().Debug("send batch request to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.regionInfos))) resp, retry, cancel, err := sender.SendReqToAddr(bo, task.ctx, task.regionInfos, req, readTimeoutUltraLong) diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index a1e3e210ff67a..11236a30ea3d8 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -81,7 +81,7 @@ func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *tikv.RP if err != nil { cancel() ss.SetRPCError(err) - e := ss.onSendFailForBatchRegions(bo, rpcCtx, regionInfos, err) + e := ss.onSendFailForBatchRegions(bo, rpcCtx, regionInfos, err, req.StoreTp) if e != nil { return nil, false, func() {}, errors.Trace(e) } @@ -91,7 +91,7 @@ func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *tikv.RP return } -func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx *tikv.RPCContext, regionInfos []RegionInfo, err error) error { +func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx *tikv.RPCContext, regionInfos []RegionInfo, err error, storeTp tikvrpc.EndpointType) error { // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { return errors.Trace(err) @@ -99,18 +99,22 @@ func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx return tikverr.ErrTiDBShuttingDown } - // The reload region param is always true. Because that every time we try, we must - // re-build the range then re-create the batch sender. As a result, the len of "failStores" - // will change. If tiflash's replica is more than two, the "reload region" will always be false. - // Now that the batch cop and mpp has a relative low qps, it's reasonable to reload every time - // when meeting io error. - rc := RegionCache{ss.GetRegionCache()} - rc.OnSendFailForBatchRegions(bo, ctx.Store, regionInfos, true, err) + if storeTp == tikvrpc.TiFlashMPP { + ss.GetRegionCache().InvalidateTiFlashMPPStores() + } else { + // The reload region param is always true. Because that every time we try, we must + // re-build the range then re-create the batch sender. As a result, the len of "failStores" + // will change. If tiflash's replica is more than two, the "reload region" will always be false. + // Now that the batch cop and mpp has a relative low qps, it's reasonable to reload every time + // when meeting io error. + rc := RegionCache{ss.GetRegionCache()} + rc.OnSendFailForBatchRegions(bo, ctx.Store, regionInfos, true, err) - // Retry on send request failure when it's not canceled. - // When a store is not available, the leader of related region should be elected quickly. - // TODO: the number of retry time should be limited:since region may be unavailable - // when some unrecoverable disaster happened. - err = bo.Backoff(tikv.BoTiFlashRPC(), errors.Errorf("send request error: %v, ctx: %v, regionInfos: %v", err, ctx, regionInfos)) + // Retry on send request failure when it's not canceled. + // When a store is not available, the leader of related region should be elected quickly. + // TODO: the number of retry time should be limited:since region may be unavailable + // when some unrecoverable disaster happened. + err = bo.Backoff(tikv.BoTiFlashRPC(), errors.Errorf("send request error: %v, ctx: %v, regionInfos: %v", err, ctx, regionInfos)) + } return errors.Trace(err) } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index b3c98402b5380..4a08166e9998d 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -78,9 +78,9 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa if !ok { return copErrorResponse{errors.Errorf("unsupported variables:%+v", variables)} } - if req.StoreType == kv.TiFlash && req.BatchCop { + if (req.StoreType == kv.TiFlash || req.StoreType == kv.TiFlashMPP) && req.BatchCop { logutil.BgLogger().Debug("send batch requests") - return c.sendBatch(ctx, req, vars, option) + return c.sendBatch(ctx, req, vars, option, req.StoreType) } if req.Streaming && req.Paging { return copErrorResponse{errors.New("streaming and paging are both on")} diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 15ba16138bb25..351b78332e03c 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -61,7 +61,11 @@ func (c *MPPClient) selectAllTiFlashStore() []kv.MPPTaskMeta { } // ConstructMPPTasks receives ScheduleRequest, which are actually collects of kv ranges. We allocates MPPTaskMeta for them and returns. -func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest, mppStoreLastFailTime map[string]time.Time, ttl time.Duration) ([]kv.MPPTaskMeta, error) { +func (c *MPPClient) ConstructMPPTasks(ctx context.Context, + req *kv.MPPBuildTasksRequest, + mppStoreLastFailTime map[string]time.Time, + ttl time.Duration, + engine kv.StoreType) ([]kv.MPPTaskMeta, error) { ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTS) bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) var tasks []*batchCopTask @@ -79,7 +83,7 @@ func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasks return c.selectAllTiFlashStore(), nil } ranges := NewKeyRanges(req.KeyRanges) - tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store, ranges, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20) + tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store, ranges, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20, engine) } if err != nil { @@ -301,8 +305,8 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req logutil.BgLogger().Info("invalid region because tiflash detected stale region", zap.String("region id", id.String())) } m.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) - m.store.GetRegionCache().InvalidateTiFlashMPPStores() } + m.store.GetRegionCache().InvalidateTiFlashMPPStores() } failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { if val.(bool) && !req.IsRoot { diff --git a/store/copr/store.go b/store/copr/store.go index 1783ee294f8e1..8650625602b93 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -120,6 +120,8 @@ func getEndPointType(t kv.StoreType) tikvrpc.EndpointType { return tikvrpc.TiFlash case kv.TiDB: return tikvrpc.TiDB + case kv.TiFlashMPP: + return tikvrpc.TiFlashMPP default: return tikvrpc.TiKV } From 17e6abf7af83aee447ead42719b9e191cfad61ae Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 16 May 2022 21:48:58 +0800 Subject: [PATCH 08/44] fix Signed-off-by: guo-shaoge --- store/copr/mpp.go | 1 + 1 file changed, 1 insertion(+) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 9545de822e314..cc78ac4b05c6f 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -379,6 +379,7 @@ func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchReques } else { m.sendError(err) } + m.store.GetRegionCache().InvalidateTiFlashMPPStores() return } From e7445bc7e9e8b4f65bfcd88ef38bc9ef7f942c2a Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 17 May 2022 12:51:24 +0800 Subject: [PATCH 09/44] only coprocessor.go and mpp.go will take care of kv.TiFlashMPP. Signed-off-by: guo-shaoge --- executor/builder.go | 1 - executor/mpp_gather.go | 5 ++--- executor/table_reader.go | 15 +++++++++++++-- kv/kv.go | 7 +++++++ planner/core/fragment.go | 34 ++++++++++++++++----------------- planner/core/task.go | 9 +-------- store/copr/batch_coprocessor.go | 33 +++++++++++++++++++++----------- store/copr/coprocessor.go | 2 +- 8 files changed, 63 insertions(+), 43 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index d965f6d3d370a..00ef7ce6f2cdb 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3266,7 +3266,6 @@ func (b *executorBuilder) buildMPPGather(v *plannercore.PhysicalTableReader) Exe is: b.is, originalPlan: v.GetTablePlan(), startTS: startTs, - storeType: v.StoreType, } return gather } diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 24695beb04e35..3a8ce366742b6 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -48,8 +48,7 @@ type MPPGather struct { mppReqs []*kv.MPPDispatchRequest - respIter distsql.SelectResult - storeType kv.StoreType + respIter distsql.SelectResult } func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { @@ -108,7 +107,7 @@ func (e *MPPGather) Open(ctx context.Context) (err error) { // TODO: Move the construct tasks logic to planner, so we can see the explain results. sender := e.originalPlan.(*plannercore.PhysicalExchangeSender) planIDs := collectPlanIDS(e.originalPlan, nil) - frags, err := plannercore.GenerateRootMPPTasks(e.ctx, e.startTS, sender, e.is, e.storeType) + frags, err := plannercore.GenerateRootMPPTasks(e.ctx, e.startTS, sender, e.is) if err != nil { return errors.Trace(err) } diff --git a/executor/table_reader.go b/executor/table_reader.go index a9decdd55b1a7..91744b5da9ff0 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/parser/model" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -314,12 +315,22 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra return result, nil } -func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges []*ranger.Range) ([]*kv.Request, error) { +func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges []*ranger.Range) (_ []*kv.Request, err error) { pids, kvRanges, err := e.kvRangeBuilder.buildKeyRangeSeparately(ranges) if err != nil { return nil, err } kvReqs := make([]*kv.Request, 0, len(kvRanges)) + + // If storeType is TiFlash and tidb_isolation_read_engine is "tiflash_mpp", + // then kvReq wil only be sent to tiflash_mpp nodes. + storeType := e.storeType + if storeType == kv.TiFlash { + storeType, err = variable.GetTiFlashEngine(e.ctx.GetSessionVars().GetIsolationReadEngines()) + if err != nil { + return nil, err + } + } for i, kvRange := range kvRanges { e.kvRanges = append(e.kvRanges, kvRange...) if err := updateExecutorTableID(ctx, e.dagPB.RootExecutor, true, []int64{pids[i]}); err != nil { @@ -337,7 +348,7 @@ func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges [ SetFromSessionVars(e.ctx.GetSessionVars()). SetFromInfoSchema(e.ctx.GetInfoSchema()). SetMemTracker(e.memTracker). - SetStoreType(e.storeType). + SetStoreType(storeType). SetAllowBatchCop(e.batchCop).Build() if err != nil { return nil, err diff --git a/kv/kv.go b/kv/kv.go index 4891c2662d571..ca0c9f1958176 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -289,8 +289,15 @@ const ( TiFlash // TiDB means the type of a store is TiDB. TiDB + // TiFlashMPP means tiflash ReadNodes. + // WATCHOUT: This label should only be used: + // 1. sysvar.go: when user set tidb_isolation_read_engines. + // 2. coprocessor.go/mpp.go: when send copTask/batchCopTask/mppTask to tiflash. + // Other modules in TiDB should only use kv.TiFlash, e.g., they don't distinguish ReadNodes/WriteNodes. + // So a lot of unnecessary modifications can be omitted. TiFlashMPP + // UnSpecified means the store type is unknown UnSpecified = 255 ) diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 1755cdd6759fa..9d978d65798a3 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" @@ -55,26 +56,20 @@ type tasksAndFrags struct { } type mppTaskGenerator struct { - ctx sessionctx.Context - startTS uint64 - is infoschema.InfoSchema - frags []*Fragment - cache map[int]tasksAndFrags - storeType kv.StoreType + ctx sessionctx.Context + startTS uint64 + is infoschema.InfoSchema + frags []*Fragment + cache map[int]tasksAndFrags } // GenerateRootMPPTasks generate all mpp tasks and return root ones. -func GenerateRootMPPTasks(ctx sessionctx.Context, - startTs uint64, - sender *PhysicalExchangeSender, - is infoschema.InfoSchema, - storeType kv.StoreType) ([]*Fragment, error) { +func GenerateRootMPPTasks(ctx sessionctx.Context, startTs uint64, sender *PhysicalExchangeSender, is infoschema.InfoSchema) ([]*Fragment, error) { g := &mppTaskGenerator{ - ctx: ctx, - startTS: startTs, - is: is, - cache: make(map[int]tasksAndFrags), - storeType: storeType, + ctx: ctx, + startTS: startTs, + is: is, + cache: make(map[int]tasksAndFrags), } return g.generateMPPTasks(sender) } @@ -351,7 +346,12 @@ func (e *mppTaskGenerator) constructMPPTasksImpl(ctx context.Context, ts *Physic ttl = 30 * time.Second } - metas, err := e.ctx.GetMPPClient().ConstructMPPTasks(ctx, req, e.ctx.GetSessionVars().MPPStoreLastFailTime, ttl, e.storeType) + // If tidb_isolation_read_engine is "tiflash_mpp", then tasks will only be sent to ReadNodes. + storeType, err := variable.GetTiFlashEngine(e.ctx.GetSessionVars().GetIsolationReadEngines()) + if err != nil { + return nil, err + } + metas, err := e.ctx.GetMPPClient().ConstructMPPTasks(ctx, req, e.ctx.GetSessionVars().MPPStoreLastFailTime, ttl, storeType) if err != nil { return nil, errors.Trace(err) } diff --git a/planner/core/task.go b/planner/core/task.go index 015952008f427..6684d26b80534 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -1981,15 +1980,9 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { sender.SetChildren(t.p) sender.cost = t.cost() - eng, err := variable.GetTiFlashEngine(ctx.GetSessionVars().GetIsolationReadEngines()) - if err != nil { - // This may never happen, because we have already checked if it's valid when user set this sys var. - logutil.BgLogger().Error("unexpect tiflash engine", zap.Error(err)) - return nil - } p := PhysicalTableReader{ tablePlan: sender, - StoreType: eng, + StoreType: kv.TiFlash, }.Init(ctx, t.p.SelectBlockOffset()) p.stats = t.p.statsInfo() collectPartitionInfosFromMPPPlan(p, t.p) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 4532757d6e6d7..b741c234317a6 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -554,8 +554,8 @@ func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, store *kvStore } // 1. Split range by region location to build copTasks. -// 2. Get all mpp nodes, using hash(copTask.reginoID) to decide which mpp node this copTask will be dispatched to. -// 3. copTasks that is sent to the same node will be put into one batchCopTask. +// 2. For each copTask build its rpcCtx , the target ReadNode will be computed using consistent hash. +// 3. All copTasks that will be sent to one ReadNode are put in one batchCopTask. func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType) (res []*batchCopTask, err error) { const cmdType = tikvrpc.CmdBatchCop var retryNum int @@ -595,7 +595,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran return nil, err } if rpcCtxs == nil { - // retry + // Retry. continue } if len(rpcCtxs) != len(tasks) { @@ -603,14 +603,25 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran } taskMap := make(map[string]*batchCopTask) for i, rpcCtx := range rpcCtxs { - if batchCopTask, ok := taskMap[rpcCtx.Addr]; ok { - batchCopTask.regionInfos = append(batchCopTask.regionInfos, RegionInfo{ - Region: tasks[i].region, - Meta: rpcCtx.Meta, - Ranges: tasks[i].ranges, - AllStores: []uint64{rpcCtx.Store.StoreID()}, - PartitionIndex: tasks[i].partitionIndex, - }) + regionInfo := RegionInfo{ + // tasks and rpcCtxs are correspond to each other. + Region: tasks[i].region, + Meta: rpcCtx.Meta, + Ranges: tasks[i].ranges, + AllStores: []uint64{rpcCtx.Store.StoreID()}, + PartitionIndex: tasks[i].partitionIndex, + } + if batchTask, ok := taskMap[rpcCtx.Addr]; ok { + batchTask.regionInfos = append(batchTask.regionInfos, regionInfo) + } else { + batchTask := &batchCopTask{ + storeAddr: rpcCtx.Addr, + cmdType: cmdType, + ctx: rpcCtx, + regionInfos: []RegionInfo{regionInfo}, + } + taskMap[rpcCtx.Addr] = batchTask + res = append(res, batchTask) } } break diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 9fd854890e497..de9b4e4ba0011 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -881,7 +881,7 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *Backoffer, rpcCtx *ti } err1 := errors.Errorf("recv stream response error: %v, task: %s", err, task) - if task.storeType == kv.TiFlash { + if task.storeType == kv.TiFlash || task.storeType == kv.TiFlashMPP { err1 = bo.Backoff(tikv.BoTiFlashRPC(), err1) } else { err1 = bo.Backoff(tikv.BoTiKVRPC(), err1) From 788422a5a9f9950f51686057c079240aa245b4d3 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 17 May 2022 13:25:35 +0800 Subject: [PATCH 10/44] fix comment Signed-off-by: guo-shaoge --- kv/kv.go | 4 ++-- sessionctx/variable/varsutil.go | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/kv/kv.go b/kv/kv.go index ca0c9f1958176..eecf886486171 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -291,10 +291,10 @@ const ( TiDB // TiFlashMPP means tiflash ReadNodes. - // WATCHOUT: This label should only be used: + // WATCHOUT: This label should only be used in: // 1. sysvar.go: when user set tidb_isolation_read_engines. // 2. coprocessor.go/mpp.go: when send copTask/batchCopTask/mppTask to tiflash. - // Other modules in TiDB should only use kv.TiFlash, e.g., they don't distinguish ReadNodes/WriteNodes. + // Other modules in TiDB should only use kv.TiFlash, they don't distinguish ReadNodes and WriteNodes. // So a lot of unnecessary modifications can be omitted. TiFlashMPP diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index df196aade5006..b87200c3d4794 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -514,15 +514,15 @@ var GAFunction4ExpressionIndex = map[string]struct{}{ ast.TiDBShard: {}, } -// GetTiFlashEngine retrun kv.TiFlash or kv.TiFlashMPP -// Assume tiflash/tiflash_mpp is included in tidb_isolation_read_engines, otherwise will give error. +// GetTiFlashEngine retrun kv.TiFlash or kv.TiFlashMPP. +// Assume tiflash or tiflash_mpp is included in tidb_isolation_read_engines, otherwise will give error. // tiflash_mpp and tiflash cannot exists at the same time, otherwise will give error. func GetTiFlashEngine(readEngines map[kv.StoreType]struct{}) (res kv.StoreType, _ error) { _, hasTiFlash := readEngines[kv.TiFlash] _, hasTiFlashMPP := readEngines[kv.TiFlashMPP] if hasTiFlash && hasTiFlashMPP { - return kv.UnSpecified, errors.New("tiflash and tiflash_mpp cannot exists at the same time in readEngines") + return kv.UnSpecified, errors.New("tiflash and tiflash_mpp cannot exist at the same time in readEngines") } if !hasTiFlash && !hasTiFlashMPP { return kv.UnSpecified, errors.New("cannot get tiflash engine, nor tiflash or tiflash_mpp exists in readEngines") From 52be11ea7941b2e2834b620ead35b266ebd76b8a Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 18 May 2022 11:46:56 +0800 Subject: [PATCH 11/44] add failpoint. change semantics of tidb_isolation_read_engines Signed-off-by: guo-shaoge --- sessionctx/variable/sysvar.go | 4 ++-- sessionctx/variable/varsutil.go | 14 ++++++-------- store/copr/batch_coprocessor.go | 24 ++++++++++++++++++++++++ 3 files changed, 32 insertions(+), 10 deletions(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index bc7e4d6a5a059..09db861995389 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -268,8 +268,8 @@ var defaultSysVars = []*SysVar{ return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBIsolationReadEngines, normalizedValue) } } - if hasTiFlash && hasTiFlashMPP { - return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBIsolationReadEngines, "tiflash when tiflash_mpp exists.") + if hasTiFlashMPP && !hasTiFlash { + return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBIsolationReadEngines, "tiflash must be set when tiflash_mpp exists.") } return formatVal, nil }, SetSession: func(s *SessionVars, val string) error { diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index b87200c3d4794..6c71349b59c96 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -515,22 +515,20 @@ var GAFunction4ExpressionIndex = map[string]struct{}{ } // GetTiFlashEngine retrun kv.TiFlash or kv.TiFlashMPP. -// Assume tiflash or tiflash_mpp is included in tidb_isolation_read_engines, otherwise will give error. -// tiflash_mpp and tiflash cannot exists at the same time, otherwise will give error. func GetTiFlashEngine(readEngines map[kv.StoreType]struct{}) (res kv.StoreType, _ error) { _, hasTiFlash := readEngines[kv.TiFlash] _, hasTiFlashMPP := readEngines[kv.TiFlashMPP] - if hasTiFlash && hasTiFlashMPP { - return kv.UnSpecified, errors.New("tiflash and tiflash_mpp cannot exist at the same time in readEngines") - } if !hasTiFlash && !hasTiFlashMPP { return kv.UnSpecified, errors.New("cannot get tiflash engine, nor tiflash or tiflash_mpp exists in readEngines") } - if hasTiFlash { - res = kv.TiFlash - } else { + if hasTiFlashMPP && !hasTiFlash { + return kv.UnSpecified, errors.New("tiflash must be set when tiflash_mpp exists in readEngines") + } + if hasTiFlashMPP { res = kv.TiFlashMPP + } else { + res = kv.TiFlash } return res, nil } diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index b741c234317a6..21aa9825edf50 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -22,6 +22,7 @@ import ( "math" "sort" "strconv" + "strings" "sync" "sync/atomic" "time" @@ -626,6 +627,29 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran } break } + + failpoint.Inject("check_only_dispatched_to_tiflash_mpp_nodes", func(val failpoint.Value) { + // This failpoint will be tested in test-infra case, because it needs setup a cluster. + str := val.(string) + addrs := strings.Split(str, ";") + if len(addrs) < 1 { + panic(fmt.Sprintf("unexpected length of tiflash_mpp node addrs: %v", len(addrs))) + } + for _, batchTask := range res { + var matched bool + for _, addr := range addrs { + if batchTask.storeAddr == addr { + matched = true + break + } + } + if !matched { + panic(fmt.Sprintf("batchCopTask send to node which is not tiflash_mpp: %v(tiflash_mpp nodes: %s)", + batchTask.storeAddr, str)) + } + } + logutil.BgLogger().Info(fmt.Sprintf("check_only_dispatched_to_tiflash_mpp_nodes checked succeed(tiflash_mpp nodes: %s)", str)) + }) return res, nil } From e61060455c371a3be0a1ab22d5c32ba90167842d Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 18 May 2022 12:00:03 +0800 Subject: [PATCH 12/44] change some comments Signed-off-by: guo-shaoge --- kv/kv.go | 2 +- planner/core/fragment.go | 2 +- store/copr/batch_coprocessor.go | 3 ++- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/kv/kv.go b/kv/kv.go index eecf886486171..65cd7c0a94edb 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -295,7 +295,7 @@ const ( // 1. sysvar.go: when user set tidb_isolation_read_engines. // 2. coprocessor.go/mpp.go: when send copTask/batchCopTask/mppTask to tiflash. // Other modules in TiDB should only use kv.TiFlash, they don't distinguish ReadNodes and WriteNodes. - // So a lot of unnecessary modifications can be omitted. + // So we can avoid to change kv.TiFlashMPP in all code pathes where kv.TiFlash occurs. TiFlashMPP // UnSpecified means the store type is unknown diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 9d978d65798a3..15c564822d4ed 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -346,7 +346,7 @@ func (e *mppTaskGenerator) constructMPPTasksImpl(ctx context.Context, ts *Physic ttl = 30 * time.Second } - // If tidb_isolation_read_engine is "tiflash_mpp", then tasks will only be sent to ReadNodes. + // If "tiflash_mpp" is set in tidb_isolation_read_engines, tasks will only be sent to ReadNodes when building batchCopTask. storeType, err := variable.GetTiFlashEngine(e.ctx.GetSessionVars().GetIsolationReadEngines()) if err != nil { return nil, err diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 21aa9825edf50..dffef6d099b73 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -629,7 +629,8 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran } failpoint.Inject("check_only_dispatched_to_tiflash_mpp_nodes", func(val failpoint.Value) { - // This failpoint will be tested in test-infra case, because it needs setup a cluster. + // This failpoint will be tested in test-infra case, because we needs setup a cluster. + // All ReadNode addrs are stored in val, each addr is seperated by semicolon. str := val.(string) addrs := strings.Split(str, ";") if len(addrs) < 1 { From fe2c235f7abe449473acc9df0dfa4c14e32784c9 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 19 May 2022 12:19:14 +0800 Subject: [PATCH 13/44] only invalid tiflash_mpp node cache if is grpc error Signed-off-by: guo-shaoge --- executor/mpp_gather.go | 9 ++++++ executor/table_reader.go | 3 +- kv/mpp.go | 19 +++++++++---- store/copr/batch_request_sender.go | 2 +- store/copr/coprocessor.go | 1 + store/copr/mpp.go | 44 ++++++++++++++++++++++++------ store/copr/store.go | 8 ++++++ 7 files changed, 69 insertions(+), 17 deletions(-) diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 3a8ce366742b6..25371eef8b123 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-tipb" @@ -52,6 +53,13 @@ type MPPGather struct { } func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { + // If storeType is TiFlash and tidb_isolation_read_engine is "tiflash_mpp", + // then batchCopTask will only be sent to tiflash_mpp nodes. + // Will be checked when send batchCopTask. + storeType, err := variable.GetTiFlashEngine(e.ctx.GetSessionVars().GetIsolationReadEngines()) + if err != nil { + return err + } dagReq, _, err := constructDAGReq(e.ctx, []plannercore.PhysicalPlan{pf.ExchangeSender}, kv.TiFlash) if err != nil { return errors.Trace(err) @@ -87,6 +95,7 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { SchemaVar: e.is.SchemaMetaVersion(), StartTs: e.startTS, State: kv.MppTaskReady, + StoreTp: storeType, } e.mppReqs = append(e.mppReqs, req) } diff --git a/executor/table_reader.go b/executor/table_reader.go index 91744b5da9ff0..9a2be51bdeaa3 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -323,7 +323,8 @@ func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges [ kvReqs := make([]*kv.Request, 0, len(kvRanges)) // If storeType is TiFlash and tidb_isolation_read_engine is "tiflash_mpp", - // then kvReq wil only be sent to tiflash_mpp nodes. + // then copTask will only be sent to tiflash_mpp nodes. + // Will be checked when sending copTask. storeType := e.storeType if storeType == kv.TiFlash { storeType, err = variable.GetTiFlashEngine(e.ctx.GetSessionVars().GetIsolationReadEngines()) diff --git a/kv/mpp.go b/kv/mpp.go index 033a9952f55e8..311cb0e984574 100644 --- a/kv/mpp.go +++ b/kv/mpp.go @@ -65,15 +65,22 @@ const ( // MPPDispatchRequest stands for a dispatching task. type MPPDispatchRequest struct { - Data []byte // data encodes the dag coprocessor request. - Meta MPPTaskMeta // mpp store is the location of tiflash store. - IsRoot bool // root task returns data to tidb directly. - Timeout uint64 // If task is assigned but doesn't receive a connect request during timeout, the task should be destroyed. + // data encodes the dag coprocessor request. + Data []byte + // mpp store is the location of tiflash store. + Meta MPPTaskMeta + // root task returns data to tidb directly. + IsRoot bool + // If task is assigned but doesn't receive a connect request during timeout, the task should be destroyed. + Timeout uint64 // SchemaVer is for any schema-ful storage (like tiflash) to validate schema correctness if necessary. SchemaVar int64 StartTs uint64 - ID int64 // identify a single task - State MppTaskStates + // Identify a single task. + ID int64 + State MppTaskStates + // Check is tiflash or tiflash_mpp. + StoreTp StoreType } // MPPClient accepts and processes mpp requests. diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index 11236a30ea3d8..5af31cc294d20 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -100,7 +100,7 @@ func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx } if storeTp == tikvrpc.TiFlashMPP { - ss.GetRegionCache().InvalidateTiFlashMPPStores() + ss.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) } else { // The reload region param is always true. Because that every time we try, we must // re-build the range then re-create the batch sender. As a result, the len of "failStores" diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index de9b4e4ba0011..962d683ed3c5d 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -78,6 +78,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa if !ok { return copErrorResponse{errors.Errorf("unsupported variables:%+v", variables)} } + // req.StoreType is setup in TableReader. if (req.StoreType == kv.TiFlash || req.StoreType == kv.TiFlashMPP) && req.BatchCop { logutil.BgLogger().Debug("send batch requests") return c.sendBatch(ctx, req, vars, option, req.StoreType) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index cc78ac4b05c6f..228badaff1faf 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -239,7 +239,12 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req } wrappedReq := tikvrpc.NewRequest(tikvrpc.CmdMPPTask, mppReq, kvrpcpb.Context{}) - wrappedReq.StoreTp = tikvrpc.TiFlash + if tp, err := getTiFlashEndPointType(req.StoreTp); err != nil { + m.sendError(err) + return + } else { + wrappedReq.StoreTp = tp + } // TODO: Handle dispatch task response correctly, including retry logic and cancel logic. var rpcResp *tikvrpc.Response @@ -251,6 +256,9 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req if originalTask != nil { sender := NewRegionBatchRequestSender(m.store.GetRegionCache(), m.store.GetTiKVClient(), m.enableCollectExecutionInfo) rpcResp, retry, _, err = sender.SendReqToAddr(bo, originalTask.ctx, originalTask.regionInfos, wrappedReq, tikv.ReadTimeoutMedium) + if err != nil && req.StoreTp == kv.TiFlashMPP { + m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) + } // No matter what the rpc error is, we won't retry the mpp dispatch tasks. // TODO: If we want to retry, we must redo the plan fragment cutting and task scheduling. // That's a hard job but we can try it in the future. @@ -268,6 +276,9 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { retry = false } else if err != nil { + if req.StoreTp == kv.TiFlashMPP { + m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) + } if bo.Backoff(tikv.BoTiFlashRPC(), err) == nil { retry = true } @@ -306,7 +317,6 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req } m.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) } - m.store.GetRegionCache().InvalidateTiFlashMPPStores() } failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { if val.(bool) && !req.IsRoot { @@ -332,13 +342,19 @@ func (m *mppIterator) cancelMppTasks() { } wrappedReq := tikvrpc.NewRequest(tikvrpc.CmdMPPCancel, killReq, kvrpcpb.Context{}) - wrappedReq.StoreTp = tikvrpc.TiFlash - usedStoreAddrs := make(map[string]bool) + // Value type is kv.StoreType, so we can decide whether to invalid tiflash_mpp store cache or not. + usedStoreAddrs := make(map[string]kv.StoreType) for _, task := range m.tasks { // get the store address of running tasks - if task.State == kv.MppTaskRunning && !usedStoreAddrs[task.Meta.GetAddress()] { - usedStoreAddrs[task.Meta.GetAddress()] = true + if _, ok := usedStoreAddrs[task.Meta.GetAddress()]; ok && task.State == kv.MppTaskRunning { + usedStoreAddrs[task.Meta.GetAddress()] = task.StoreTp + if tp, err := getTiFlashEndPointType(task.StoreTp); err != nil { + m.sendError(err) + return + } else { + wrappedReq.StoreTp = tp + } } else if task.State == kv.MppTaskCancelled { return } @@ -346,10 +362,13 @@ func (m *mppIterator) cancelMppTasks() { } // send cancel cmd to all stores where tasks run - for addr := range usedStoreAddrs { + for addr, storeTp := range usedStoreAddrs { _, err := m.store.GetTiKVClient().SendRequest(context.Background(), addr, wrappedReq, tikv.ReadTimeoutShort) logutil.BgLogger().Debug("cancel task ", zap.Uint64("query id ", m.startTs), zap.String(" on addr ", addr)) if err != nil { + if storeTp == kv.TiFlashMPP { + m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) + } logutil.BgLogger().Error("cancel task error: ", zap.Error(err), zap.Uint64(" for query id ", m.startTs), zap.String(" on addr ", addr)) } } @@ -365,7 +384,12 @@ func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchReques } wrappedReq := tikvrpc.NewRequest(tikvrpc.CmdMPPConn, connReq, kvrpcpb.Context{}) - wrappedReq.StoreTp = tikvrpc.TiFlash + if tp, err := getTiFlashEndPointType(req.StoreTp); err != nil { + m.sendError(err) + return + } else { + wrappedReq.StoreTp = tp + } // Drain results from root task. // We don't need to process any special error. When we meet errors, just let it fail. @@ -379,7 +403,9 @@ func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchReques } else { m.sendError(err) } - m.store.GetRegionCache().InvalidateTiFlashMPPStores() + if req.StoreTp == kv.TiFlashMPP { + m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) + } return } diff --git a/store/copr/store.go b/store/copr/store.go index 79c4fba1ba824..73060cceb4e2b 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -16,6 +16,7 @@ package copr import ( "context" + "fmt" "math/rand" "sync/atomic" "time" @@ -131,6 +132,13 @@ func getEndPointType(t kv.StoreType) tikvrpc.EndpointType { return tikvrpc.TiKV } } +func getTiFlashEndPointType(t kv.StoreType) (tikvrpc.EndpointType, error) { + tp := getEndPointType(t) + if tp != tikvrpc.TiFlash && tp != tikvrpc.TiFlashMPP { + return tp, errors.New(fmt.Sprintf("unexpected endpoint tp, expect tiflash or tiflash_mpp, got: %v", tp)) + } + return tp, nil +} // Backoffer wraps tikv.Backoffer and converts the error which returns by the functions of tikv.Backoffer to tidb error. type Backoffer = backoff.Backoffer From 3c128b4e88f0d235b2f88d9eb8f83ae85dbb1535 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 19 May 2022 16:25:15 +0800 Subject: [PATCH 14/44] add failpoint check_store_type_of_batch_cop_task Signed-off-by: guo-shaoge --- sessionctx/variable/sysvar.go | 2 +- store/copr/batch_request_sender.go | 21 +++++++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 09db861995389..e502d27b63fe7 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -269,7 +269,7 @@ var defaultSysVars = []*SysVar{ } } if hasTiFlashMPP && !hasTiFlash { - return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBIsolationReadEngines, "tiflash must be set when tiflash_mpp exists.") + return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBIsolationReadEngines, "tiflash_mpp(tiflash must be set at the same time)") } return formatVal, nil }, SetSession: func(s *SessionVars, val string) error { diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index 5af31cc294d20..4b422156e377e 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -16,11 +16,14 @@ package copr import ( "context" + "fmt" "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/util/logutil" tikverr "github.com/tikv/client-go/v2/error" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" @@ -65,6 +68,24 @@ func NewRegionBatchRequestSender(cache *RegionCache, client tikv.Client, enableC // SendReqToAddr send batch cop request func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *tikv.RPCContext, regionInfos []RegionInfo, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { + failpoint.Inject("check_store_type_of_batch_cop_task", func(val failpoint.Value) { + // Will be tested in test-infra. + storeTpStr := val.(string) + expectStoreTp := tikvrpc.TiFlash + switch storeTpStr { + case "tiflash": + expectStoreTp = tikvrpc.TiFlash + case "tiflash_mpp": + expectStoreTp = tikvrpc.TiFlashMPP + default: + panic("unexpected failpoint val, must be tiflash or tiflash_mpp") + } + if expectStoreTp != req.StoreTp { + panic(fmt.Sprintf("unexpected store type, expect: %v, got: %v", expectStoreTp, req.StoreTp)) + } + logutil.BgLogger().Info("failpoint check_store_type_of_batch_cop_task succeed") + }) + cancel = func() {} if e := tikvrpc.SetContext(req, rpcCtx.Meta, rpcCtx.Peer); e != nil { return nil, false, cancel, errors.Trace(e) From 33af6b0184b0bad8a36e37d9961223410514b430 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Sun, 22 May 2022 23:21:10 +0800 Subject: [PATCH 15/44] fix partition table dispatching Signed-off-by: guo-shaoge --- executor/table_reader.go | 52 ++++++++++++++++++++------------- store/copr/batch_coprocessor.go | 25 ++++++++++++---- store/copr/mpp.go | 2 +- 3 files changed, 52 insertions(+), 27 deletions(-) diff --git a/executor/table_reader.go b/executor/table_reader.go index 9a2be51bdeaa3..e0a503c03343a 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -193,13 +193,17 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { // [9734095886065816709 9734095886065816708 9734095886065816707 65535 3 1] => rows reverse in UnionScan firstPartRanges, secondPartRanges = secondPartRanges, firstPartRanges } - kvReq, err := e.buildKVReq(ctx, firstPartRanges) + storeType, err := e.considerTiFlashMPPStoreType() + if err != nil { + return err + } + kvReq, err := e.buildKVReq(ctx, firstPartRanges, storeType) if err != nil { return err } e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...) if len(secondPartRanges) != 0 { - kvReq, err = e.buildKVReq(ctx, secondPartRanges) + kvReq, err = e.buildKVReq(ctx, secondPartRanges, storeType) if err != nil { return err } @@ -270,13 +274,29 @@ func (e *TableReaderExecutor) Close() error { return err } +func (e *TableReaderExecutor) considerTiFlashMPPStoreType() (_ kv.StoreType, err error) { + // If storeType is TiFlash and tidb_isolation_read_engine is "tiflash_mpp", + // then copTask will only be sent to tiflash_mpp nodes. + // Ideally, storeType should be setup ok at the planner phase rather than here. + // But this will cause too many changes, so kv.TiFlashMPP will only be considered in TableReader/MPPGather. + storeType := e.storeType + if storeType == kv.TiFlash { + storeType, err = variable.GetTiFlashEngine(e.ctx.GetSessionVars().GetIsolationReadEngines()) + } + return storeType, err +} + // buildResp first builds request and sends it to tikv using distsql.Select. It uses SelectResult returned by the callee // to fetch all results. -func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (distsql.SelectResult, error) { +func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (_ distsql.SelectResult, err error) { + storeType, err := e.considerTiFlashMPPStoreType() + if err != nil { + return nil, err + } if e.storeType == kv.TiFlash && e.kvRangeBuilder != nil { if !e.batchCop { // TiFlash cannot support to access multiple tables/partitions within one KVReq, so we have to build KVReq for each partition separately. - kvReqs, err := e.buildKVReqSeparately(ctx, ranges) + kvReqs, err := e.buildKVReqSeparately(ctx, ranges, storeType) if err != nil { return nil, err } @@ -291,7 +311,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra return distsql.NewSerialSelectResults(results), nil } // Use PartitionTable Scan - kvReq, err := e.buildKVReqForPartitionTableScan(ctx, ranges) + kvReq, err := e.buildKVReqForPartitionTableScan(ctx, ranges, storeType) if err != nil { return nil, err } @@ -302,7 +322,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra return result, nil } - kvReq, err := e.buildKVReq(ctx, ranges) + kvReq, err := e.buildKVReq(ctx, ranges, storeType) if err != nil { return nil, err } @@ -315,23 +335,13 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra return result, nil } -func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges []*ranger.Range) (_ []*kv.Request, err error) { +func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges []*ranger.Range, storeType kv.StoreType) (_ []*kv.Request, err error) { pids, kvRanges, err := e.kvRangeBuilder.buildKeyRangeSeparately(ranges) if err != nil { return nil, err } kvReqs := make([]*kv.Request, 0, len(kvRanges)) - // If storeType is TiFlash and tidb_isolation_read_engine is "tiflash_mpp", - // then copTask will only be sent to tiflash_mpp nodes. - // Will be checked when sending copTask. - storeType := e.storeType - if storeType == kv.TiFlash { - storeType, err = variable.GetTiFlashEngine(e.ctx.GetSessionVars().GetIsolationReadEngines()) - if err != nil { - return nil, err - } - } for i, kvRange := range kvRanges { e.kvRanges = append(e.kvRanges, kvRange...) if err := updateExecutorTableID(ctx, e.dagPB.RootExecutor, true, []int64{pids[i]}); err != nil { @@ -359,7 +369,7 @@ func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges [ return kvReqs, nil } -func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Context, ranges []*ranger.Range) (*kv.Request, error) { +func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Context, ranges []*ranger.Range, storeType kv.StoreType) (_ *kv.Request, err error) { pids, kvRanges, err := e.kvRangeBuilder.buildKeyRangeSeparately(ranges) if err != nil { return nil, err @@ -387,7 +397,7 @@ func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Contex SetFromSessionVars(e.ctx.GetSessionVars()). SetFromInfoSchema(e.ctx.GetInfoSchema()). SetMemTracker(e.memTracker). - SetStoreType(e.storeType). + SetStoreType(storeType). SetAllowBatchCop(e.batchCop).Build() if err != nil { return nil, err @@ -395,7 +405,7 @@ func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Contex return kvReq, nil } -func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.Range) (*kv.Request, error) { +func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.Range, storeType kv.StoreType) (_ *kv.Request, err error) { var builder distsql.RequestBuilder var reqBuilder *distsql.RequestBuilder if e.kvRangeBuilder != nil { @@ -418,7 +428,7 @@ func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.R SetFromSessionVars(e.ctx.GetSessionVars()). SetFromInfoSchema(e.ctx.GetInfoSchema()). SetMemTracker(e.memTracker). - SetStoreType(e.storeType). + SetStoreType(storeType). SetAllowBatchCop(e.batchCop) return reqBuilder.Build() } diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index dffef6d099b73..10e4746a869c5 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -540,12 +540,27 @@ func buildBatchCopTasksForNonPartitionedTable(bo *backoff.Backoffer, } else if engine == kv.TiFlashMPP { return buildBatchCopTasksConsistentHash(bo, store, []*KeyRanges{ranges}, storeType) } else { - return nil, errors.New(fmt.Sprint("unexpected engine type for tiflash: %v", engine)) + return nil, errors.New(fmt.Sprintf("unexpected engine type: %v", engine)) } } -func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType, mppStoreLastFailTime map[string]time.Time, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64, partitionIDs []int64) ([]*batchCopTask, error) { - batchTasks, err := buildBatchCopTasksCore(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) +func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, + store *kvStore, + rangesForEachPhysicalTable []*KeyRanges, + storeType kv.StoreType, + mppStoreLastFailTime map[string]time.Time, + ttl time.Duration, + balanceWithContinuity bool, + balanceContinuousRegionCount int64, + partitionIDs []int64, + engine kv.StoreType) (batchTasks []*batchCopTask, err error) { + if engine == kv.TiFlash { + batchTasks, err = buildBatchCopTasksCore(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) + } else if engine == kv.TiFlashMPP { + batchTasks, err = buildBatchCopTasksConsistentHash(bo, store, rangesForEachPhysicalTable, storeType) + } else { + err = errors.New(fmt.Sprintf("unexpected engine type: %v", engine)) + } if err != nil { return nil, err } @@ -804,7 +819,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V keyRanges = append(keyRanges, NewKeyRanges(pi.KeyRanges)) partitionIDs = append(partitionIDs, pi.ID) } - tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store.kvStore, keyRanges, req.StoreType, nil, 0, false, 0, partitionIDs) + tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store.kvStore, keyRanges, req.StoreType, nil, 0, false, 0, partitionIDs, storeType) } else { ranges := NewKeyRanges(req.KeyRanges) tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store.kvStore, ranges, req.StoreType, nil, 0, false, 0, storeType) @@ -977,7 +992,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoff.Ba } keyRanges = append(keyRanges, NewKeyRanges(ranges)) } - ret, err := buildBatchCopTasksForPartitionedTable(bo, b.store, keyRanges, b.req.StoreType, nil, 0, false, 0, pid) + ret, err := buildBatchCopTasksForPartitionedTable(bo, b.store, keyRanges, b.req.StoreType, nil, 0, false, 0, pid, b.storeType) return ret, err } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 228badaff1faf..da58fefa0365a 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -77,7 +77,7 @@ func (c *MPPClient) ConstructMPPTasks(ctx context.Context, rangesForEachPartition[i] = NewKeyRanges(p.KeyRanges) partitionIDs[i] = p.ID } - tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store, rangesForEachPartition, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20, partitionIDs) + tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store, rangesForEachPartition, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20, partitionIDs, engine) } else { if req.KeyRanges == nil { return c.selectAllTiFlashStore(), nil From d5b21ded5b67af16e39e2cf0ee74313f8a5a086b Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 23 May 2022 11:42:17 +0800 Subject: [PATCH 16/44] fix go.mod Signed-off-by: guo-shaoge --- go.mod | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 55e50469856b1..ead011f73e8bd 100644 --- a/go.mod +++ b/go.mod @@ -63,7 +63,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.1-0.20220518162527-de7ca289ac77 + github.com/tikv/client-go/v2 v2.0.1-0.20220523032253-700cbe60d2b1 github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710 github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible @@ -201,7 +201,6 @@ require ( gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect sigs.k8s.io/yaml v1.2.0 // indirect - stathat.com/c/consistent v1.0.0 // indirect ) replace github.com/pingcap/tidb/parser => ./parser From 7e851b64b50ddbf128ab4fc1dc0024f0a6dc8c8e Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 23 May 2022 11:44:08 +0800 Subject: [PATCH 17/44] fix Signed-off-by: guo-shaoge --- go.mod | 2 -- go.sum | 2 ++ 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index ead011f73e8bd..a0356c3bb7740 100644 --- a/go.mod +++ b/go.mod @@ -207,5 +207,3 @@ replace github.com/pingcap/tidb/parser => ./parser // fix potential security issue(CVE-2020-26160) introduced by indirect dependency. replace github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible - -replace github.com/tikv/client-go/v2 => /home/guojiangtao/work/client-go diff --git a/go.sum b/go.sum index a574447898b64..3b72f1b03fa3d 100644 --- a/go.sum +++ b/go.sum @@ -757,6 +757,8 @@ github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df/go.mod h1:6Fq8o github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= 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/tikv/client-go/v2 v2.0.1-0.20220523032253-700cbe60d2b1 h1:2efjpIbTIJYRymXUbWy+hIvWDhUydeRXIu/Wo/r/gbk= +github.com/tikv/client-go/v2 v2.0.1-0.20220523032253-700cbe60d2b1/go.mod h1:KzWkFRax8foxw13dSXAQZN+dLgixwahT10ZaAK9V/pg= github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710 h1:jxgmKOscXSjaFEKQGRyY5qOpK8hLqxs2irb/uDJMtwk= github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710/go.mod h1:AtvppPwkiyUgQlR1W9qSqfTB+OsOIu19jDCOxOsPkmU= github.com/tklauser/go-sysconf v0.3.9 h1:JeUVdAOWhhxVcU6Eqr/ATFHgXk/mmiItdKeJPev3vTo= From 706dfdfd6324ef89eb6e03071a1db9b016a22500 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 25 May 2022 15:22:09 +0800 Subject: [PATCH 18/44] fix comment Signed-off-by: guo-shaoge --- kv/kv.go | 2 +- sessionctx/variable/varsutil.go | 2 +- store/copr/batch_coprocessor.go | 5 ++--- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/kv/kv.go b/kv/kv.go index 65cd7c0a94edb..8dda0ded30b74 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -295,7 +295,7 @@ const ( // 1. sysvar.go: when user set tidb_isolation_read_engines. // 2. coprocessor.go/mpp.go: when send copTask/batchCopTask/mppTask to tiflash. // Other modules in TiDB should only use kv.TiFlash, they don't distinguish ReadNodes and WriteNodes. - // So we can avoid to change kv.TiFlashMPP in all code pathes where kv.TiFlash occurs. + // So we can avoid to change kv.TiFlashMPP in all code paths where kv.TiFlash occurs. TiFlashMPP // UnSpecified means the store type is unknown diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 6c71349b59c96..ca07808b3fc94 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -514,7 +514,7 @@ var GAFunction4ExpressionIndex = map[string]struct{}{ ast.TiDBShard: {}, } -// GetTiFlashEngine retrun kv.TiFlash or kv.TiFlashMPP. +// GetTiFlashEngine return kv.TiFlash or kv.TiFlashMPP. func GetTiFlashEngine(readEngines map[kv.StoreType]struct{}) (res kv.StoreType, _ error) { _, hasTiFlash := readEngines[kv.TiFlash] _, hasTiFlashMPP := readEngines[kv.TiFlashMPP] diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 10e4746a869c5..a51ba043cb4e8 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -578,10 +578,9 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran cache := store.GetRegionCache() for { - logutil.BgLogger().Info(fmt.Sprintf("buildBatchCopTasksConsistentHash retryNum: %v. ", retryNum)) retryNum++ if retryNum >= 10 { - return nil, errors.New("too many times of retry to GetTiFlashRPCContext()") + return nil, errors.New("too many times of retry to GetTiFlashMPPRPCContextByConsistentHash()") } var rangesLen int @@ -611,7 +610,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran return nil, err } if rpcCtxs == nil { - // Retry. + logutil.BgLogger().Info("buildBatchCopTasksConsistentHash retry because rcpCtx is nil", zap.Int("retryNum", retryNum)) continue } if len(rpcCtxs) != len(tasks) { From 9ab9afc79ae7b53d93f8e5fa6a499f5d2f1dea50 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 25 May 2022 15:38:17 +0800 Subject: [PATCH 19/44] refresh tiflash_mpp store cache every 30 seconds Signed-off-by: guo-shaoge --- domain/domain.go | 56 ++++++++++++++++++++++++++++++++++++++++++++-- session/session.go | 6 +++++ 2 files changed, 60 insertions(+), 2 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 5c7064920a3a5..d32ff1c10d1e4 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -56,6 +56,7 @@ import ( "github.com/pingcap/tidb/util/expensivequery" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" + "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/txnkv/transaction" pd "github.com/tikv/pd/client" clientv3 "go.etcd.io/etcd/client/v3" @@ -1089,6 +1090,56 @@ func (do *Domain) LoadSysVarCacheLoop(ctx sessionctx.Context) error { return nil } +// WatchTiFlashMPPStoreChange create a routine to watch. +// TODO: we can start watch only when user add tiflash_mpp into tidb_isolatoin_read_engines. +// tiflashMPPStoreKey is not put to etcd yet, store cache will only be invalidated every 30 seconds. +func (do *Domain) WatchTiFlashMPPStoreChange() error { + var watchCh clientv3.WatchChan + if do.etcdClient != nil { + watchCh = do.etcdClient.Watch(context.Background(), tiflashMPPStoreKey) + } + do.wg.Add(1) + duration := 30 * time.Second + go func() { + defer func() { + do.wg.Done() + logutil.BgLogger().Info("WatchTiFlashMPPStoreChange exit") + }() + + var count int + for { + ok := true + var watched bool + select { + case <-do.exit: + return + case _, ok = <-watchCh: + watched = true + case <-time.After(duration): + } + if !ok { + logutil.BgLogger().Error("WatchTiFlashMPPStoreChange watch channel closed") + watchCh = do.etcdClient.Watch(context.Background(), tiflashMPPStoreKey) + count++ + if count > 10 { + time.Sleep(time.Duration(count) * time.Second) + } + continue + } + count = 0 + switch s := do.store.(type) { + case tikv.Storage: + s.GetRegionCache().InvalidateTiFlashMPPStores() + logutil.BgLogger().Info("tiflash_mpp store cache invalied, will update next query", zap.Bool("watched", watched)) + default: + logutil.BgLogger().Info("ignore non tikv store to watch tiflashMPPStoreKey") + return + } + } + }() + return nil +} + // PrivilegeHandle returns the MySQLPrivilege. func (do *Domain) PrivilegeHandle() *privileges.Handle { return do.privHandle @@ -1546,8 +1597,9 @@ func (do *Domain) ExpensiveQueryHandle() *expensivequery.Handle { } const ( - privilegeKey = "/tidb/privilege" - sysVarCacheKey = "/tidb/sysvars" + privilegeKey = "/tidb/privilege" + sysVarCacheKey = "/tidb/sysvars" + tiflashMPPStoreKey = "/tiflash/new_tiflash_mpp_stores" ) // NotifyUpdatePrivilege updates privilege key in etcd, TiDB client that watches diff --git a/session/session.go b/session/session.go index decee805527d9..4fbb1fefa9083 100644 --- a/session/session.go +++ b/session/session.go @@ -2876,6 +2876,12 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { return nil, err } + // Invalid client-go tiflash_mpp stores if necessary. + err = dom.WatchTiFlashMPPStoreChange() + if err != nil { + return nil, err + } + if len(cfg.Instance.PluginLoad) > 0 { err := plugin.Init(context.Background(), plugin.Config{EtcdClient: dom.GetEtcdClient()}) if err != nil { From eca3e6023ee4f7cd3461db2cd7071d072d4cfbc8 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 25 May 2022 15:55:57 +0800 Subject: [PATCH 20/44] fix Signed-off-by: guo-shaoge --- store/copr/mpp.go | 32 +++++++++++++++++++------------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index da58fefa0365a..dbf797aa2f1b8 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -238,18 +238,20 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req } } + // TODO: Handle dispatch task response correctly, including retry logic and cancel logic. + var rpcResp *tikvrpc.Response + var err error + var retry bool + var tp tikvrpc.EndpointType + wrappedReq := tikvrpc.NewRequest(tikvrpc.CmdMPPTask, mppReq, kvrpcpb.Context{}) - if tp, err := getTiFlashEndPointType(req.StoreTp); err != nil { + tp, err = getTiFlashEndPointType(req.StoreTp) + if err != nil { m.sendError(err) return - } else { - wrappedReq.StoreTp = tp } + wrappedReq.StoreTp = tp - // TODO: Handle dispatch task response correctly, including retry logic and cancel logic. - var rpcResp *tikvrpc.Response - var err error - var retry bool // If copTasks is not empty, we should send request according to region distribution. // Or else it's the task without region, which always happens in high layer task without table. // In that case @@ -342,6 +344,8 @@ func (m *mppIterator) cancelMppTasks() { } wrappedReq := tikvrpc.NewRequest(tikvrpc.CmdMPPCancel, killReq, kvrpcpb.Context{}) + var tp tikvrpc.EndpointType + var err error // Value type is kv.StoreType, so we can decide whether to invalid tiflash_mpp store cache or not. usedStoreAddrs := make(map[string]kv.StoreType) @@ -349,12 +353,12 @@ func (m *mppIterator) cancelMppTasks() { // get the store address of running tasks if _, ok := usedStoreAddrs[task.Meta.GetAddress()]; ok && task.State == kv.MppTaskRunning { usedStoreAddrs[task.Meta.GetAddress()] = task.StoreTp - if tp, err := getTiFlashEndPointType(task.StoreTp); err != nil { + tp, err = getTiFlashEndPointType(task.StoreTp) + if err != nil { m.sendError(err) return - } else { - wrappedReq.StoreTp = tp } + wrappedReq.StoreTp = tp } else if task.State == kv.MppTaskCancelled { return } @@ -383,13 +387,15 @@ func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchReques }, } + var err error + var tp tikvrpc.EndpointType wrappedReq := tikvrpc.NewRequest(tikvrpc.CmdMPPConn, connReq, kvrpcpb.Context{}) - if tp, err := getTiFlashEndPointType(req.StoreTp); err != nil { + tp, err = getTiFlashEndPointType(req.StoreTp) + if err != nil { m.sendError(err) return - } else { - wrappedReq.StoreTp = tp } + wrappedReq.StoreTp = tp // Drain results from root task. // We don't need to process any special error. When we meet errors, just let it fail. From e912a64b27dbe5d1b80922eb774786538f520381 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 26 May 2022 19:29:37 +0800 Subject: [PATCH 21/44] fix lint Signed-off-by: guo-shaoge --- store/copr/batch_coprocessor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index a51ba043cb4e8..80f73681a1a2e 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -644,7 +644,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran failpoint.Inject("check_only_dispatched_to_tiflash_mpp_nodes", func(val failpoint.Value) { // This failpoint will be tested in test-infra case, because we needs setup a cluster. - // All ReadNode addrs are stored in val, each addr is seperated by semicolon. + // All ReadNode addrs are stored in val, each addr is separated by semicolon. str := val.(string) addrs := strings.Split(str, ";") if len(addrs) < 1 { From dbaeaa686adbe74a08330b306573486a3e1915e2 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 2 Aug 2022 16:07:24 +0800 Subject: [PATCH 22/44] fix ignoring tiflash_mpp when booststraping Signed-off-by: guo-shaoge --- util/engine/engine.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/util/engine/engine.go b/util/engine/engine.go index 68c369154888e..f71737de6a8ab 100644 --- a/util/engine/engine.go +++ b/util/engine/engine.go @@ -21,7 +21,7 @@ import ( // IsTiFlash tests whether the store is based on tiflash engine. func IsTiFlash(store *metapb.Store) bool { for _, label := range store.Labels { - if label.Key == "engine" && label.Value == "tiflash" { + if label.Key == "engine" && (label.Value == "tiflash_mpp" || label.Value == "tiflash") { return true } } From d5e083c21bdbb7cd59bc72c12028e9006db91755 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 20 Sep 2022 15:43:00 +0800 Subject: [PATCH 23/44] change MPPTask dispatch from ConsistentHash(regionID) to ConsistentHash(batchCopTask.addr) Signed-off-by: guo-shaoge --- go.mod | 2 + go.sum | 2 - store/copr/batch_coprocessor.go | 253 +++++++++++++++++++------------- 3 files changed, 157 insertions(+), 100 deletions(-) diff --git a/go.mod b/go.mod index 5e0f9239632ea..a1fbd4c7681f9 100644 --- a/go.mod +++ b/go.mod @@ -240,3 +240,5 @@ replace ( github.com/dgrijalva/jwt-go => github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible github.com/pingcap/tidb/parser => ./parser ) + +replace github.com/tikv/client-go/v2 => /home/guojiangtao/work/client-go diff --git a/go.sum b/go.sum index a4113ec9e8f73..772a5210582ad 100644 --- a/go.sum +++ b/go.sum @@ -906,8 +906,6 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= 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/tikv/client-go/v2 v2.0.1-0.20220818084834-0d0ae0dcfb1f h1:/nr7P8uzJQ7u3wPEBHCokrsVmuDvi/1x/zI/ydk5n8U= -github.com/tikv/client-go/v2 v2.0.1-0.20220818084834-0d0ae0dcfb1f/go.mod h1:v3DEt8LS9olI6D6El17pYBWq7B28hw3NnDFTxQHDLpY= github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db h1:r1eMh9Rny3hfWuBuxOnbsCRrR4FhthiNxLQ5rAUtaww= github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db/go.mod h1:ew8kS0yIcEaSetuuywkTLIUBR+sz3J5XvAYRae11qwc= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro= diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 29b9cc86dba3c..431ce81cf91df 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -21,7 +21,7 @@ import ( "io" "math" "strconv" - "strings" + _"strings" "sync" "sync/atomic" "time" @@ -41,6 +41,7 @@ import ( "github.com/tikv/client-go/v2/tikvrpc" "go.uber.org/zap" "golang.org/x/exp/slices" + "github.com/stathat/consistent" ) // batchCopTask comprises of multiple copTask that will send to same store. @@ -283,12 +284,11 @@ func balanceBatchCopTaskWithContinuity(storeTaskMap map[uint64]*batchCopTask, ca } // balanceBatchCopTask balance the regions between available stores, the basic rule is -// 1. the first region of each original batch cop task belongs to its original store because some -// meta data(like the rpc context) in batchCopTask is related to it -// 2. for the remaining regions: -// if there is only 1 available store, then put the region to the related store -// otherwise, these region will be balance between TiFlash stores. -// +// 1. the first region of each original batch cop task belongs to its original store because some +// meta data(like the rpc context) in batchCopTask is related to it +// 2. for the remaining regions: +// if there is only 1 available store, then put the region to the related store +// otherwise, these region will be balance between TiFlash stores. // Currently, there are two balance strategies. // The first balance strategy: use a greedy algorithm to put it into the store with highest weight. This strategy only consider the region count between TiFlash stores. // @@ -539,7 +539,7 @@ func buildBatchCopTasksForNonPartitionedTable(bo *backoff.Backoffer, if engine == kv.TiFlash { return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) } else if engine == kv.TiFlashMPP { - return buildBatchCopTasksConsistentHash(bo, store, []*KeyRanges{ranges}, storeType) + return buildBatchCopTasksConsistentHash(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) } else { return nil, errors.New(fmt.Sprintf("unexpected engine type: %v", engine)) } @@ -558,7 +558,7 @@ func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, if engine == kv.TiFlash { batchTasks, err = buildBatchCopTasksCore(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) } else if engine == kv.TiFlashMPP { - batchTasks, err = buildBatchCopTasksConsistentHash(bo, store, rangesForEachPhysicalTable, storeType) + batchTasks, err = buildBatchCopTasksConsistentHash(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) } else { err = errors.New(fmt.Sprintf("unexpected engine type: %v", engine)) } @@ -570,105 +570,162 @@ func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, return batchTasks, nil } -// 1. Split range by region location to build copTasks. -// 2. For each copTask build its rpcCtx , the target ReadNode will be computed using consistent hash. -// 3. All copTasks that will be sent to one ReadNode are put in one batchCopTask. -func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType) (res []*batchCopTask, err error) { - const cmdType = tikvrpc.CmdBatchCop - var retryNum int +func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType, mppStoreLastFailTime map[string]time.Time, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64) ([]*batchCopTask, error) { + batchTasks, err := buildBatchCopTasksCore(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) + if err != nil { + return nil, err + } cache := store.GetRegionCache() + mppStores, err := cache.GetTiFlashMPPStores(bo.TiKVBackoffer()) + if err != nil { + return nil, err + } + if len(mppStores) == 0 { + return nil, errors.New("Number of tiflash_mpp node is zero") + } - for { - retryNum++ - if retryNum >= 10 { - return nil, errors.New("too many times of retry to GetTiFlashMPPRPCContextByConsistentHash()") - } - - var rangesLen int - tasks := make([]*copTask, 0) - regionIDs := make([]tikv.RegionVerID, 0) - - for i, ranges := range rangesForEachPhysicalTable { - rangesLen += ranges.Len() - locations, err := cache.SplitKeyRangesByLocations(bo, ranges) - if err != nil { - return nil, errors.Trace(err) - } - for _, lo := range locations { - tasks = append(tasks, &copTask{ - region: lo.Location.Region, - ranges: lo.Ranges, - cmdType: cmdType, - storeType: storeType, - partitionIndex: int64(i), - }) - regionIDs = append(regionIDs, lo.Location.Region) - } - } - - rpcCtxs, err := cache.GetTiFlashMPPRPCContextByConsistentHash(bo.TiKVBackoffer(), regionIDs) + hasher := consistent.New() + for _, store := range mppStores { + hasher.Add(store.GetAddr()) + } + for _, task := range batchTasks { + addr, err := hasher.Get(task.storeAddr) if err != nil { return nil, err } - if rpcCtxs == nil { - logutil.BgLogger().Info("buildBatchCopTasksConsistentHash retry because rcpCtx is nil", zap.Int("retryNum", retryNum)) - continue - } - if len(rpcCtxs) != len(tasks) { - return nil, errors.New(fmt.Sprintf("unexpected length of rpcCtxs, expect: %v, got: %v", len(tasks), len(rpcCtxs))) - } - taskMap := make(map[string]*batchCopTask) - for i, rpcCtx := range rpcCtxs { - regionInfo := RegionInfo{ - // tasks and rpcCtxs are correspond to each other. - Region: tasks[i].region, - Meta: rpcCtx.Meta, - Ranges: tasks[i].ranges, - AllStores: []uint64{rpcCtx.Store.StoreID()}, - PartitionIndex: tasks[i].partitionIndex, - } - if batchTask, ok := taskMap[rpcCtx.Addr]; ok { - batchTask.regionInfos = append(batchTask.regionInfos, regionInfo) - } else { - batchTask := &batchCopTask{ - storeAddr: rpcCtx.Addr, - cmdType: cmdType, - ctx: rpcCtx, - regionInfos: []RegionInfo{regionInfo}, - } - taskMap[rpcCtx.Addr] = batchTask - res = append(res, batchTask) + var store *tikv.Store + for _, s := range mppStores { + if s.GetAddr() == addr { + store = s + break } } - break - } - - failpoint.Inject("check_only_dispatched_to_tiflash_mpp_nodes", func(val failpoint.Value) { - // This failpoint will be tested in test-infra case, because we needs setup a cluster. - // All ReadNode addrs are stored in val, each addr is separated by semicolon. - str := val.(string) - addrs := strings.Split(str, ";") - if len(addrs) < 1 { - panic(fmt.Sprintf("unexpected length of tiflash_mpp node addrs: %v", len(addrs))) - } - for _, batchTask := range res { - var matched bool - for _, addr := range addrs { - if batchTask.storeAddr == addr { - matched = true - break - } - } - if !matched { - panic(fmt.Sprintf("batchCopTask send to node which is not tiflash_mpp: %v(tiflash_mpp nodes: %s)", - batchTask.storeAddr, str)) - } + if store == nil { + return nil, errors.New(fmt.Sprintf("cannot find mpp store: %v", addr)) } - logutil.BgLogger().Info(fmt.Sprintf("check_only_dispatched_to_tiflash_mpp_nodes checked succeed(tiflash_mpp nodes: %s)", str)) - }) - return res, nil + + task.storeAddr = addr + task.ctx.Store = store + task.ctx.Addr = addr + } + logutil.BgLogger().Info(fmt.Sprintf("gjt debug batchTasks: len: %d\n", len(batchTasks))) + for i, task := range batchTasks { + logutil.BgLogger().Info(fmt.Sprintf("gjt debug each batchTasks[%d], addr: %s, len(RegionInfo): %d\n", i, task.storeAddr, len(task.regionInfos))) + } + return batchTasks, nil } +// // 1. Split range by region location to build copTasks. +// // 2. For each copTask build its rpcCtx , the target ReadNode will be computed using consistent hash. +// // 3. All copTasks that will be sent to one ReadNode are put in one batchCopTask. +// func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType) (res []*batchCopTask, err error) { +// const cmdType = tikvrpc.CmdBatchCop +// var retryNum int +// cache := store.GetRegionCache() +// +// for { +// retryNum++ +// if retryNum >= 10 { +// return nil, errors.New("too many times of retry to GetTiFlashMPPRPCContextByConsistentHash()") +// } +// +// var rangesLen int +// tasks := make([]*copTask, 0) +// regionIDs := make([]tikv.RegionVerID, 0) +// +// for i, ranges := range rangesForEachPhysicalTable { +// rangesLen += ranges.Len() +// locations, err := cache.SplitKeyRangesByLocations(bo, ranges) +// if err != nil { +// return nil, errors.Trace(err) +// } +// for _, lo := range locations { +// tasks = append(tasks, &copTask{ +// region: lo.Location.Region, +// ranges: lo.Ranges, +// cmdType: cmdType, +// storeType: storeType, +// partitionIndex: int64(i), +// }) +// regionIDs = append(regionIDs, lo.Location.Region) +// } +// } +// +// rpcCtxs, err := cache.GetTiFlashMPPRPCContextByConsistentHash(bo.TiKVBackoffer(), regionIDs) +// if err != nil { +// return nil, err +// } +// if rpcCtxs == nil { +// logutil.BgLogger().Info("buildBatchCopTasksConsistentHash retry because rcpCtx is nil", zap.Int("retryNum", retryNum)) +// continue +// } +// if len(rpcCtxs) != len(tasks) { +// return nil, errors.New(fmt.Sprintf("unexpected length of rpcCtxs, expect: %v, got: %v", len(tasks), len(rpcCtxs))) +// } +// logutil.BgLogger().Info(fmt.Sprintf("gjt debug cop tsaks len(tasks): %v\n", len(tasks))) +// for i, t := range tasks { +// logutil.BgLogger().Info(fmt.Sprintf("gjt debug regoin: %v, cmdType: %v, storeType: %v\n", t.region, t.cmdType, t.storeType)) +// logutil.BgLogger().Info(fmt.Sprintf("gjt debug rpcCtxs. id: %v, region: %v, addr: %v\n", i, rpcCtxs[i].Region, rpcCtxs[i].Addr)) +// } +// taskMap := make(map[string]*batchCopTask) +// for i, rpcCtx := range rpcCtxs { +// regionInfo := RegionInfo{ +// // tasks and rpcCtxs are correspond to each other. +// Region: tasks[i].region, +// Meta: rpcCtx.Meta, +// Ranges: tasks[i].ranges, +// AllStores: []uint64{rpcCtx.Store.StoreID()}, +// PartitionIndex: tasks[i].partitionIndex, +// } +// if batchTask, ok := taskMap[rpcCtx.Addr]; ok { +// batchTask.regionInfos = append(batchTask.regionInfos, regionInfo) +// } else { +// batchTask := &batchCopTask{ +// storeAddr: rpcCtx.Addr, +// cmdType: cmdType, +// ctx: rpcCtx, +// regionInfos: []RegionInfo{regionInfo}, +// } +// taskMap[rpcCtx.Addr] = batchTask +// res = append(res, batchTask) +// } +// } +// logutil.BgLogger().Info(fmt.Sprintf("gjt debug all batchTasks: len(tasksMap): %v", len(taskMap))) +// for addr, t := range taskMap { +// logutil.BgLogger().Info(fmt.Sprintf("gjt debug addr: %v, len(RegionInfo): %v", addr, len(t.regionInfos))) +// for i, reg := range t.regionInfos { +// logutil.BgLogger().Info(fmt.Sprintf("gjt debug region id: %v, reg: %v", i, reg)) +// } +// } +// break +// } +// +// failpoint.Inject("check_only_dispatched_to_tiflash_mpp_nodes", func(val failpoint.Value) { +// // This failpoint will be tested in test-infra case, because we needs setup a cluster. +// // All ReadNode addrs are stored in val, each addr is separated by semicolon. +// str := val.(string) +// addrs := strings.Split(str, ";") +// if len(addrs) < 1 { +// panic(fmt.Sprintf("unexpected length of tiflash_mpp node addrs: %v", len(addrs))) +// } +// for _, batchTask := range res { +// var matched bool +// for _, addr := range addrs { +// if batchTask.storeAddr == addr { +// matched = true +// break +// } +// } +// if !matched { +// panic(fmt.Sprintf("batchCopTask send to node which is not tiflash_mpp: %v(tiflash_mpp nodes: %s)", +// batchTask.storeAddr, str)) +// } +// } +// logutil.BgLogger().Info(fmt.Sprintf("check_only_dispatched_to_tiflash_mpp_nodes checked succeed(tiflash_mpp nodes: %s)", str)) +// }) +// return res, nil +// } + // When `partitionIDs != nil`, it means that buildBatchCopTasksCore is constructing a batch cop tasks for PartitionTableScan. // At this time, `len(rangesForEachPhysicalTable) == len(partitionIDs)` and `rangesForEachPhysicalTable[i]` is for partition `partitionIDs[i]`. // Otherwise, `rangesForEachPhysicalTable[0]` indicates the range for the single physical table. From a1def16f6787888191121a895b4c20743638babf Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 20 Sep 2022 16:22:04 +0800 Subject: [PATCH 24/44] fix fmt Signed-off-by: guo-shaoge --- domain/domain.go | 3 ++- store/copr/batch_coprocessor.go | 25 +++++++++++++------------ 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 3d2bf22c6b7ca..0f64322977334 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1364,7 +1364,8 @@ func (do *Domain) LoadSysVarCacheLoop(ctx sessionctx.Context) error { // WatchTiFlashMPPStoreChange create a routine to watch. // TODO: we can start watch only when user add tiflash_mpp into tidb_isolatoin_read_engines. -// tiflashMPPStoreKey is not put to etcd yet, store cache will only be invalidated every 30 seconds. +// +// tiflashMPPStoreKey is not put to etcd yet, store cache will only be invalidated every 30 seconds. func (do *Domain) WatchTiFlashMPPStoreChange() error { var watchCh clientv3.WatchChan if do.etcdClient != nil { diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index d046699a4d8a5..d35d023d14d1d 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -21,7 +21,7 @@ import ( "io" "math" "strconv" - _"strings" + _ "strings" "sync" "sync/atomic" "time" @@ -36,12 +36,12 @@ import ( "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/util/logutil" + "github.com/stathat/consistent" "github.com/tikv/client-go/v2/metrics" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" "go.uber.org/zap" "golang.org/x/exp/slices" - "github.com/stathat/consistent" ) // batchCopTask comprises of multiple copTask that will send to same store. @@ -284,11 +284,12 @@ func balanceBatchCopTaskWithContinuity(storeTaskMap map[uint64]*batchCopTask, ca } // balanceBatchCopTask balance the regions between available stores, the basic rule is -// 1. the first region of each original batch cop task belongs to its original store because some -// meta data(like the rpc context) in batchCopTask is related to it -// 2. for the remaining regions: -// if there is only 1 available store, then put the region to the related store -// otherwise, these region will be balance between TiFlash stores. +// 1. the first region of each original batch cop task belongs to its original store because some +// meta data(like the rpc context) in batchCopTask is related to it +// 2. for the remaining regions: +// if there is only 1 available store, then put the region to the related store +// otherwise, these region will be balance between TiFlash stores. +// // Currently, there are two balance strategies. // The first balance strategy: use a greedy algorithm to put it into the store with highest weight. This strategy only consider the region count between TiFlash stores. // @@ -622,17 +623,17 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran // const cmdType = tikvrpc.CmdBatchCop // var retryNum int // cache := store.GetRegionCache() -// +// // for { // retryNum++ // if retryNum >= 10 { // return nil, errors.New("too many times of retry to GetTiFlashMPPRPCContextByConsistentHash()") // } -// +// // var rangesLen int // tasks := make([]*copTask, 0) // regionIDs := make([]tikv.RegionVerID, 0) -// +// // for i, ranges := range rangesForEachPhysicalTable { // rangesLen += ranges.Len() // locations, err := cache.SplitKeyRangesByLocations(bo, ranges) @@ -650,7 +651,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran // regionIDs = append(regionIDs, lo.Location.Region) // } // } -// +// // rpcCtxs, err := cache.GetTiFlashMPPRPCContextByConsistentHash(bo.TiKVBackoffer(), regionIDs) // if err != nil { // return nil, err @@ -699,7 +700,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran // } // break // } -// +// // failpoint.Inject("check_only_dispatched_to_tiflash_mpp_nodes", func(val failpoint.Value) { // // This failpoint will be tested in test-infra case, because we needs setup a cluster. // // All ReadNode addrs are stored in val, each addr is separated by semicolon. From 34a26400f4f1a9efcbcb53921e8057bdff59363d Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 21 Sep 2022 10:36:50 +0800 Subject: [PATCH 25/44] using config(disaggregated_tiflash) instead of isolation_read_engines Signed-off-by: guo-shaoge --- config/config.go | 1 + planner/core/logical_plan_builder.go | 13 +++++++++++++ 2 files changed, 14 insertions(+) diff --git a/config/config.go b/config/config.go index a22a7614ffc4a..999493a0949eb 100644 --- a/config/config.go +++ b/config/config.go @@ -276,6 +276,7 @@ type Config struct { Plugin Plugin `toml:"plugin" json:"plugin"` MaxServerConnections uint32 `toml:"max-server-connections" json:"max-server-connections"` RunDDL bool `toml:"run-ddl" json:"run-ddl"` + DisaggregatedTiFlash bool `toml:"disaggregated_tiflash" json: "disaggregated_tiflash"` } // UpdateTempStoragePath is to update the `TempStoragePath` if port/statusPort was changed diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index e25548ca541db..4ec8224e1be30 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" @@ -49,6 +50,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/store/driver/backoff" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/table/temptable" @@ -63,6 +65,7 @@ import ( "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/set" + "github.com/tikv/client-go/v2/tikv" ) const ( @@ -674,6 +677,16 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { ds.preferStoreType = 0 return } + if config.GetGlobalConfig().DisaggregatedTiFlash { + bo := backoff.NewBackofferWithVars(context.Background(), 5000, nil) + mppStores, err := ds.ctx.GetStore().(tikv.Storage).GetRegionCache().GetTiFlashMPPStores(bo.TiKVBackoffer()) + if err != nil || len(mppStores) == 0 { + errMsg := fmt.Sprintf("TiFlash ReadNodes number is zero") + warning := ErrInternal.GenWithStack(errMsg) + ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + return + } + } for _, path := range ds.possibleAccessPaths { if path.StoreType == kv.TiFlash { ds.preferStoreType |= preferTiFlash From d05c46c4f72837583fd31cda6bf554d4d1cbff2c Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 21 Sep 2022 16:10:50 +0800 Subject: [PATCH 26/44] 1. delete usage of kv.TiFlashMPP 2. only touch config.DisaggregatedTiFlash when dispatcing request Signed-off-by: guo-shaoge --- domain/domain.go | 8 +- executor/mpp_gather.go | 9 -- executor/table_reader.go | 43 ++------ kv/kv.go | 10 -- kv/mpp.go | 4 +- planner/core/fragment.go | 8 +- planner/core/logical_plan_builder.go | 23 ++-- planner/core/planbuilder.go | 7 +- session/session.go | 10 +- sessionctx/variable/sysvar.go | 11 -- sessionctx/variable/varsutil.go | 20 ---- store/copr/batch_coprocessor.go | 153 +++------------------------ store/copr/batch_request_sender.go | 28 +---- store/copr/coprocessor.go | 4 +- store/copr/mpp.go | 58 ++++------ store/copr/store.go | 14 +-- 16 files changed, 91 insertions(+), 319 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 0f64322977334..5f37956ebaf76 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1363,9 +1363,9 @@ func (do *Domain) LoadSysVarCacheLoop(ctx sessionctx.Context) error { } // WatchTiFlashMPPStoreChange create a routine to watch. -// TODO: we can start watch only when user add tiflash_mpp into tidb_isolatoin_read_engines. +// TODO: tiflashMPPStoreKey is not put to etcd yet(finish this when AutoScaler is done) // -// tiflashMPPStoreKey is not put to etcd yet, store cache will only be invalidated every 30 seconds. +// store cache will only be invalidated every 30 seconds. func (do *Domain) WatchTiFlashMPPStoreChange() error { var watchCh clientv3.WatchChan if do.etcdClient != nil { @@ -1403,9 +1403,9 @@ func (do *Domain) WatchTiFlashMPPStoreChange() error { switch s := do.store.(type) { case tikv.Storage: s.GetRegionCache().InvalidateTiFlashMPPStores() - logutil.BgLogger().Info("tiflash_mpp store cache invalied, will update next query", zap.Bool("watched", watched)) + logutil.BgLogger().Debug("tiflash_mpp store cache invalied, will update next query", zap.Bool("watched", watched)) default: - logutil.BgLogger().Info("ignore non tikv store to watch tiflashMPPStoreKey") + logutil.BgLogger().Debug("No need to watch TiFlashMPP for non-tikv store") return } } diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index e0115ac94474e..42526774dbdd5 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-tipb" @@ -53,13 +52,6 @@ type MPPGather struct { } func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { - // If storeType is TiFlash and tidb_isolation_read_engine is "tiflash_mpp", - // then batchCopTask will only be sent to tiflash_mpp nodes. - // Will be checked when send batchCopTask. - storeType, err := variable.GetTiFlashEngine(e.ctx.GetSessionVars().GetIsolationReadEngines()) - if err != nil { - return err - } dagReq, err := constructDAGReq(e.ctx, []plannercore.PhysicalPlan{pf.ExchangeSender}, kv.TiFlash) if err != nil { return errors.Trace(err) @@ -97,7 +89,6 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { SchemaVar: e.is.SchemaMetaVersion(), StartTs: e.startTS, State: kv.MppTaskReady, - StoreTp: storeType, } e.mppReqs = append(e.mppReqs, req) } diff --git a/executor/table_reader.go b/executor/table_reader.go index 17b5e25b2511a..097beb1c8a547 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/parser/model" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -198,17 +197,13 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { // [9734095886065816709 9734095886065816708 9734095886065816707 65535 3 1] => rows reverse in UnionScan firstPartRanges, secondPartRanges = secondPartRanges, firstPartRanges } - storeType, err := e.considerTiFlashMPPStoreType() - if err != nil { - return err - } - kvReq, err := e.buildKVReq(ctx, firstPartRanges, storeType) + kvReq, err := e.buildKVReq(ctx, firstPartRanges) if err != nil { return err } e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...) if len(secondPartRanges) != 0 { - kvReq, err = e.buildKVReq(ctx, secondPartRanges, storeType) + kvReq, err = e.buildKVReq(ctx, secondPartRanges) if err != nil { return err } @@ -279,29 +274,13 @@ func (e *TableReaderExecutor) Close() error { return err } -func (e *TableReaderExecutor) considerTiFlashMPPStoreType() (_ kv.StoreType, err error) { - // If storeType is TiFlash and tidb_isolation_read_engine is "tiflash_mpp", - // then copTask will only be sent to tiflash_mpp nodes. - // Ideally, storeType should be setup ok at the planner phase rather than here. - // But this will cause too many changes, so kv.TiFlashMPP will only be considered in TableReader/MPPGather. - storeType := e.storeType - if storeType == kv.TiFlash { - storeType, err = variable.GetTiFlashEngine(e.ctx.GetSessionVars().GetIsolationReadEngines()) - } - return storeType, err -} - // buildResp first builds request and sends it to tikv using distsql.Select. It uses SelectResult returned by the callee // to fetch all results. func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (_ distsql.SelectResult, err error) { - storeType, err := e.considerTiFlashMPPStoreType() - if err != nil { - return nil, err - } if e.storeType == kv.TiFlash && e.kvRangeBuilder != nil { if !e.batchCop { // TiFlash cannot support to access multiple tables/partitions within one KVReq, so we have to build KVReq for each partition separately. - kvReqs, err := e.buildKVReqSeparately(ctx, ranges, storeType) + kvReqs, err := e.buildKVReqSeparately(ctx, ranges) if err != nil { return nil, err } @@ -316,7 +295,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra return distsql.NewSerialSelectResults(results), nil } // Use PartitionTable Scan - kvReq, err := e.buildKVReqForPartitionTableScan(ctx, ranges, storeType) + kvReq, err := e.buildKVReqForPartitionTableScan(ctx, ranges) if err != nil { return nil, err } @@ -327,7 +306,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra return result, nil } - kvReq, err := e.buildKVReq(ctx, ranges, storeType) + kvReq, err := e.buildKVReq(ctx, ranges) if err != nil { return nil, err } @@ -343,7 +322,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra return result, nil } -func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges []*ranger.Range, storeType kv.StoreType) (_ []*kv.Request, err error) { +func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges []*ranger.Range) (_ []*kv.Request, err error) { pids, kvRanges, err := e.kvRangeBuilder.buildKeyRangeSeparately(ranges) if err != nil { return nil, err @@ -367,7 +346,7 @@ func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges [ SetFromSessionVars(e.ctx.GetSessionVars()). SetFromInfoSchema(e.ctx.GetInfoSchema()). SetMemTracker(e.memTracker). - SetStoreType(storeType). + SetStoreType(e.storeType). SetPaging(e.paging). SetAllowBatchCop(e.batchCop). SetClosestReplicaReadAdjuster(newClosestReadAdjuster(e.ctx, &reqBuilder.Request, e.netDataSize)). @@ -380,7 +359,7 @@ func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges [ return kvReqs, nil } -func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Context, ranges []*ranger.Range, storeType kv.StoreType) (_ *kv.Request, err error) { +func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Context, ranges []*ranger.Range) (_ *kv.Request, err error) { pids, kvRanges, err := e.kvRangeBuilder.buildKeyRangeSeparately(ranges) if err != nil { return nil, err @@ -408,7 +387,7 @@ func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Contex SetFromSessionVars(e.ctx.GetSessionVars()). SetFromInfoSchema(e.ctx.GetInfoSchema()). SetMemTracker(e.memTracker). - SetStoreType(storeType). + SetStoreType(e.storeType). SetPaging(e.paging). SetAllowBatchCop(e.batchCop). SetClosestReplicaReadAdjuster(newClosestReadAdjuster(e.ctx, &reqBuilder.Request, e.netDataSize)). @@ -419,7 +398,7 @@ func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Contex return kvReq, nil } -func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.Range, storeType kv.StoreType) (_ *kv.Request, err error) { +func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.Range) (_ *kv.Request, err error) { var builder distsql.RequestBuilder var reqBuilder *distsql.RequestBuilder if e.kvRangeBuilder != nil { @@ -442,7 +421,7 @@ func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.R SetFromSessionVars(e.ctx.GetSessionVars()). SetFromInfoSchema(e.ctx.GetInfoSchema()). SetMemTracker(e.memTracker). - SetStoreType(storeType). + SetStoreType(e.storeType). SetAllowBatchCop(e.batchCop). SetClosestReplicaReadAdjuster(newClosestReadAdjuster(e.ctx, &reqBuilder.Request, e.netDataSize)). SetPaging(e.paging) diff --git a/kv/kv.go b/kv/kv.go index aa2097865f062..2ca6bdcc561dd 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -316,14 +316,6 @@ const ( // TiDB means the type of a store is TiDB. TiDB - // TiFlashMPP means tiflash ReadNodes. - // WATCHOUT: This label should only be used in: - // 1. sysvar.go: when user set tidb_isolation_read_engines. - // 2. coprocessor.go/mpp.go: when send copTask/batchCopTask/mppTask to tiflash. - // Other modules in TiDB should only use kv.TiFlash, they don't distinguish ReadNodes and WriteNodes. - // So we can avoid to change kv.TiFlashMPP in all code paths where kv.TiFlash occurs. - TiFlashMPP - // UnSpecified means the store type is unknown UnSpecified = 255 ) @@ -336,8 +328,6 @@ func (t StoreType) Name() string { return "tidb" } else if t == TiKV { return "tikv" - } else if t == TiFlashMPP { - return "tiflash_mpp" } return "unspecified" } diff --git a/kv/mpp.go b/kv/mpp.go index 60be60d2fc66f..b3fc209d5f35c 100644 --- a/kv/mpp.go +++ b/kv/mpp.go @@ -80,15 +80,13 @@ type MPPDispatchRequest struct { // Identify a single task. ID int64 State MppTaskStates - // Check is tiflash or tiflash_mpp. - StoreTp StoreType } // MPPClient accepts and processes mpp requests. type MPPClient interface { // ConstructMPPTasks schedules task for a plan fragment. // TODO:: This interface will be refined after we support more executors. - ConstructMPPTasks(context.Context, *MPPBuildTasksRequest, *sync.Map, time.Duration, StoreType) ([]MPPTaskMeta, error) + ConstructMPPTasks(context.Context, *MPPBuildTasksRequest, *sync.Map, time.Duration) ([]MPPTaskMeta, error) // DispatchMPPTasks dispatches ALL mpp requests at once, and returns an iterator that transfers the data. DispatchMPPTasks(ctx context.Context, vars interface{}, reqs []*MPPDispatchRequest, needTriggerFallback bool, startTs uint64) Response diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 99c5e606c66da..df2d537a802a8 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" @@ -356,12 +355,7 @@ func (e *mppTaskGenerator) constructMPPTasksImpl(ctx context.Context, ts *Physic ttl = 30 * time.Second } - // If "tiflash_mpp" is set in tidb_isolation_read_engines, tasks will only be sent to ReadNodes when building batchCopTask. - storeType, err := variable.GetTiFlashEngine(e.ctx.GetSessionVars().GetIsolationReadEngines()) - if err != nil { - return nil, err - } - metas, err := e.ctx.GetMPPClient().ConstructMPPTasks(ctx, req, e.ctx.GetSessionVars().MPPStoreLastFailTime, ttl, storeType) + metas, err := e.ctx.GetMPPClient().ConstructMPPTasks(ctx, req, e.ctx.GetSessionVars().MPPStoreLastFailTime, ttl) if err != nil { return nil, errors.Trace(err) } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 4ec8224e1be30..8277599840fc4 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -677,15 +677,11 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { ds.preferStoreType = 0 return } - if config.GetGlobalConfig().DisaggregatedTiFlash { - bo := backoff.NewBackofferWithVars(context.Background(), 5000, nil) - mppStores, err := ds.ctx.GetStore().(tikv.Storage).GetRegionCache().GetTiFlashMPPStores(bo.TiKVBackoffer()) - if err != nil || len(mppStores) == 0 { - errMsg := fmt.Sprintf("TiFlash ReadNodes number is zero") - warning := ErrInternal.GenWithStack(errMsg) - ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) - return - } + if config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashReadNodeAvailable(ds.ctx) { + errMsg := fmt.Sprintf("TiFlash ReadNodes number is zero") + warning := ErrInternal.GenWithStack(errMsg) + ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + return } for _, path := range ds.possibleAccessPaths { if path.StoreType == kv.TiFlash { @@ -704,6 +700,15 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { } } +func isTiFlashReadNodeAvailable(ctx sessionctx.Context) bool { + bo := backoff.NewBackofferWithVars(context.Background(), 5000, nil) + mppStores, err := ctx.GetStore().(tikv.Storage).GetRegionCache().GetTiFlashMPPStores(bo.TiKVBackoffer()) + if err != nil || len(mppStores) == 0 { + return false + } + return true +} + func resetNotNullFlag(schema *expression.Schema, start, end int) { for i := start; i < end; i++ { col := *schema.Columns[i] diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index c14e1962ac72d..2823da8b9954a 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1305,6 +1305,7 @@ func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, availableEngine := map[kv.StoreType]struct{}{} var availableEngineStr string for i := len(paths) - 1; i >= 0; i-- { + // For warning message. if _, ok := availableEngine[paths[i].StoreType]; !ok { availableEngine[paths[i].StoreType] = struct{}{} if availableEngineStr != "" { @@ -1312,7 +1313,11 @@ func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, } availableEngineStr += paths[i].StoreType.Name() } - if _, ok := isolationReadEngines[paths[i].StoreType]; !ok && paths[i].StoreType != kv.TiDB { + _, exists := isolationReadEngines[paths[i].StoreType] + // Prune this path if: + // 1. path.StoreType doesn't exists in isolationReadEngines or + // 2. TiFlash is disaggregated and the number of ReadNode is zero. + if (!exists && paths[i].StoreType != kv.TiDB) || (exists && paths[i].StoreType == kv.TiFlash && config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashReadNodeAvailable(ctx)) { paths = append(paths[:i], paths[i+1:]...) } } diff --git a/session/session.go b/session/session.go index 7592c799ba991..b4e7fb0b870ee 100644 --- a/session/session.go +++ b/session/session.go @@ -2785,10 +2785,12 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { return nil, err } - // Invalid client-go tiflash_mpp stores if necessary. - err = dom.WatchTiFlashMPPStoreChange() - if err != nil { - return nil, err + if !config.GetGlobalConfig().DisaggregatedTiFlash { + // Invalid client-go tiflash_mpp stores if necessary. + err = dom.WatchTiFlashMPPStoreChange() + if err != nil { + return nil, err + } } if len(cfg.Instance.PluginLoad) > 0 { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index a2dc88fa7dd92..37fd015fce28f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -264,8 +264,6 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TiDBIsolationReadEngines, Value: strings.Join(config.GetGlobalConfig().IsolationRead.Engines, ","), Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { engines := strings.Split(normalizedValue, ",") var formatVal string - var hasTiFlash bool - var hasTiFlashMPP bool for i, engine := range engines { engine = strings.TrimSpace(engine) if i != 0 { @@ -276,19 +274,12 @@ var defaultSysVars = []*SysVar{ formatVal += kv.TiKV.Name() case strings.EqualFold(engine, kv.TiFlash.Name()): formatVal += kv.TiFlash.Name() - hasTiFlash = true case strings.EqualFold(engine, kv.TiDB.Name()): formatVal += kv.TiDB.Name() - case strings.EqualFold(engine, kv.TiFlashMPP.Name()): - formatVal += kv.TiFlashMPP.Name() - hasTiFlashMPP = true default: return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBIsolationReadEngines, normalizedValue) } } - if hasTiFlashMPP && !hasTiFlash { - return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(TiDBIsolationReadEngines, "tiflash_mpp(tiflash must be set at the same time)") - } return formatVal, nil }, SetSession: func(s *SessionVars, val string) error { s.IsolationReadEngines = make(map[kv.StoreType]struct{}) @@ -300,8 +291,6 @@ var defaultSysVars = []*SysVar{ s.IsolationReadEngines[kv.TiFlash] = struct{}{} case kv.TiDB.Name(): s.IsolationReadEngines[kv.TiDB] = struct{}{} - case kv.TiFlashMPP.Name(): - s.IsolationReadEngines[kv.TiFlashMPP] = struct{}{} } } return nil diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 433ede29b4587..811cf65bb50b6 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -22,7 +22,6 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" @@ -499,22 +498,3 @@ var GAFunction4ExpressionIndex = map[string]struct{}{ ast.JSONKeys: {}, ast.JSONLength: {}, } - -// GetTiFlashEngine return kv.TiFlash or kv.TiFlashMPP. -func GetTiFlashEngine(readEngines map[kv.StoreType]struct{}) (res kv.StoreType, _ error) { - _, hasTiFlash := readEngines[kv.TiFlash] - _, hasTiFlashMPP := readEngines[kv.TiFlashMPP] - - if !hasTiFlash && !hasTiFlashMPP { - return kv.UnSpecified, errors.New("cannot get tiflash engine, nor tiflash or tiflash_mpp exists in readEngines") - } - if hasTiFlashMPP && !hasTiFlash { - return kv.UnSpecified, errors.New("tiflash must be set when tiflash_mpp exists in readEngines") - } - if hasTiFlashMPP { - res = kv.TiFlashMPP - } else { - res = kv.TiFlash - } - return res, nil -} diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index d35d023d14d1d..561bb4c04f03a 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/log" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" @@ -535,14 +536,11 @@ func buildBatchCopTasksForNonPartitionedTable(bo *backoff.Backoffer, mppStoreLastFailTime *sync.Map, ttl time.Duration, balanceWithContinuity bool, - balanceContinuousRegionCount int64, - engine kv.StoreType) ([]*batchCopTask, error) { - if engine == kv.TiFlash { - return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) - } else if engine == kv.TiFlashMPP { + balanceContinuousRegionCount int64) ([]*batchCopTask, error) { + if config.GetGlobalConfig().DisaggregatedTiFlash { return buildBatchCopTasksConsistentHash(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) } else { - return nil, errors.New(fmt.Sprintf("unexpected engine type: %v", engine)) + return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) } } @@ -554,14 +552,11 @@ func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, ttl time.Duration, balanceWithContinuity bool, balanceContinuousRegionCount int64, - partitionIDs []int64, - engine kv.StoreType) (batchTasks []*batchCopTask, err error) { - if engine == kv.TiFlash { - batchTasks, err = buildBatchCopTasksCore(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) - } else if engine == kv.TiFlashMPP { + partitionIDs []int64) (batchTasks []*batchCopTask, err error) { + if config.GetGlobalConfig().DisaggregatedTiFlash { batchTasks, err = buildBatchCopTasksConsistentHash(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) } else { - err = errors.New(fmt.Sprintf("unexpected engine type: %v", engine)) + batchTasks, err = buildBatchCopTasksCore(bo, store, rangesForEachPhysicalTable, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) } if err != nil { return nil, err @@ -609,124 +604,14 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran task.ctx.Store = store task.ctx.Addr = addr } - logutil.BgLogger().Info(fmt.Sprintf("gjt debug batchTasks: len: %d\n", len(batchTasks))) - for i, task := range batchTasks { - logutil.BgLogger().Info(fmt.Sprintf("gjt debug each batchTasks[%d], addr: %s, len(RegionInfo): %d\n", i, task.storeAddr, len(task.regionInfos))) + // TODO: Add metrics + logutil.BgLogger().Info("build batchCop tasks using ConsistentHash done.", zap.Int("len(tasks)", len(batchTasks))) + for _, task := range batchTasks { + logutil.BgLogger().Debug("batchTasks detailed info", zap.String("addr", task.storeAddr), zap.Int("RegionInfo number", len(task.regionInfos))) } return batchTasks, nil } -// // 1. Split range by region location to build copTasks. -// // 2. For each copTask build its rpcCtx , the target ReadNode will be computed using consistent hash. -// // 3. All copTasks that will be sent to one ReadNode are put in one batchCopTask. -// func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, rangesForEachPhysicalTable []*KeyRanges, storeType kv.StoreType) (res []*batchCopTask, err error) { -// const cmdType = tikvrpc.CmdBatchCop -// var retryNum int -// cache := store.GetRegionCache() -// -// for { -// retryNum++ -// if retryNum >= 10 { -// return nil, errors.New("too many times of retry to GetTiFlashMPPRPCContextByConsistentHash()") -// } -// -// var rangesLen int -// tasks := make([]*copTask, 0) -// regionIDs := make([]tikv.RegionVerID, 0) -// -// for i, ranges := range rangesForEachPhysicalTable { -// rangesLen += ranges.Len() -// locations, err := cache.SplitKeyRangesByLocations(bo, ranges) -// if err != nil { -// return nil, errors.Trace(err) -// } -// for _, lo := range locations { -// tasks = append(tasks, &copTask{ -// region: lo.Location.Region, -// ranges: lo.Ranges, -// cmdType: cmdType, -// storeType: storeType, -// partitionIndex: int64(i), -// }) -// regionIDs = append(regionIDs, lo.Location.Region) -// } -// } -// -// rpcCtxs, err := cache.GetTiFlashMPPRPCContextByConsistentHash(bo.TiKVBackoffer(), regionIDs) -// if err != nil { -// return nil, err -// } -// if rpcCtxs == nil { -// logutil.BgLogger().Info("buildBatchCopTasksConsistentHash retry because rcpCtx is nil", zap.Int("retryNum", retryNum)) -// continue -// } -// if len(rpcCtxs) != len(tasks) { -// return nil, errors.New(fmt.Sprintf("unexpected length of rpcCtxs, expect: %v, got: %v", len(tasks), len(rpcCtxs))) -// } -// logutil.BgLogger().Info(fmt.Sprintf("gjt debug cop tsaks len(tasks): %v\n", len(tasks))) -// for i, t := range tasks { -// logutil.BgLogger().Info(fmt.Sprintf("gjt debug regoin: %v, cmdType: %v, storeType: %v\n", t.region, t.cmdType, t.storeType)) -// logutil.BgLogger().Info(fmt.Sprintf("gjt debug rpcCtxs. id: %v, region: %v, addr: %v\n", i, rpcCtxs[i].Region, rpcCtxs[i].Addr)) -// } -// taskMap := make(map[string]*batchCopTask) -// for i, rpcCtx := range rpcCtxs { -// regionInfo := RegionInfo{ -// // tasks and rpcCtxs are correspond to each other. -// Region: tasks[i].region, -// Meta: rpcCtx.Meta, -// Ranges: tasks[i].ranges, -// AllStores: []uint64{rpcCtx.Store.StoreID()}, -// PartitionIndex: tasks[i].partitionIndex, -// } -// if batchTask, ok := taskMap[rpcCtx.Addr]; ok { -// batchTask.regionInfos = append(batchTask.regionInfos, regionInfo) -// } else { -// batchTask := &batchCopTask{ -// storeAddr: rpcCtx.Addr, -// cmdType: cmdType, -// ctx: rpcCtx, -// regionInfos: []RegionInfo{regionInfo}, -// } -// taskMap[rpcCtx.Addr] = batchTask -// res = append(res, batchTask) -// } -// } -// logutil.BgLogger().Info(fmt.Sprintf("gjt debug all batchTasks: len(tasksMap): %v", len(taskMap))) -// for addr, t := range taskMap { -// logutil.BgLogger().Info(fmt.Sprintf("gjt debug addr: %v, len(RegionInfo): %v", addr, len(t.regionInfos))) -// for i, reg := range t.regionInfos { -// logutil.BgLogger().Info(fmt.Sprintf("gjt debug region id: %v, reg: %v", i, reg)) -// } -// } -// break -// } -// -// failpoint.Inject("check_only_dispatched_to_tiflash_mpp_nodes", func(val failpoint.Value) { -// // This failpoint will be tested in test-infra case, because we needs setup a cluster. -// // All ReadNode addrs are stored in val, each addr is separated by semicolon. -// str := val.(string) -// addrs := strings.Split(str, ";") -// if len(addrs) < 1 { -// panic(fmt.Sprintf("unexpected length of tiflash_mpp node addrs: %v", len(addrs))) -// } -// for _, batchTask := range res { -// var matched bool -// for _, addr := range addrs { -// if batchTask.storeAddr == addr { -// matched = true -// break -// } -// } -// if !matched { -// panic(fmt.Sprintf("batchCopTask send to node which is not tiflash_mpp: %v(tiflash_mpp nodes: %s)", -// batchTask.storeAddr, str)) -// } -// } -// logutil.BgLogger().Info(fmt.Sprintf("check_only_dispatched_to_tiflash_mpp_nodes checked succeed(tiflash_mpp nodes: %s)", str)) -// }) -// return res, nil -// } - // When `partitionIDs != nil`, it means that buildBatchCopTasksCore is constructing a batch cop tasks for PartitionTableScan. // At this time, `len(rangesForEachPhysicalTable) == len(partitionIDs)` and `rangesForEachPhysicalTable[i]` is for partition `partitionIDs[i]`. // Otherwise, `rangesForEachPhysicalTable[0]` indicates the range for the single physical table. @@ -860,7 +745,7 @@ func convertRegionInfosToPartitionTableRegions(batchTasks []*batchCopTask, parti } } -func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.Variables, option *kv.ClientSendOption, storeType kv.StoreType) kv.Response { +func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.Variables, option *kv.ClientSendOption) kv.Response { if req.KeepOrder || req.Desc { return copErrorResponse{errors.New("batch coprocessor cannot prove keep order or desc property")} } @@ -877,10 +762,10 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V keyRanges = append(keyRanges, NewKeyRanges(pi.KeyRanges)) partitionIDs = append(partitionIDs, pi.ID) } - tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store.kvStore, keyRanges, req.StoreType, nil, 0, false, 0, partitionIDs, storeType) + tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store.kvStore, keyRanges, req.StoreType, nil, 0, false, 0, partitionIDs) } else { ranges := NewKeyRanges(req.KeyRanges) - tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store.kvStore, ranges, req.StoreType, nil, 0, false, 0, storeType) + tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store.kvStore, ranges, req.StoreType, nil, 0, false, 0) } if err != nil { @@ -893,7 +778,6 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V vars: vars, rpcCancel: tikv.NewRPCanceller(), enableCollectExecutionInfo: option.EnableCollectExecutionInfo, - storeType: storeType, } ctx = context.WithValue(ctx, tikv.RPCCancellerCtxKey{}, it.rpcCancel) it.tasks = tasks @@ -923,9 +807,6 @@ type batchCopIterator struct { closed uint32 enableCollectExecutionInfo bool - - // For tiflash_mpp, will use consistent hashsing to dispatch batchCopTask. - storeType kv.StoreType } func (b *batchCopIterator) run(ctx context.Context) { @@ -1031,7 +912,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoff.Ba ranges = append(ranges, *ran) }) } - ret, err := buildBatchCopTasksForNonPartitionedTable(bo, b.store, NewKeyRanges(ranges), b.req.StoreType, nil, 0, false, 0, b.storeType) + ret, err := buildBatchCopTasksForNonPartitionedTable(bo, b.store, NewKeyRanges(ranges), b.req.StoreType, nil, 0, false, 0) return ret, err } // Retry Partition Table Scan @@ -1050,7 +931,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoff.Ba } keyRanges = append(keyRanges, NewKeyRanges(ranges)) } - ret, err := buildBatchCopTasksForPartitionedTable(bo, b.store, keyRanges, b.req.StoreType, nil, 0, false, 0, pid, b.storeType) + ret, err := buildBatchCopTasksForPartitionedTable(bo, b.store, keyRanges, b.req.StoreType, nil, 0, false, 0, pid) return ret, err } @@ -1083,7 +964,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *backoff.Backo if b.req.ResourceGroupTagger != nil { b.req.ResourceGroupTagger(req) } - req.StoreTp = getEndPointType(b.storeType) + req.StoreTp = getEndPointType(kv.TiFlash) logutil.BgLogger().Debug("send batch request to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.regionInfos))) resp, retry, cancel, err := sender.SendReqToAddr(bo, task.ctx, task.regionInfos, req, readTimeoutUltraLong) diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index 4b422156e377e..124b88aa51d75 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -16,14 +16,12 @@ package copr import ( "context" - "fmt" "time" "github.com/pingcap/errors" - "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/config" tikverr "github.com/tikv/client-go/v2/error" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" @@ -68,24 +66,6 @@ func NewRegionBatchRequestSender(cache *RegionCache, client tikv.Client, enableC // SendReqToAddr send batch cop request func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *tikv.RPCContext, regionInfos []RegionInfo, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { - failpoint.Inject("check_store_type_of_batch_cop_task", func(val failpoint.Value) { - // Will be tested in test-infra. - storeTpStr := val.(string) - expectStoreTp := tikvrpc.TiFlash - switch storeTpStr { - case "tiflash": - expectStoreTp = tikvrpc.TiFlash - case "tiflash_mpp": - expectStoreTp = tikvrpc.TiFlashMPP - default: - panic("unexpected failpoint val, must be tiflash or tiflash_mpp") - } - if expectStoreTp != req.StoreTp { - panic(fmt.Sprintf("unexpected store type, expect: %v, got: %v", expectStoreTp, req.StoreTp)) - } - logutil.BgLogger().Info("failpoint check_store_type_of_batch_cop_task succeed") - }) - cancel = func() {} if e := tikvrpc.SetContext(req, rpcCtx.Meta, rpcCtx.Peer); e != nil { return nil, false, cancel, errors.Trace(e) @@ -102,7 +82,7 @@ func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *tikv.RP if err != nil { cancel() ss.SetRPCError(err) - e := ss.onSendFailForBatchRegions(bo, rpcCtx, regionInfos, err, req.StoreTp) + e := ss.onSendFailForBatchRegions(bo, rpcCtx, regionInfos, err) if e != nil { return nil, false, func() {}, errors.Trace(e) } @@ -112,7 +92,7 @@ func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *tikv.RP return } -func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx *tikv.RPCContext, regionInfos []RegionInfo, err error, storeTp tikvrpc.EndpointType) error { +func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx *tikv.RPCContext, regionInfos []RegionInfo, err error) error { // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { return errors.Trace(err) @@ -120,7 +100,7 @@ func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx return tikverr.ErrTiDBShuttingDown } - if storeTp == tikvrpc.TiFlashMPP { + if config.GetGlobalConfig().DisaggregatedTiFlash { ss.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) } else { // The reload region param is always true. Because that every time we try, we must diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 224ae4794ce29..d848ba8e01f1f 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -85,9 +85,9 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa return copErrorResponse{errors.Errorf("unsupported variables:%+v", variables)} } // req.StoreType is setup in TableReader. - if (req.StoreType == kv.TiFlash || req.StoreType == kv.TiFlashMPP) && req.BatchCop { + if req.StoreType == kv.TiFlash && req.BatchCop { logutil.BgLogger().Debug("send batch requests") - return c.sendBatch(ctx, req, vars, option, req.StoreType) + return c.sendBatch(ctx, req, vars, option) } failpoint.Inject("DisablePaging", func(_ failpoint.Value) { req.Paging.Enable = false diff --git a/store/copr/mpp.go b/store/copr/mpp.go index c9a22f76f29fb..b00b6a325975f 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -62,7 +62,7 @@ func (c *MPPClient) selectAllTiFlashStore() []kv.MPPTaskMeta { } // ConstructMPPTasks receives ScheduleRequest, which are actually collects of kv ranges. We allocates MPPTaskMeta for them and returns. -func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest, mppStoreLastFailTime *sync.Map, ttl time.Duration, engine kv.StoreType) ([]kv.MPPTaskMeta, error) { +func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest, mppStoreLastFailTime *sync.Map, ttl time.Duration) ([]kv.MPPTaskMeta, error) { ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTS) bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) var tasks []*batchCopTask @@ -74,13 +74,13 @@ func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasks rangesForEachPartition[i] = NewKeyRanges(p.KeyRanges) partitionIDs[i] = p.ID } - tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store, rangesForEachPartition, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20, partitionIDs, engine) + tasks, err = buildBatchCopTasksForPartitionedTable(bo, c.store, rangesForEachPartition, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20, partitionIDs) } else { if req.KeyRanges == nil { return c.selectAllTiFlashStore(), nil } ranges := NewKeyRanges(req.KeyRanges) - tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store, ranges, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20, engine) + tasks, err = buildBatchCopTasksForNonPartitionedTable(bo, c.store, ranges, kv.TiFlash, mppStoreLastFailTime, ttl, true, 20) } if err != nil { @@ -235,19 +235,14 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req } } + disaggregatedTiFlash := config.GetGlobalConfig().DisaggregatedTiFlash + wrappedReq := tikvrpc.NewRequest(tikvrpc.CmdMPPTask, mppReq, kvrpcpb.Context{}) + wrappedReq.StoreTp = getEndPointType(kv.TiFlash) + // TODO: Handle dispatch task response correctly, including retry logic and cancel logic. var rpcResp *tikvrpc.Response var err error var retry bool - var tp tikvrpc.EndpointType - - wrappedReq := tikvrpc.NewRequest(tikvrpc.CmdMPPTask, mppReq, kvrpcpb.Context{}) - tp, err = getTiFlashEndPointType(req.StoreTp) - if err != nil { - m.sendError(err) - return - } - wrappedReq.StoreTp = tp // If copTasks is not empty, we should send request according to region distribution. // Or else it's the task without region, which always happens in high layer task without table. @@ -255,7 +250,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req if originalTask != nil { sender := NewRegionBatchRequestSender(m.store.GetRegionCache(), m.store.GetTiKVClient(), m.enableCollectExecutionInfo) rpcResp, retry, _, err = sender.SendReqToAddr(bo, originalTask.ctx, originalTask.regionInfos, wrappedReq, tikv.ReadTimeoutMedium) - if err != nil && req.StoreTp == kv.TiFlashMPP { + if err != nil && disaggregatedTiFlash { m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) } // No matter what the rpc error is, we won't retry the mpp dispatch tasks. @@ -275,7 +270,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { retry = false } else if err != nil { - if req.StoreTp == kv.TiFlashMPP { + if disaggregatedTiFlash { m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) } if bo.Backoff(tikv.BoTiFlashRPC(), err) == nil { @@ -340,22 +335,15 @@ func (m *mppIterator) cancelMppTasks() { Meta: &mpp.TaskMeta{StartTs: m.startTs}, } + disaggregatedTiFlash := config.GetGlobalConfig().DisaggregatedTiFlash wrappedReq := tikvrpc.NewRequest(tikvrpc.CmdMPPCancel, killReq, kvrpcpb.Context{}) - var tp tikvrpc.EndpointType - var err error + wrappedReq.StoreTp = getEndPointType(kv.TiFlash) - // Value type is kv.StoreType, so we can decide whether to invalid tiflash_mpp store cache or not. - usedStoreAddrs := make(map[string]kv.StoreType) + usedStoreAddrs := make(map[string]bool) for _, task := range m.tasks { // get the store address of running tasks - if _, ok := usedStoreAddrs[task.Meta.GetAddress()]; ok && task.State == kv.MppTaskRunning { - usedStoreAddrs[task.Meta.GetAddress()] = task.StoreTp - tp, err = getTiFlashEndPointType(task.StoreTp) - if err != nil { - m.sendError(err) - return - } - wrappedReq.StoreTp = tp + if task.State == kv.MppTaskRunning && !usedStoreAddrs[task.Meta.GetAddress()] { + usedStoreAddrs[task.Meta.GetAddress()] = true } else if task.State == kv.MppTaskCancelled { return } @@ -364,16 +352,16 @@ func (m *mppIterator) cancelMppTasks() { // send cancel cmd to all stores where tasks run wg := util.WaitGroupWrapper{} - for addr, storeTp := range usedStoreAddrs { + for addr := range usedStoreAddrs { storeAddr := addr wg.Run(func() { _, err := m.store.GetTiKVClient().SendRequest(context.Background(), storeAddr, wrappedReq, tikv.ReadTimeoutShort) logutil.BgLogger().Debug("cancel task", zap.Uint64("query id ", m.startTs), zap.String("on addr", storeAddr)) if err != nil { - if storeTp == kv.TiFlashMPP { + logutil.BgLogger().Error("cancel task error", zap.Error(err), zap.Uint64("query id", m.startTs), zap.String("on addr", storeAddr)) + if disaggregatedTiFlash { m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) } - logutil.BgLogger().Error("cancel task error", zap.Error(err), zap.Uint64("query id", m.startTs), zap.String("on addr", storeAddr)) } }) } @@ -390,14 +378,10 @@ func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchReques } var err error - var tp tikvrpc.EndpointType + disaggregatedTiFlash := config.GetGlobalConfig().DisaggregatedTiFlash + wrappedReq := tikvrpc.NewRequest(tikvrpc.CmdMPPConn, connReq, kvrpcpb.Context{}) - tp, err = getTiFlashEndPointType(req.StoreTp) - if err != nil { - m.sendError(err) - return - } - wrappedReq.StoreTp = tp + wrappedReq.StoreTp = getEndPointType(kv.TiFlash) // Drain results from root task. // We don't need to process any special error. When we meet errors, just let it fail. @@ -411,7 +395,7 @@ func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchReques } else { m.sendError(err) } - if req.StoreTp == kv.TiFlashMPP { + if disaggregatedTiFlash { m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) } return diff --git a/store/copr/store.go b/store/copr/store.go index 73060cceb4e2b..6af8bf6febcc5 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -16,12 +16,12 @@ package copr import ( "context" - "fmt" "math/rand" "sync/atomic" "time" "github.com/pingcap/errors" + tidb_config "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" @@ -123,22 +123,16 @@ func getEndPointType(t kv.StoreType) tikvrpc.EndpointType { case kv.TiKV: return tikvrpc.TiKV case kv.TiFlash: + if tidb_config.GetGlobalConfig().DisaggregatedTiFlash { + return tikvrpc.TiFlashMPP + } return tikvrpc.TiFlash case kv.TiDB: return tikvrpc.TiDB - case kv.TiFlashMPP: - return tikvrpc.TiFlashMPP default: return tikvrpc.TiKV } } -func getTiFlashEndPointType(t kv.StoreType) (tikvrpc.EndpointType, error) { - tp := getEndPointType(t) - if tp != tikvrpc.TiFlash && tp != tikvrpc.TiFlashMPP { - return tp, errors.New(fmt.Sprintf("unexpected endpoint tp, expect tiflash or tiflash_mpp, got: %v", tp)) - } - return tp, nil -} // Backoffer wraps tikv.Backoffer and converts the error which returns by the functions of tikv.Backoffer to tidb error. type Backoffer = backoff.Backoffer From 99e6f3268e5be2fc64b3c069a3db78bc0478d030 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 21 Sep 2022 16:13:07 +0800 Subject: [PATCH 27/44] fix go.mod Signed-off-by: guo-shaoge --- go.mod | 2 -- go.sum | 2 ++ 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 99eebd97ea197..e758f294e119d 100644 --- a/go.mod +++ b/go.mod @@ -243,5 +243,3 @@ replace ( github.com/pingcap/tidb/parser => ./parser go.opencensus.io => go.opencensus.io v0.23.1-0.20220331163232-052120675fac ) - -replace github.com/tikv/client-go/v2 => /home/guojiangtao/work/client-go diff --git a/go.sum b/go.sum index 55337c0bb72ce..7cbe99ed6235e 100644 --- a/go.sum +++ b/go.sum @@ -901,6 +901,8 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= 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/tikv/client-go/v2 v2.0.1-0.20220913051514-ffaaf7131a8d h1:p8XInTnkUlLabBT7bDS3aZCeemO6tJ/7b5mHN8WbSIE= +github.com/tikv/client-go/v2 v2.0.1-0.20220913051514-ffaaf7131a8d/go.mod h1:6pedLz7wiINLHXwCT1+yMZmzuG42+ubtBkkfcwoukIo= github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db h1:r1eMh9Rny3hfWuBuxOnbsCRrR4FhthiNxLQ5rAUtaww= github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db/go.mod h1:ew8kS0yIcEaSetuuywkTLIUBR+sz3J5XvAYRae11qwc= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro= From 639585fc32864f93940ac9edff65766ebb96f4b5 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 21 Sep 2022 16:54:57 +0800 Subject: [PATCH 28/44] fix config fmt Signed-off-by: guo-shaoge --- config/config.go | 2 +- kv/kv.go | 1 - store/copr/coprocessor.go | 1 - 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/config/config.go b/config/config.go index 999493a0949eb..5a430b58bf6ac 100644 --- a/config/config.go +++ b/config/config.go @@ -276,7 +276,7 @@ type Config struct { Plugin Plugin `toml:"plugin" json:"plugin"` MaxServerConnections uint32 `toml:"max-server-connections" json:"max-server-connections"` RunDDL bool `toml:"run-ddl" json:"run-ddl"` - DisaggregatedTiFlash bool `toml:"disaggregated_tiflash" json: "disaggregated_tiflash"` + DisaggregatedTiFlash bool `toml:"disaggregated-tiflash" json: "disaggregated-tiflash"` } // UpdateTempStoragePath is to update the `TempStoragePath` if port/statusPort was changed diff --git a/kv/kv.go b/kv/kv.go index 2ca6bdcc561dd..d39dc1ee5c862 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -315,7 +315,6 @@ const ( TiFlash // TiDB means the type of a store is TiDB. TiDB - // UnSpecified means the store type is unknown UnSpecified = 255 ) diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index d848ba8e01f1f..c46ec0e6f9e1b 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -84,7 +84,6 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa if !ok { return copErrorResponse{errors.Errorf("unsupported variables:%+v", variables)} } - // req.StoreType is setup in TableReader. if req.StoreType == kv.TiFlash && req.BatchCop { logutil.BgLogger().Debug("send batch requests") return c.sendBatch(ctx, req, vars, option) From 21d3d0a7649ebcf8634c263df5b4ae6bf2f25b44 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 21 Sep 2022 20:25:04 +0800 Subject: [PATCH 29/44] fix config fmt Signed-off-by: guo-shaoge --- config/config.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/config.go b/config/config.go index 5a430b58bf6ac..86c2be689b083 100644 --- a/config/config.go +++ b/config/config.go @@ -276,7 +276,7 @@ type Config struct { Plugin Plugin `toml:"plugin" json:"plugin"` MaxServerConnections uint32 `toml:"max-server-connections" json:"max-server-connections"` RunDDL bool `toml:"run-ddl" json:"run-ddl"` - DisaggregatedTiFlash bool `toml:"disaggregated-tiflash" json: "disaggregated-tiflash"` + DisaggregatedTiFlash bool `toml:"disaggregated-tiflash" json:"disaggregated-tiflash"` } // UpdateTempStoragePath is to update the `TempStoragePath` if port/statusPort was changed From aab99125d2c987a7cc9161070320b02ac2f8a3db Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 22 Sep 2022 17:21:14 +0800 Subject: [PATCH 30/44] fix Signed-off-by: guo-shaoge --- store/copr/batch_coprocessor.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 561bb4c04f03a..bdeeeaa32e307 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -21,7 +21,6 @@ import ( "io" "math" "strconv" - _ "strings" "sync" "sync/atomic" "time" @@ -539,9 +538,8 @@ func buildBatchCopTasksForNonPartitionedTable(bo *backoff.Backoffer, balanceContinuousRegionCount int64) ([]*batchCopTask, error) { if config.GetGlobalConfig().DisaggregatedTiFlash { return buildBatchCopTasksConsistentHash(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) - } else { - return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) } + return buildBatchCopTasksCore(bo, store, []*KeyRanges{ranges}, storeType, mppStoreLastFailTime, ttl, balanceWithContinuity, balanceContinuousRegionCount) } func buildBatchCopTasksForPartitionedTable(bo *backoff.Backoffer, @@ -597,7 +595,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran } } if store == nil { - return nil, errors.New(fmt.Sprintf("cannot find mpp store: %v", addr)) + return nil, errors.New("cannot find mpp store: " + addr) } task.storeAddr = addr From c19d5bad48f8bbf3d8e4e31a805a81ffd85be54b Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Fri, 23 Sep 2022 11:20:42 +0800 Subject: [PATCH 31/44] fix bazel_lint Signed-off-by: guo-shaoge --- planner/core/logical_plan_builder.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 8277599840fc4..5c7da0675ec3b 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -678,7 +678,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { return } if config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashReadNodeAvailable(ds.ctx) { - errMsg := fmt.Sprintf("TiFlash ReadNodes number is zero") + errMsg := "The number of TiFlash ReadNode is zero, cannot use TiFlash" warning := ErrInternal.GenWithStack(errMsg) ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) return From 14353279a98ac1b36d61ccc10426f7f7f6f7f251 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 11 Oct 2022 19:50:56 +0800 Subject: [PATCH 32/44] change all tiflash_mpp to tiflash_compute Signed-off-by: guo-shaoge --- domain/domain.go | 22 +++++++++++----------- executor/table_reader.go | 9 ++++----- planner/core/fragment.go | 1 - planner/core/logical_plan_builder.go | 11 ++++++----- planner/core/planbuilder.go | 6 +++--- session/session.go | 4 ++-- store/copr/batch_coprocessor.go | 12 ++++++------ store/copr/batch_request_sender.go | 2 +- store/copr/mpp.go | 8 ++++---- store/copr/store.go | 2 +- util/engine/engine.go | 2 +- 11 files changed, 39 insertions(+), 40 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 5f37956ebaf76..60fcdb5d4b389 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1362,21 +1362,21 @@ func (do *Domain) LoadSysVarCacheLoop(ctx sessionctx.Context) error { return nil } -// WatchTiFlashMPPStoreChange create a routine to watch. -// TODO: tiflashMPPStoreKey is not put to etcd yet(finish this when AutoScaler is done) +// WatchTiFlashComputeStoreChange create a routine to watch if the topology of tiflash_compute node is changed. +// TODO: tiflashComputeStoreKey is not put to etcd yet(finish this when AutoScaler is done) // // store cache will only be invalidated every 30 seconds. -func (do *Domain) WatchTiFlashMPPStoreChange() error { +func (do *Domain) WatchTiFlashComputeStoreChange() error { var watchCh clientv3.WatchChan if do.etcdClient != nil { - watchCh = do.etcdClient.Watch(context.Background(), tiflashMPPStoreKey) + watchCh = do.etcdClient.Watch(context.Background(), tiflashComputeStoreKey) } do.wg.Add(1) duration := 30 * time.Second go func() { defer func() { do.wg.Done() - logutil.BgLogger().Info("WatchTiFlashMPPStoreChange exit") + logutil.BgLogger().Info("WatchTiFlashComputeStoreChange exit") }() var count int @@ -1391,8 +1391,8 @@ func (do *Domain) WatchTiFlashMPPStoreChange() error { case <-time.After(duration): } if !ok { - logutil.BgLogger().Error("WatchTiFlashMPPStoreChange watch channel closed") - watchCh = do.etcdClient.Watch(context.Background(), tiflashMPPStoreKey) + logutil.BgLogger().Error("WatchTiFlashComputeStoreChange watch channel closed") + watchCh = do.etcdClient.Watch(context.Background(), tiflashComputeStoreKey) count++ if count > 10 { time.Sleep(time.Duration(count) * time.Second) @@ -1402,10 +1402,10 @@ func (do *Domain) WatchTiFlashMPPStoreChange() error { count = 0 switch s := do.store.(type) { case tikv.Storage: - s.GetRegionCache().InvalidateTiFlashMPPStores() - logutil.BgLogger().Debug("tiflash_mpp store cache invalied, will update next query", zap.Bool("watched", watched)) + s.GetRegionCache().InvalidateTiFlashComputeStores() + logutil.BgLogger().Debug("tiflash_compute store cache invalied, will update next query", zap.Bool("watched", watched)) default: - logutil.BgLogger().Debug("No need to watch TiFlashMPP for non-tikv store") + logutil.BgLogger().Debug("No need to watch tiflash_compute store cache for non-tikv store") return } } @@ -1872,7 +1872,7 @@ func (do *Domain) ExpensiveQueryHandle() *expensivequery.Handle { const ( privilegeKey = "/tidb/privilege" sysVarCacheKey = "/tidb/sysvars" - tiflashMPPStoreKey = "/tiflash/new_tiflash_mpp_stores" + tiflashComputeStoreKey = "/tiflash/new_tiflash_compute_stores" ) // NotifyUpdatePrivilege updates privilege key in etcd, TiDB client that watches diff --git a/executor/table_reader.go b/executor/table_reader.go index 097beb1c8a547..fd162a63bfca9 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -276,7 +276,7 @@ func (e *TableReaderExecutor) Close() error { // buildResp first builds request and sends it to tikv using distsql.Select. It uses SelectResult returned by the callee // to fetch all results. -func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (_ distsql.SelectResult, err error) { +func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (distsql.SelectResult, error) { if e.storeType == kv.TiFlash && e.kvRangeBuilder != nil { if !e.batchCop { // TiFlash cannot support to access multiple tables/partitions within one KVReq, so we have to build KVReq for each partition separately. @@ -322,13 +322,12 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra return result, nil } -func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges []*ranger.Range) (_ []*kv.Request, err error) { +func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges []*ranger.Range) ([]*kv.Request, error) { pids, kvRanges, err := e.kvRangeBuilder.buildKeyRangeSeparately(ranges) if err != nil { return nil, err } kvReqs := make([]*kv.Request, 0, len(kvRanges)) - for i, kvRange := range kvRanges { e.kvRanges = append(e.kvRanges, kvRange...) if err := updateExecutorTableID(ctx, e.dagPB.RootExecutor, true, []int64{pids[i]}); err != nil { @@ -359,7 +358,7 @@ func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges [ return kvReqs, nil } -func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Context, ranges []*ranger.Range) (_ *kv.Request, err error) { +func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Context, ranges []*ranger.Range) (*kv.Request, error) { pids, kvRanges, err := e.kvRangeBuilder.buildKeyRangeSeparately(ranges) if err != nil { return nil, err @@ -398,7 +397,7 @@ func (e *TableReaderExecutor) buildKVReqForPartitionTableScan(ctx context.Contex return kvReq, nil } -func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.Range) (_ *kv.Request, err error) { +func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.Range) (*kv.Request, error) { var builder distsql.RequestBuilder var reqBuilder *distsql.RequestBuilder if e.kvRangeBuilder != nil { diff --git a/planner/core/fragment.go b/planner/core/fragment.go index df2d537a802a8..ec5785404757f 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -354,7 +354,6 @@ func (e *mppTaskGenerator) constructMPPTasksImpl(ctx context.Context, ts *Physic logutil.BgLogger().Warn("MPP store fail ttl is invalid", zap.Error(err)) ttl = 30 * time.Second } - metas, err := e.ctx.GetMPPClient().ConstructMPPTasks(ctx, req, e.ctx.GetSessionVars().MPPStoreLastFailTime, ttl) if err != nil { return nil, errors.Trace(err) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 5c7da0675ec3b..5da4f2884ba3a 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -677,8 +677,9 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { ds.preferStoreType = 0 return } - if config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashReadNodeAvailable(ds.ctx) { - errMsg := "The number of TiFlash ReadNode is zero, cannot use TiFlash" + if config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashComputeNodeAvailable(ds.ctx) { + // TiFlash is in disaggregated mode, need to make sure tiflash_compute node is available. + errMsg := "Number of tiflash_compute nodes is zero" warning := ErrInternal.GenWithStack(errMsg) ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) return @@ -700,10 +701,10 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { } } -func isTiFlashReadNodeAvailable(ctx sessionctx.Context) bool { +func isTiFlashComputeNodeAvailable(ctx sessionctx.Context) bool { bo := backoff.NewBackofferWithVars(context.Background(), 5000, nil) - mppStores, err := ctx.GetStore().(tikv.Storage).GetRegionCache().GetTiFlashMPPStores(bo.TiKVBackoffer()) - if err != nil || len(mppStores) == 0 { + stores, err := ctx.GetStore().(tikv.Storage).GetRegionCache().GetTiFlashComputeStores(bo.TiKVBackoffer()) + if err != nil || len(stores) == 0 { return false } return true diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 94b30bca14fbd..5241bf8e7db5a 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1305,7 +1305,7 @@ func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, availableEngine := map[kv.StoreType]struct{}{} var availableEngineStr string for i := len(paths) - 1; i >= 0; i-- { - // For warning message. + // availableEngineStr is for warning message. if _, ok := availableEngine[paths[i].StoreType]; !ok { availableEngine[paths[i].StoreType] = struct{}{} if availableEngineStr != "" { @@ -1316,8 +1316,8 @@ func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, _, exists := isolationReadEngines[paths[i].StoreType] // Prune this path if: // 1. path.StoreType doesn't exists in isolationReadEngines or - // 2. TiFlash is disaggregated and the number of ReadNode is zero. - if (!exists && paths[i].StoreType != kv.TiDB) || (exists && paths[i].StoreType == kv.TiFlash && config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashReadNodeAvailable(ctx)) { + // 2. TiFlash is disaggregated and the number of tiflash_compute node is zero. + if (!exists && paths[i].StoreType != kv.TiDB) || (exists && paths[i].StoreType == kv.TiFlash && config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashComputeNodeAvailable(ctx)) { paths = append(paths[:i], paths[i+1:]...) } } diff --git a/session/session.go b/session/session.go index da85b4d81458a..93f0094bf1ba0 100644 --- a/session/session.go +++ b/session/session.go @@ -2799,8 +2799,8 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { } if !config.GetGlobalConfig().DisaggregatedTiFlash { - // Invalid client-go tiflash_mpp stores if necessary. - err = dom.WatchTiFlashMPPStoreChange() + // Invalid client-go tiflash_compute store cache if necessary. + err = dom.WatchTiFlashComputeStoreChange() if err != nil { return nil, err } diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index ba67c45cf79d4..5a0ce87ba2d53 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -572,16 +572,16 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran return nil, err } cache := store.GetRegionCache() - mppStores, err := cache.GetTiFlashMPPStores(bo.TiKVBackoffer()) + stores, err := cache.GetTiFlashComputeStores(bo.TiKVBackoffer()) if err != nil { return nil, err } - if len(mppStores) == 0 { - return nil, errors.New("Number of tiflash_mpp node is zero") + if len(stores) == 0 { + return nil, errors.New("Number of tiflash_compute node is zero") } hasher := consistent.New() - for _, store := range mppStores { + for _, store := range stores { hasher.Add(store.GetAddr()) } for _, task := range batchTasks { @@ -590,14 +590,14 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran return nil, err } var store *tikv.Store - for _, s := range mppStores { + for _, s := range stores { if s.GetAddr() == addr { store = s break } } if store == nil { - return nil, errors.New("cannot find mpp store: " + addr) + return nil, errors.New("cannot find tiflash_compute store: " + addr) } task.storeAddr = addr diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index 124b88aa51d75..3a3bd339f4e6c 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -101,7 +101,7 @@ func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx } if config.GetGlobalConfig().DisaggregatedTiFlash { - ss.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) + ss.GetRegionCache().InvalidateTiFlashComputeStoresIfGRPCError(err) } else { // The reload region param is always true. Because that every time we try, we must // re-build the range then re-create the batch sender. As a result, the len of "failStores" diff --git a/store/copr/mpp.go b/store/copr/mpp.go index b00b6a325975f..1a0df84fa86d1 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -251,7 +251,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req sender := NewRegionBatchRequestSender(m.store.GetRegionCache(), m.store.GetTiKVClient(), m.enableCollectExecutionInfo) rpcResp, retry, _, err = sender.SendReqToAddr(bo, originalTask.ctx, originalTask.regionInfos, wrappedReq, tikv.ReadTimeoutMedium) if err != nil && disaggregatedTiFlash { - m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) + m.store.GetRegionCache().InvalidateTiFlashComputeStoresIfGRPCError(err) } // No matter what the rpc error is, we won't retry the mpp dispatch tasks. // TODO: If we want to retry, we must redo the plan fragment cutting and task scheduling. @@ -271,7 +271,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req retry = false } else if err != nil { if disaggregatedTiFlash { - m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) + m.store.GetRegionCache().InvalidateTiFlashComputeStoresIfGRPCError(err) } if bo.Backoff(tikv.BoTiFlashRPC(), err) == nil { retry = true @@ -360,7 +360,7 @@ func (m *mppIterator) cancelMppTasks() { if err != nil { logutil.BgLogger().Error("cancel task error", zap.Error(err), zap.Uint64("query id", m.startTs), zap.String("on addr", storeAddr)) if disaggregatedTiFlash { - m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) + m.store.GetRegionCache().InvalidateTiFlashComputeStoresIfGRPCError(err) } } }) @@ -396,7 +396,7 @@ func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchReques m.sendError(err) } if disaggregatedTiFlash { - m.store.GetRegionCache().InvalidateTiFlashMPPStoresIfGRPCError(err) + m.store.GetRegionCache().InvalidateTiFlashComputeStoresIfGRPCError(err) } return } diff --git a/store/copr/store.go b/store/copr/store.go index 6af8bf6febcc5..758109b81d805 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -124,7 +124,7 @@ func getEndPointType(t kv.StoreType) tikvrpc.EndpointType { return tikvrpc.TiKV case kv.TiFlash: if tidb_config.GetGlobalConfig().DisaggregatedTiFlash { - return tikvrpc.TiFlashMPP + return tikvrpc.TiFlashCompute } return tikvrpc.TiFlash case kv.TiDB: diff --git a/util/engine/engine.go b/util/engine/engine.go index f71737de6a8ab..0a1614041f3bc 100644 --- a/util/engine/engine.go +++ b/util/engine/engine.go @@ -21,7 +21,7 @@ import ( // IsTiFlash tests whether the store is based on tiflash engine. func IsTiFlash(store *metapb.Store) bool { for _, label := range store.Labels { - if label.Key == "engine" && (label.Value == "tiflash_mpp" || label.Value == "tiflash") { + if label.Key == "engine" && (label.Value == "tiflash_compute" || label.Value == "tiflash") { return true } } From 361d7a892fd0cffa6d0dbf848bf46a370a513e53 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 11 Oct 2022 20:03:54 +0800 Subject: [PATCH 33/44] fix fmt Signed-off-by: guo-shaoge --- domain/domain.go | 4 ++-- planner/core/logical_plan_builder.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 4e5fc94965856..b09735669f773 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1883,8 +1883,8 @@ func (do *Domain) ServerMemoryLimitHandle() *servermemorylimit.Handle { } const ( - privilegeKey = "/tidb/privilege" - sysVarCacheKey = "/tidb/sysvars" + privilegeKey = "/tidb/privilege" + sysVarCacheKey = "/tidb/sysvars" tiflashComputeStoreKey = "/tiflash/new_tiflash_compute_stores" ) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 6f34a03697aed..93ea4e891b9d8 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -65,8 +65,8 @@ import ( "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/set" - "github.com/tikv/client-go/v2/tikv" "github.com/pingcap/tidb/util/size" + "github.com/tikv/client-go/v2/tikv" ) const ( From c01ec1fd0b9ab55168062c0b4d32d4fe3c0a7b2e Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 13 Oct 2022 16:06:04 +0800 Subject: [PATCH 34/44] fix go.mod Signed-off-by: guo-shaoge --- go.mod | 4 ++-- go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 4f407ee587d63..f43e742b73c8e 100644 --- a/go.mod +++ b/go.mod @@ -67,7 +67,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20220729040631-518f63d66278 github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3 github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20220913025519-586cff113d10 + github.com/pingcap/kvproto v0.0.0-20220929075948-06e08d5ed64c github.com/pingcap/log v1.1.0 github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e @@ -86,7 +86,7 @@ require ( github.com/stretchr/testify v1.8.0 github.com/tdakkota/asciicheck v0.1.1 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.1-0.20220923061703-33efe476e022 + github.com/tikv/client-go/v2 v2.0.1-0.20221013065437-e76cd3e2403a github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 github.com/twmb/murmur3 v1.1.3 diff --git a/go.sum b/go.sum index 88a44ead4a9fd..a47db19c3a9c0 100644 --- a/go.sum +++ b/go.sum @@ -756,8 +756,8 @@ 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-20220510035547-0e2f26c0a46a/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= -github.com/pingcap/kvproto v0.0.0-20220913025519-586cff113d10 h1:/92S0s/TCoCmK2vv6WbkXNeqtLn90sHRJ5Vlx1Sigas= -github.com/pingcap/kvproto v0.0.0-20220913025519-586cff113d10/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= +github.com/pingcap/kvproto v0.0.0-20220929075948-06e08d5ed64c h1:ceg4xjEEXNgPsScTQ5dtidiltLF4h17Y/jUqfyLAy9E= +github.com/pingcap/kvproto v0.0.0-20220929075948-06e08d5ed64c/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= @@ -905,8 +905,8 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= 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/tikv/client-go/v2 v2.0.1-0.20220923061703-33efe476e022 h1:TxDSQAmtGdE34BvOaYF35mRrAXePeZEq8quvuAwrKsI= -github.com/tikv/client-go/v2 v2.0.1-0.20220923061703-33efe476e022/go.mod h1:6pedLz7wiINLHXwCT1+yMZmzuG42+ubtBkkfcwoukIo= +github.com/tikv/client-go/v2 v2.0.1-0.20221013065437-e76cd3e2403a h1:Yuw3tchockhy+PIOVxiW0nu9cSqueXgUDN+6jnDaES8= +github.com/tikv/client-go/v2 v2.0.1-0.20221013065437-e76cd3e2403a/go.mod h1:gdXot2ofS2EOGtrXQ2qyESonQX/gFmgtfBCqCOSWg9E= github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db h1:r1eMh9Rny3hfWuBuxOnbsCRrR4FhthiNxLQ5rAUtaww= github.com/tikv/pd/client v0.0.0-20220725055910-7187a7ab72db/go.mod h1:ew8kS0yIcEaSetuuywkTLIUBR+sz3J5XvAYRae11qwc= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro= From 67e014d1820df86b959d30d8fc6d7fed20fb031e Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Sun, 16 Oct 2022 17:03:04 +0800 Subject: [PATCH 35/44] fix config def value Signed-off-by: guo-shaoge --- config/config.go | 1 + 1 file changed, 1 insertion(+) diff --git a/config/config.go b/config/config.go index 4c59c37c00d06..3bc910ff71cf2 100644 --- a/config/config.go +++ b/config/config.go @@ -962,6 +962,7 @@ var defaultConf = Config{ NewCollationsEnabledOnFirstBootstrap: true, EnableGlobalKill: true, TrxSummary: DefaultTrxSummary(), + DisaggregatedTiFlash: false, } var ( From 2ece8610038cf9360896186596e8dd5dab197d64 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Sun, 20 Nov 2022 21:35:38 +0800 Subject: [PATCH 36/44] disable cop and batch cop in Disaggregated Tiflash Signed-off-by: guo-shaoge --- planner/core/find_best_task.go | 4 +++- planner/core/task.go | 4 ++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index c73ce9f3c086d..b43e9c3297fc8 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/tracing" + "github.com/pingcap/tidb/config" "go.uber.org/zap" ) @@ -1875,7 +1876,8 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid } } } - if prop.TaskTp == property.MppTaskType { + // In disaggregated tiflash mode, only MPP is allowed, Cop and BatchCop is deprecated. + if prop.TaskTp == property.MppTaskType || config.GetGlobalConfig().DisaggregatedTiFlash && ts.StoreType == kv.TiFlash { if ts.KeepOrder { return invalidTask, nil } diff --git a/planner/core/task.go b/planner/core/task.go index 7b35c1bbae19c..a03e360cf6f8f 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1778,6 +1778,10 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { } attachPlan2Task(proj, newMpp) return newMpp + case NoMpp: + t = mpp.convertToRootTask(p.ctx) + attachPlan2Task(p, t) + return t default: return invalidTask } From 8ffe715d19a90c76bcf8b36097372a9e75b44cbf Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 6 Dec 2022 16:31:34 +0800 Subject: [PATCH 37/44] fix fmt Signed-off-by: guo-shaoge --- planner/core/find_best_task.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index e4a6d5ae13d2f..afc5223b9be94 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" @@ -38,7 +39,6 @@ import ( "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/tracing" - "github.com/pingcap/tidb/config" "go.uber.org/zap" ) From bdd16e554e5badf13c601e7028bf7433fdac0db7 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 6 Dec 2022 16:39:44 +0800 Subject: [PATCH 38/44] update bazel Signed-off-by: guo-shaoge --- planner/core/BUILD.bazel | 1 + store/copr/BUILD.bazel | 1 + 2 files changed, 2 insertions(+) diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index dcbb25512f458..0495f1f50cae0 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -106,6 +106,7 @@ go_library( "//sessiontxn/staleread", "//statistics", "//statistics/handle", + "//store/driver/backoff", "//table", "//table/tables", "//table/temptable", diff --git a/store/copr/BUILD.bazel b/store/copr/BUILD.bazel index 674ca8f6c54e1..e3ba1419ed165 100644 --- a/store/copr/BUILD.bazel +++ b/store/copr/BUILD.bazel @@ -41,6 +41,7 @@ go_library( "@com_github_pingcap_kvproto//pkg/mpp", "@com_github_pingcap_log//:log", "@com_github_pingcap_tipb//go-tipb", + "@com_github_stathat_consistent//:consistent", "@com_github_tikv_client_go_v2//config", "@com_github_tikv_client_go_v2//error", "@com_github_tikv_client_go_v2//metrics", From f7e278a8da3b102af507f113d60d87ebd4651ada Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 8 Dec 2022 17:24:03 +0800 Subject: [PATCH 39/44] add case Signed-off-by: guo-shaoge --- executor/tiflashtest/tiflash_test.go | 25 +++++++++++++++++++++++++ planner/core/planbuilder.go | 13 +++++++++++-- session/session.go | 2 +- 3 files changed, 37 insertions(+), 3 deletions(-) diff --git a/executor/tiflashtest/tiflash_test.go b/executor/tiflashtest/tiflash_test.go index ffd949ee28f66..baba133b3d5a7 100644 --- a/executor/tiflashtest/tiflash_test.go +++ b/executor/tiflashtest/tiflash_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/parser/auth" @@ -1278,3 +1279,27 @@ func TestBindingFromHistoryWithTiFlashBindable(t *testing.T) { planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows() tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with tiflash engine") } + +func TestDisaggregatedTiFlash(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = true + }) + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c1 int)") + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + + err = tk.ExecToErr("select * from t;") + require.Contains(t, err.Error(), "Please check tiflash_compute node is available") + + config.UpdateGlobal(func(conf *config.Config) { + conf.DisaggregatedTiFlash = false + }) + tk.MustQuery("select * from t;").Check(testkit.Rows()) +} diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 7a7e507634093..021e46f198586 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1392,6 +1392,7 @@ func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, isolationReadEngines := ctx.GetSessionVars().GetIsolationReadEngines() availableEngine := map[kv.StoreType]struct{}{} var availableEngineStr string + var noTiFlashComputeNode bool for i := len(paths) - 1; i >= 0; i-- { // availableEngineStr is for warning message. if _, ok := availableEngine[paths[i].StoreType]; !ok { @@ -1405,7 +1406,11 @@ func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, // Prune this path if: // 1. path.StoreType doesn't exists in isolationReadEngines or // 2. TiFlash is disaggregated and the number of tiflash_compute node is zero. - if (!exists && paths[i].StoreType != kv.TiDB) || (exists && paths[i].StoreType == kv.TiFlash && config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashComputeNodeAvailable(ctx)) { + if paths[i].StoreType == kv.TiFlash { + noTiFlashComputeNode = (exists && paths[i].StoreType == kv.TiFlash && + config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashComputeNodeAvailable(ctx)) + } + if (!exists && paths[i].StoreType != kv.TiDB) || noTiFlashComputeNode { paths = append(paths[:i], paths[i+1:]...) } } @@ -1414,7 +1419,11 @@ func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, if len(paths) == 0 { helpMsg := "" if engineVals == "tiflash" { - helpMsg = ". Please check tiflash replica or ensure the query is readonly" + if noTiFlashComputeNode { + helpMsg = ". Please check tiflash_compute node is available" + } else { + helpMsg = ". Please check tiflash replica or ensure the query is readonly" + } } err = ErrInternal.GenWithStackByArgs(fmt.Sprintf("No access path for table '%s' is found with '%v' = '%v', valid values can be '%s'%s.", tblName.String(), variable.TiDBIsolationReadEngines, engineVals, availableEngineStr, helpMsg)) diff --git a/session/session.go b/session/session.go index 2afaa08178722..91306c22adf93 100644 --- a/session/session.go +++ b/session/session.go @@ -3018,7 +3018,7 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { return nil, err } - if !config.GetGlobalConfig().DisaggregatedTiFlash { + if config.GetGlobalConfig().DisaggregatedTiFlash { // Invalid client-go tiflash_compute store cache if necessary. err = dom.WatchTiFlashComputeStoreChange() if err != nil { From 8d592b8167dc2e497c167659a10c009851ae38bc Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 8 Dec 2022 17:56:52 +0800 Subject: [PATCH 40/44] update domain.go add recover log Signed-off-by: guo-shaoge --- domain/domain.go | 31 +++++++++++++++++++------------ kv/mpp.go | 17 ++++++----------- session/session.go | 2 +- 3 files changed, 26 insertions(+), 24 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 36612ee487983..0d4324405c07c 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -1417,24 +1417,26 @@ func (do *Domain) LoadSysVarCacheLoop(ctx sessionctx.Context) error { return nil } -// WatchTiFlashComputeStoreChange create a routine to watch if the topology of tiflash_compute node is changed. -// TODO: tiflashComputeStoreKey is not put to etcd yet(finish this when AutoScaler is done) +// WatchTiFlashComputeNodeChange create a routine to watch if the topology of tiflash_compute node is changed. +// TODO: tiflashComputeNodeKey is not put to etcd yet(finish this when AutoScaler is done) // // store cache will only be invalidated every 30 seconds. -func (do *Domain) WatchTiFlashComputeStoreChange() error { +func (do *Domain) WatchTiFlashComputeNodeChange() error { var watchCh clientv3.WatchChan if do.etcdClient != nil { - watchCh = do.etcdClient.Watch(context.Background(), tiflashComputeStoreKey) + watchCh = do.etcdClient.Watch(context.Background(), tiflashComputeNodeKey) } do.wg.Add(1) - duration := 30 * time.Second + duration := 10 * time.Second go func() { defer func() { do.wg.Done() - logutil.BgLogger().Info("WatchTiFlashComputeStoreChange exit") + logutil.BgLogger().Info("WatchTiFlashComputeNodeChange exit") + util.Recover(metrics.LabelDomain, "WatchTiFlashComputeNodeChange", nil, false) }() var count int + var logCount int for { ok := true var watched bool @@ -1446,8 +1448,8 @@ func (do *Domain) WatchTiFlashComputeStoreChange() error { case <-time.After(duration): } if !ok { - logutil.BgLogger().Error("WatchTiFlashComputeStoreChange watch channel closed") - watchCh = do.etcdClient.Watch(context.Background(), tiflashComputeStoreKey) + logutil.BgLogger().Error("WatchTiFlashComputeNodeChange watch channel closed") + watchCh = do.etcdClient.Watch(context.Background(), tiflashComputeNodeKey) count++ if count > 10 { time.Sleep(time.Duration(count) * time.Second) @@ -1457,8 +1459,13 @@ func (do *Domain) WatchTiFlashComputeStoreChange() error { count = 0 switch s := do.store.(type) { case tikv.Storage: + logCount++ s.GetRegionCache().InvalidateTiFlashComputeStores() - logutil.BgLogger().Debug("tiflash_compute store cache invalied, will update next query", zap.Bool("watched", watched)) + if logCount == 60 { + // Print log every 60*duration seconds. + logutil.BgLogger().Debug("tiflash_compute store cache invalied, will update next query", zap.Bool("watched", watched)) + logCount = 0 + } default: logutil.BgLogger().Debug("No need to watch tiflash_compute store cache for non-tikv store") return @@ -2118,9 +2125,9 @@ func (do *Domain) ServerMemoryLimitHandle() *servermemorylimit.Handle { } const ( - privilegeKey = "/tidb/privilege" - sysVarCacheKey = "/tidb/sysvars" - tiflashComputeStoreKey = "/tiflash/new_tiflash_compute_stores" + privilegeKey = "/tidb/privilege" + sysVarCacheKey = "/tidb/sysvars" + tiflashComputeNodeKey = "/tiflash/new_tiflash_compute_nodes" ) // NotifyUpdatePrivilege updates privilege key in etcd, TiDB client that watches diff --git a/kv/mpp.go b/kv/mpp.go index b3fc209d5f35c..b0752f8186deb 100644 --- a/kv/mpp.go +++ b/kv/mpp.go @@ -66,20 +66,15 @@ const ( // MPPDispatchRequest stands for a dispatching task. type MPPDispatchRequest struct { - // data encodes the dag coprocessor request. - Data []byte - // mpp store is the location of tiflash store. - Meta MPPTaskMeta - // root task returns data to tidb directly. - IsRoot bool - // If task is assigned but doesn't receive a connect request during timeout, the task should be destroyed. - Timeout uint64 + Data []byte // data encodes the dag coprocessor request. + Meta MPPTaskMeta // mpp store is the location of tiflash store. + IsRoot bool // root task returns data to tidb directly. + Timeout uint64 // If task is assigned but doesn't receive a connect request during timeout, the task should be destroyed. // SchemaVer is for any schema-ful storage (like tiflash) to validate schema correctness if necessary. SchemaVar int64 StartTs uint64 - // Identify a single task. - ID int64 - State MppTaskStates + ID int64 // identify a single task + State MppTaskStates } // MPPClient accepts and processes mpp requests. diff --git a/session/session.go b/session/session.go index 91306c22adf93..5615d7542e46e 100644 --- a/session/session.go +++ b/session/session.go @@ -3020,7 +3020,7 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { if config.GetGlobalConfig().DisaggregatedTiFlash { // Invalid client-go tiflash_compute store cache if necessary. - err = dom.WatchTiFlashComputeStoreChange() + err = dom.WatchTiFlashComputeNodeChange() if err != nil { return nil, err } From a51ecf68f904723f592721305ceb4f6d9fc423f0 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 8 Dec 2022 17:59:37 +0800 Subject: [PATCH 41/44] update err msg Signed-off-by: guo-shaoge --- planner/core/logical_plan_builder.go | 2 +- store/copr/batch_coprocessor.go | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index d5084e33c187c..6bc8a677bc6de 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -690,7 +690,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { } if config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashComputeNodeAvailable(ds.ctx) { // TiFlash is in disaggregated mode, need to make sure tiflash_compute node is available. - errMsg := "Number of tiflash_compute nodes is zero" + errMsg := "No available tiflash_compute node" warning := ErrInternal.GenWithStack(errMsg) ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) return diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index b6acd4cf99cef..801eebc40de9a 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -577,7 +577,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran return nil, err } if len(stores) == 0 { - return nil, errors.New("Number of tiflash_compute node is zero") + return nil, errors.New("No available tiflash_compute node") } hasher := consistent.New() @@ -604,8 +604,7 @@ func buildBatchCopTasksConsistentHash(bo *backoff.Backoffer, store *kvStore, ran task.ctx.Store = store task.ctx.Addr = addr } - // TODO: Add metrics - logutil.BgLogger().Info("build batchCop tasks using ConsistentHash done.", zap.Int("len(tasks)", len(batchTasks))) + logutil.BgLogger().Info("build batchCop tasks for disaggregated tiflash using ConsistentHash done.", zap.Int("len(tasks)", len(batchTasks))) for _, task := range batchTasks { logutil.BgLogger().Debug("batchTasks detailed info", zap.String("addr", task.storeAddr), zap.Int("RegionInfo number", len(task.regionInfos))) } From d2cb9c00ff12bf246a41279ca0932621fb7f2ebc Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 8 Dec 2022 21:00:38 +0800 Subject: [PATCH 42/44] update case Signed-off-by: guo-shaoge --- store/copr/batch_coprocessor_test.go | 55 ++++++++++++++++++++++++++++ 1 file changed, 55 insertions(+) diff --git a/store/copr/batch_coprocessor_test.go b/store/copr/batch_coprocessor_test.go index aafa19071a392..5616f61c54365 100644 --- a/store/copr/batch_coprocessor_test.go +++ b/store/copr/batch_coprocessor_test.go @@ -22,6 +22,7 @@ import ( "time" "github.com/pingcap/tidb/kv" + "github.com/stathat/consistent" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/tikv" ) @@ -150,3 +151,57 @@ func TestDeepCopyStoreTaskMap(t *testing.T) { require.Equal(t, 2, len(task.regionInfos)) } } + +// Make sure no duplicated ip:addr. +func generateOneAddr() string { + var ip string + for i := 0; i < 4; i++ { + if i != 0 { + ip += "." + } + ip += strconv.Itoa(rand.Intn(255)) + } + return ip + ":" + strconv.Itoa(rand.Intn(65535)) +} + +func generateDifferentAddrs(num int) (res []string) { + addrMap := make(map[string]struct{}) + for len(addrMap) < num { + addr := generateOneAddr() + if _, ok := addrMap[addr]; !ok { + addrMap[addr] = struct{}{} + } + } + for addr := range addrMap { + res = append(res, addr) + } + return +} + +func TestConsistentHash(t *testing.T) { + allAddrs := generateDifferentAddrs(100) + + computeNodes := allAddrs[:30] + storageNodes := allAddrs[30:] + firstRoundMap := make(map[string]string) + for round := 0; round < 100; round++ { + hasher := consistent.New() + rand.Shuffle(len(computeNodes), func(i, j int) { + computeNodes[i], computeNodes[j] = computeNodes[j], computeNodes[i] + }) + for _, computeNode := range computeNodes { + hasher.Add(computeNode) + } + for _, storageNode := range storageNodes { + computeNode, err := hasher.Get(storageNode) + require.NoError(t, err) + if round == 0 { + firstRoundMap[storageNode] = computeNode + } else { + firstRoundAddr, ok := firstRoundMap[storageNode] + require.True(t, ok) + require.Equal(t, firstRoundAddr, computeNode) + } + } + } +} From 77ca0390d87e5e92e144d46c826f7dec5d0b96d8 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 8 Dec 2022 21:14:20 +0800 Subject: [PATCH 43/44] update bazel Signed-off-by: guo-shaoge --- store/copr/BUILD.bazel | 1 + 1 file changed, 1 insertion(+) diff --git a/store/copr/BUILD.bazel b/store/copr/BUILD.bazel index e3ba1419ed165..eb3eb2f016424 100644 --- a/store/copr/BUILD.bazel +++ b/store/copr/BUILD.bazel @@ -76,6 +76,7 @@ go_test( "//testkit/testsetup", "//util/paging", "@com_github_pingcap_kvproto//pkg/coprocessor", + "@com_github_stathat_consistent//:consistent", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//config", "@com_github_tikv_client_go_v2//testutils", From c7726a45b36c596fd9f0b37841fa6d44aa12046b Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 8 Dec 2022 22:47:17 +0800 Subject: [PATCH 44/44] fix prunt path logic Signed-off-by: guo-shaoge --- planner/core/planbuilder.go | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 021e46f198586..d8677ea7dd072 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1392,7 +1392,8 @@ func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, isolationReadEngines := ctx.GetSessionVars().GetIsolationReadEngines() availableEngine := map[kv.StoreType]struct{}{} var availableEngineStr string - var noTiFlashComputeNode bool + var outputComputeNodeErrMsg bool + noTiFlashComputeNode := config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashComputeNodeAvailable(ctx) for i := len(paths) - 1; i >= 0; i-- { // availableEngineStr is for warning message. if _, ok := availableEngine[paths[i].StoreType]; !ok { @@ -1406,11 +1407,11 @@ func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, // Prune this path if: // 1. path.StoreType doesn't exists in isolationReadEngines or // 2. TiFlash is disaggregated and the number of tiflash_compute node is zero. - if paths[i].StoreType == kv.TiFlash { - noTiFlashComputeNode = (exists && paths[i].StoreType == kv.TiFlash && - config.GetGlobalConfig().DisaggregatedTiFlash && !isTiFlashComputeNodeAvailable(ctx)) + shouldPruneTiFlashCompute := noTiFlashComputeNode && exists && paths[i].StoreType == kv.TiFlash + if shouldPruneTiFlashCompute { + outputComputeNodeErrMsg = true } - if (!exists && paths[i].StoreType != kv.TiDB) || noTiFlashComputeNode { + if (!exists && paths[i].StoreType != kv.TiDB) || shouldPruneTiFlashCompute { paths = append(paths[:i], paths[i+1:]...) } } @@ -1419,7 +1420,7 @@ func filterPathByIsolationRead(ctx sessionctx.Context, paths []*util.AccessPath, if len(paths) == 0 { helpMsg := "" if engineVals == "tiflash" { - if noTiFlashComputeNode { + if outputComputeNodeErrMsg { helpMsg = ". Please check tiflash_compute node is available" } else { helpMsg = ". Please check tiflash replica or ensure the query is readonly"