From 5f7236fb55e6ba01bbbafbc86749ac922b351154 Mon Sep 17 00:00:00 2001 From: glorv Date: Mon, 3 Jul 2023 14:53:12 +0800 Subject: [PATCH 01/15] resourcemanager: do not check existence when add resource group (#6717) ref tikv/pd#5851, ref pingcap/tidb#45050 Signed-off-by: glorv Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- errors.toml | 5 ----- pkg/errs/errno.go | 7 +++---- pkg/mcs/resourcemanager/server/manager.go | 8 ++------ .../mcs/resourcemanager/resource_manager_test.go | 12 +++++------- 4 files changed, 10 insertions(+), 22 deletions(-) diff --git a/errors.toml b/errors.toml index ed3cd32d52a..43fc6a582aa 100644 --- a/errors.toml +++ b/errors.toml @@ -606,11 +606,6 @@ error = ''' invalid group settings, please check the group name, priority and the number of resources ''' -["PD:resourcemanager:ErrResourceGroupAlreadyExists"] -error = ''' -the %s resource group already exists -''' - ["PD:schedule:ErrCreateOperator"] error = ''' unable to create operator, %s diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index 0fda57fa7c9..0bd2a57dba5 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -372,8 +372,7 @@ var ( // Resource Manager errors var ( - ErrResourceGroupAlreadyExists = errors.Normalize("the %s resource group already exists", errors.RFCCodeText("PD:resourcemanager:ErrResourceGroupAlreadyExists")) - ErrResourceGroupNotExists = errors.Normalize("the %s resource group does not exist", errors.RFCCodeText("PD:resourcemanager:ErrGroupNotExists")) - ErrDeleteReservedGroup = errors.Normalize("cannot delete reserved group", errors.RFCCodeText("PD:resourcemanager:ErrDeleteReservedGroup")) - ErrInvalidGroup = errors.Normalize("invalid group settings, please check the group name, priority and the number of resources", errors.RFCCodeText("PD:resourcemanager:ErrInvalidGroup")) + ErrResourceGroupNotExists = errors.Normalize("the %s resource group does not exist", errors.RFCCodeText("PD:resourcemanager:ErrGroupNotExists")) + ErrDeleteReservedGroup = errors.Normalize("cannot delete reserved group", errors.RFCCodeText("PD:resourcemanager:ErrDeleteReservedGroup")) + ErrInvalidGroup = errors.Normalize("invalid group settings, please check the group name, priority and the number of resources", errors.RFCCodeText("PD:resourcemanager:ErrInvalidGroup")) ) diff --git a/pkg/mcs/resourcemanager/server/manager.go b/pkg/mcs/resourcemanager/server/manager.go index a98d274f506..b054a37e0ac 100644 --- a/pkg/mcs/resourcemanager/server/manager.go +++ b/pkg/mcs/resourcemanager/server/manager.go @@ -154,6 +154,8 @@ func (m *Manager) Init(ctx context.Context) { } // AddResourceGroup puts a resource group. +// NOTE: AddResourceGroup should also be idempotent because tidb depends +// on this retry mechanism. func (m *Manager) AddResourceGroup(grouppb *rmpb.ResourceGroup) error { // Check the name. if len(grouppb.Name) == 0 || len(grouppb.Name) > 32 { @@ -163,12 +165,6 @@ func (m *Manager) AddResourceGroup(grouppb *rmpb.ResourceGroup) error { if grouppb.GetPriority() > 16 { return errs.ErrInvalidGroup } - m.RLock() - _, ok := m.groups[grouppb.Name] - m.RUnlock() - if ok { - return errs.ErrResourceGroupAlreadyExists.FastGenByArgs(grouppb.Name) - } group := FromProtoResourceGroup(grouppb) m.Lock() defer m.Unlock() diff --git a/tests/integrations/mcs/resourcemanager/resource_manager_test.go b/tests/integrations/mcs/resourcemanager/resource_manager_test.go index 467eba6c518..5cbed81d57e 100644 --- a/tests/integrations/mcs/resourcemanager/resource_manager_test.go +++ b/tests/integrations/mcs/resourcemanager/resource_manager_test.go @@ -711,7 +711,7 @@ func (suite *resourceManagerClientTestSuite) TestBasicResourceGroupCURD() { testCasesSet1 := []struct { name string mode rmpb.GroupMode - addSuccess bool + isNewGroup bool modifySuccess bool expectMarshal string modifySettings func(*rmpb.ResourceGroup) @@ -789,8 +789,8 @@ func (suite *resourceManagerClientTestSuite) TestBasicResourceGroupCURD() { } // Create Resource Group resp, err := cli.AddResourceGroup(suite.ctx, group) - checkErr(err, tcase.addSuccess) - if tcase.addSuccess { + checkErr(err, true) + if tcase.isNewGroup { finalNum++ re.Contains(resp, "Success!") } @@ -860,11 +860,9 @@ func (suite *resourceManagerClientTestSuite) TestBasicResourceGroupCURD() { resp, err := http.Post(getAddr(i)+"/resource-manager/api/v1/config/group", "application/json", strings.NewReader(string(createJSON))) re.NoError(err) defer resp.Body.Close() - if tcase.addSuccess { - re.Equal(http.StatusOK, resp.StatusCode) + re.Equal(http.StatusOK, resp.StatusCode) + if tcase.isNewGroup { finalNum++ - } else { - re.Equal(http.StatusInternalServerError, resp.StatusCode) } // Modify Resource Group From 05f71e02d5e85b267aa02b3c8c31eb7e08e9eb6a Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Mon, 3 Jul 2023 15:08:12 +0800 Subject: [PATCH 02/15] mcs: use patch method in keyspace group (#6713) ref tikv/pd#6233 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- server/apiv2/handlers/tso_keyspace_group.go | 29 ++++++++++++---- .../mcs/keyspace/tso_keyspace_group_test.go | 2 +- tools/pd-ctl/pdctl/command/global.go | 34 +++++++++++++++---- .../pdctl/command/keyspace_group_command.go | 21 +++++++----- 4 files changed, 64 insertions(+), 22 deletions(-) diff --git a/server/apiv2/handlers/tso_keyspace_group.go b/server/apiv2/handlers/tso_keyspace_group.go index 5de8fd6a9cc..7030c332406 100644 --- a/server/apiv2/handlers/tso_keyspace_group.go +++ b/server/apiv2/handlers/tso_keyspace_group.go @@ -40,9 +40,9 @@ func RegisterTSOKeyspaceGroup(r *gin.RouterGroup) { router.GET("", GetKeyspaceGroups) router.GET("/:id", GetKeyspaceGroupByID) router.DELETE("/:id", DeleteKeyspaceGroupByID) + router.PATCH("/:id", SetNodesForKeyspaceGroup) // only to support set nodes + router.PATCH("/:id/*node", SetPriorityForKeyspaceGroup) // only to support set priority router.POST("/:id/alloc", AllocNodesForKeyspaceGroup) - router.POST("/:id/nodes", SetNodesForKeyspaceGroup) - router.POST("/:id/priority", SetPriorityForKeyspaceGroup) router.POST("/:id/split", SplitKeyspaceGroupByID) router.DELETE("/:id/split", FinishSplitKeyspaceByID) router.POST("/:id/merge", MergeKeyspaceGroups) @@ -436,8 +436,7 @@ func SetNodesForKeyspaceGroup(c *gin.Context) { // SetPriorityForKeyspaceGroupParams defines the params for setting priority of tso node for the keyspace group. type SetPriorityForKeyspaceGroupParams struct { - Node string `json:"node"` - Priority int `json:"priority"` + Priority int `json:"priority"` } // SetPriorityForKeyspaceGroup sets priority of tso node for the keyspace group. @@ -447,6 +446,11 @@ func SetPriorityForKeyspaceGroup(c *gin.Context) { c.AbortWithStatusJSON(http.StatusBadRequest, "invalid keyspace group id") return } + node, err := parseNodeAddress(c) + if err != nil { + c.AbortWithStatusJSON(http.StatusBadRequest, "invalid node address") + return + } svr := c.MustGet(middlewares.ServerContextKey).(*server.Server) manager := svr.GetKeyspaceGroupManager() if manager == nil { @@ -468,12 +472,12 @@ func SetPriorityForKeyspaceGroup(c *gin.Context) { // check if node exists members := kg.Members if slice.NoneOf(members, func(i int) bool { - return members[i].Address == setParams.Node + return members[i].Address == node }) { c.AbortWithStatusJSON(http.StatusBadRequest, "tso node does not exist in the keyspace group") } // set priority - err = manager.SetPriorityForKeyspaceGroup(id, setParams.Node, setParams.Priority) + err = manager.SetPriorityForKeyspaceGroup(id, node, setParams.Priority) if err != nil { c.AbortWithStatusJSON(http.StatusInternalServerError, err.Error()) return @@ -492,6 +496,19 @@ func validateKeyspaceGroupID(c *gin.Context) (uint32, error) { return uint32(id), nil } +func parseNodeAddress(c *gin.Context) (string, error) { + node := c.Param("node") + if node == "" { + return "", errors.New("invalid node address") + } + // In pd-ctl, we use url.PathEscape to escape the node address and replace the % to \%. + // But in the gin framework, it will unescape the node address automatically. + // So we need to replace the \/ to /. + node = strings.ReplaceAll(node, "\\/", "/") + node = strings.TrimPrefix(node, "/") + return node, nil +} + func isValid(id uint32) bool { return id >= utils.DefaultKeyspaceGroupID && id <= utils.MaxKeyspaceGroupCountInUse } diff --git a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go index 41bcba0e90b..dc33016eafb 100644 --- a/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go +++ b/tests/integrations/mcs/keyspace/tso_keyspace_group_test.go @@ -354,7 +354,7 @@ func (suite *keyspaceGroupTestSuite) tryGetKeyspaceGroup(id uint32) (*endpoint.K func (suite *keyspaceGroupTestSuite) trySetNodesForKeyspaceGroup(id int, request *handlers.SetNodesForKeyspaceGroupParams) (*endpoint.KeyspaceGroup, int) { data, err := json.Marshal(request) suite.NoError(err) - httpReq, err := http.NewRequest(http.MethodPost, suite.server.GetAddr()+keyspaceGroupsPrefix+fmt.Sprintf("/%d/nodes", id), bytes.NewBuffer(data)) + httpReq, err := http.NewRequest(http.MethodPatch, suite.server.GetAddr()+keyspaceGroupsPrefix+fmt.Sprintf("/%d", id), bytes.NewBuffer(data)) suite.NoError(err) resp, err := suite.dialClient.Do(httpReq) suite.NoError(err) diff --git a/tools/pd-ctl/pdctl/command/global.go b/tools/pd-ctl/pdctl/command/global.go index 8d888b60b1f..85fe63ac8be 100644 --- a/tools/pd-ctl/pdctl/command/global.go +++ b/tools/pd-ctl/pdctl/command/global.go @@ -165,7 +165,7 @@ func getEndpoints(cmd *cobra.Command) []string { return strings.Split(addrs, ",") } -func postJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { +func requestJSON(cmd *cobra.Command, method, prefix string, input map[string]interface{}) { data, err := json.Marshal(input) if err != nil { cmd.Println(err) @@ -175,19 +175,31 @@ func postJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { endpoints := getEndpoints(cmd) err = tryURLs(cmd, endpoints, func(endpoint string) error { var msg []byte - var r *http.Response + var req *http.Request + var resp *http.Response url := endpoint + "/" + prefix - r, err = dialClient.Post(url, "application/json", bytes.NewBuffer(data)) + switch method { + case http.MethodPost, http.MethodPut, http.MethodPatch, http.MethodDelete, http.MethodGet: + req, err = http.NewRequest(method, url, bytes.NewBuffer(data)) + if err != nil { + return err + } + req.Header.Set("Content-Type", "application/json") + resp, err = dialClient.Do(req) + default: + err := errors.Errorf("method %s not supported", method) + return err + } if err != nil { return err } - defer r.Body.Close() - if r.StatusCode != http.StatusOK { - msg, err = io.ReadAll(r.Body) + defer resp.Body.Close() + if resp.StatusCode != http.StatusOK { + msg, err = io.ReadAll(resp.Body) if err != nil { return err } - return errors.Errorf("[%d] %s", r.StatusCode, msg) + return errors.Errorf("[%d] %s", resp.StatusCode, msg) } return nil }) @@ -198,6 +210,14 @@ func postJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { cmd.Println("Success!") } +func postJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { + requestJSON(cmd, http.MethodPost, prefix, input) +} + +func patchJSON(cmd *cobra.Command, prefix string, input map[string]interface{}) { + requestJSON(cmd, http.MethodPatch, prefix, input) +} + // do send a request to server. Default is Get. func do(endpoint, prefix, method string, resp *string, customHeader http.Header, b *bodyOption) error { var err error diff --git a/tools/pd-ctl/pdctl/command/keyspace_group_command.go b/tools/pd-ctl/pdctl/command/keyspace_group_command.go index a4be612a301..b5acf0fa7e8 100644 --- a/tools/pd-ctl/pdctl/command/keyspace_group_command.go +++ b/tools/pd-ctl/pdctl/command/keyspace_group_command.go @@ -288,17 +288,17 @@ func setNodesKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { cmd.Printf("Failed to parse the keyspace group ID: %s\n", err) return } - addresses := make([]string, 0, len(args)-1) + nodes := make([]string, 0, len(args)-1) for _, arg := range args[1:] { u, err := url.ParseRequestURI(arg) if u == nil || err != nil { cmd.Printf("Failed to parse the tso node address: %s\n", err) return } - addresses = append(addresses, arg) + nodes = append(nodes, arg) } - postJSON(cmd, fmt.Sprintf("%s/%s/nodes", keyspaceGroupsPrefix, args[0]), map[string]interface{}{ - "Nodes": addresses, + patchJSON(cmd, fmt.Sprintf("%s/%s", keyspaceGroupsPrefix, args[0]), map[string]interface{}{ + "Nodes": nodes, }) } @@ -313,21 +313,26 @@ func setPriorityKeyspaceGroupCommandFunc(cmd *cobra.Command, args []string) { return } - address := args[1] - u, err := url.ParseRequestURI(address) + node := args[1] + u, err := url.ParseRequestURI(node) if u == nil || err != nil { cmd.Printf("Failed to parse the tso node address: %s\n", err) return } + // Escape the node address to avoid the error of parsing the url + // But the url.PathEscape will escape the '/' to '%2F', which % will cause the error of parsing the url + // So we need to replace the % to \% + node = url.PathEscape(node) + node = strings.ReplaceAll(node, "%", "\\%") + priority, err := strconv.ParseInt(args[2], 10, 32) if err != nil { cmd.Printf("Failed to parse the priority: %s\n", err) return } - postJSON(cmd, fmt.Sprintf("%s/%s/priority", keyspaceGroupsPrefix, args[0]), map[string]interface{}{ - "Node": address, + patchJSON(cmd, fmt.Sprintf("%s/%s/%s", keyspaceGroupsPrefix, args[0], node), map[string]interface{}{ "Priority": priority, }) } From ff67696aa11d8681d6d513fa1e9c35d6fe46049c Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Mon, 3 Jul 2023 16:20:13 +0800 Subject: [PATCH 03/15] *: fix memory leak introduced by timer.After (#6720) close tikv/pd#6719 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/client.go | 4 +- client/pd_service_discovery.go | 4 +- client/resource_manager_client.go | 4 +- client/timerpool/pool.go | 43 +++++++++++++++ client/timerpool/pool_test.go | 70 ++++++++++++++++++++++++ client/tso_dispatcher.go | 58 +++++++++++++++----- client/tso_service_discovery.go | 8 ++- client/tso_stream.go | 4 +- pkg/election/lease.go | 14 ++++- pkg/keyspace/keyspace.go | 2 + pkg/mcs/discovery/register.go | 3 + pkg/mcs/resourcemanager/server/server.go | 4 +- pkg/mcs/tso/server/server.go | 4 +- pkg/mcs/utils/util.go | 4 +- pkg/replication/replication_mode.go | 8 ++- pkg/schedule/coordinator.go | 19 ++++--- pkg/tso/keyspace_group_manager.go | 4 +- server/api/pprof.go | 4 +- server/grpc_service.go | 8 ++- server/server.go | 14 ++++- tools/pd-simulator/simulator/client.go | 23 ++++---- 21 files changed, 253 insertions(+), 53 deletions(-) create mode 100644 client/timerpool/pool.go create mode 100644 client/timerpool/pool_test.go diff --git a/client/client.go b/client/client.go index 0597f7344ee..5fbc5f9eb23 100644 --- a/client/client.go +++ b/client/client.go @@ -526,6 +526,8 @@ func newClientWithKeyspaceName( func (c *client) initRetry(f func(s string) error, str string) error { var err error + ticker := time.NewTicker(time.Second) + defer ticker.Stop() for i := 0; i < c.option.maxRetryTimes; i++ { if err = f(str); err == nil { return nil @@ -533,7 +535,7 @@ func (c *client) initRetry(f func(s string) error, str string) error { select { case <-c.ctx.Done(): return err - case <-time.After(time.Second): + case <-ticker.C: } } return errors.WithStack(err) diff --git a/client/pd_service_discovery.go b/client/pd_service_discovery.go index 9966b8c75cb..bd7ed31209a 100644 --- a/client/pd_service_discovery.go +++ b/client/pd_service_discovery.go @@ -206,6 +206,8 @@ func (c *pdServiceDiscovery) Init() error { func (c *pdServiceDiscovery) initRetry(f func() error) error { var err error + ticker := time.NewTicker(time.Second) + defer ticker.Stop() for i := 0; i < c.option.maxRetryTimes; i++ { if err = f(); err == nil { return nil @@ -213,7 +215,7 @@ func (c *pdServiceDiscovery) initRetry(f func() error) error { select { case <-c.ctx.Done(): return err - case <-time.After(time.Second): + case <-ticker.C: } } return errors.WithStack(err) diff --git a/client/resource_manager_client.go b/client/resource_manager_client.go index 160ccbd0fff..4b9896dfefa 100644 --- a/client/resource_manager_client.go +++ b/client/resource_manager_client.go @@ -389,6 +389,8 @@ func (c *client) tryResourceManagerConnect(ctx context.Context, connection *reso err error stream rmpb.ResourceManager_AcquireTokenBucketsClient ) + ticker := time.NewTicker(retryInterval) + defer ticker.Stop() for i := 0; i < maxRetryTimes; i++ { cc, err := c.resourceManagerClient() if err != nil { @@ -406,7 +408,7 @@ func (c *client) tryResourceManagerConnect(ctx context.Context, connection *reso select { case <-ctx.Done(): return err - case <-time.After(retryInterval): + case <-ticker.C: } } return err diff --git a/client/timerpool/pool.go b/client/timerpool/pool.go new file mode 100644 index 00000000000..28ffacfc629 --- /dev/null +++ b/client/timerpool/pool.go @@ -0,0 +1,43 @@ +// Copyright 2020 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Note: This file is copied from https://go-review.googlesource.com/c/go/+/276133 + +package timerpool + +import ( + "sync" + "time" +) + +// GlobalTimerPool is a global pool for reusing *time.Timer. +var GlobalTimerPool TimerPool + +// TimerPool is a wrapper of sync.Pool which caches *time.Timer for reuse. +type TimerPool struct { + pool sync.Pool +} + +// Get returns a timer with a given duration. +func (tp *TimerPool) Get(d time.Duration) *time.Timer { + if v := tp.pool.Get(); v != nil { + timer := v.(*time.Timer) + timer.Reset(d) + return timer + } + return time.NewTimer(d) +} + +// Put tries to call timer.Stop() before putting it back into pool, +// if the timer.Stop() returns false (it has either already expired or been stopped), +// have a shot at draining the channel with residual time if there is one. +func (tp *TimerPool) Put(timer *time.Timer) { + if !timer.Stop() { + select { + case <-timer.C: + default: + } + } + tp.pool.Put(timer) +} diff --git a/client/timerpool/pool_test.go b/client/timerpool/pool_test.go new file mode 100644 index 00000000000..d6dffc723a9 --- /dev/null +++ b/client/timerpool/pool_test.go @@ -0,0 +1,70 @@ +// Copyright 2020 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Note: This file is copied from https://go-review.googlesource.com/c/go/+/276133 + +package timerpool + +import ( + "testing" + "time" +) + +func TestTimerPool(t *testing.T) { + var tp TimerPool + + for i := 0; i < 100; i++ { + timer := tp.Get(20 * time.Millisecond) + + select { + case <-timer.C: + t.Errorf("timer expired too early") + continue + default: + } + + select { + case <-time.After(100 * time.Millisecond): + t.Errorf("timer didn't expire on time") + case <-timer.C: + } + + tp.Put(timer) + } +} + +const timeout = 10 * time.Millisecond + +func BenchmarkTimerUtilization(b *testing.B) { + b.Run("TimerWithPool", func(b *testing.B) { + for i := 0; i < b.N; i++ { + t := GlobalTimerPool.Get(timeout) + GlobalTimerPool.Put(t) + } + }) + b.Run("TimerWithoutPool", func(b *testing.B) { + for i := 0; i < b.N; i++ { + t := time.NewTimer(timeout) + t.Stop() + } + }) +} + +func BenchmarkTimerPoolParallel(b *testing.B) { + b.RunParallel(func(pb *testing.PB) { + for pb.Next() { + t := GlobalTimerPool.Get(timeout) + GlobalTimerPool.Put(t) + } + }) +} + +func BenchmarkTimerNativeParallel(b *testing.B) { + b.RunParallel(func(pb *testing.PB) { + for pb.Next() { + t := time.NewTimer(timeout) + t.Stop() + } + }) +} diff --git a/client/tso_dispatcher.go b/client/tso_dispatcher.go index 37bea8db9e5..9fd5c586bf9 100644 --- a/client/tso_dispatcher.go +++ b/client/tso_dispatcher.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/log" "github.com/tikv/pd/client/errs" "github.com/tikv/pd/client/grpcutil" + "github.com/tikv/pd/client/timerpool" "github.com/tikv/pd/client/tsoutil" "go.uber.org/zap" "google.golang.org/grpc" @@ -139,11 +140,24 @@ func (c *tsoClient) updateTSODispatcher() { } type deadline struct { - timer <-chan time.Time + timer *time.Timer done chan struct{} cancel context.CancelFunc } +func newTSDeadline( + timeout time.Duration, + done chan struct{}, + cancel context.CancelFunc, +) *deadline { + timer := timerpool.GlobalTimerPool.Get(timeout) + return &deadline{ + timer: timer, + done: done, + cancel: cancel, + } +} + func (c *tsoClient) tsCancelLoop() { defer c.wg.Done() @@ -172,19 +186,21 @@ func (c *tsoClient) tsCancelLoop() { func (c *tsoClient) watchTSDeadline(ctx context.Context, dcLocation string) { if _, exist := c.tsDeadline.Load(dcLocation); !exist { - tsDeadlineCh := make(chan deadline, 1) + tsDeadlineCh := make(chan *deadline, 1) c.tsDeadline.Store(dcLocation, tsDeadlineCh) - go func(dc string, tsDeadlineCh <-chan deadline) { + go func(dc string, tsDeadlineCh <-chan *deadline) { for { select { case d := <-tsDeadlineCh: select { - case <-d.timer: + case <-d.timer.C: log.Error("[tso] tso request is canceled due to timeout", zap.String("dc-location", dc), errs.ZapError(errs.ErrClientGetTSOTimeout)) d.cancel() + timerpool.GlobalTimerPool.Put(d.timer) case <-d.done: - continue + timerpool.GlobalTimerPool.Put(d.timer) case <-ctx.Done(): + timerpool.GlobalTimerPool.Put(d.timer) return } case <-ctx.Done(): @@ -234,6 +250,8 @@ func (c *tsoClient) checkAllocator( }() cc, u := c.GetTSOAllocatorClientConnByDCLocation(dc) healthCli := healthpb.NewHealthClient(cc) + ticker := time.NewTicker(time.Second) + defer ticker.Stop() for { // the pd/allocator leader change, we need to re-establish the stream if u != url { @@ -259,7 +277,7 @@ func (c *tsoClient) checkAllocator( select { case <-dispatcherCtx.Done(): return - case <-time.After(time.Second): + case <-ticker.C: // To ensure we can get the latest allocator leader // and once the leader is changed, we can exit this function. _, u = c.GetTSOAllocatorClientConnByDCLocation(dc) @@ -366,6 +384,7 @@ func (c *tsoClient) handleDispatcher( // Loop through each batch of TSO requests and send them for processing. streamLoopTimer := time.NewTimer(c.option.timeout) + defer streamLoopTimer.Stop() tsoBatchLoop: for { select { @@ -389,6 +408,15 @@ tsoBatchLoop: if maxBatchWaitInterval >= 0 { tbc.adjustBestBatchSize() } + // Stop the timer if it's not stopped. + if !streamLoopTimer.Stop() { + select { + case <-streamLoopTimer.C: // try to drain from the channel + default: + } + } + // We need be careful here, see more details in the comments of Timer.Reset. + // https://pkg.go.dev/time@master#Timer.Reset streamLoopTimer.Reset(c.option.timeout) // Choose a stream to send the TSO gRPC request. streamChoosingLoop: @@ -403,16 +431,20 @@ tsoBatchLoop: if c.updateTSOConnectionCtxs(dispatcherCtx, dc, &connectionCtxs) { continue streamChoosingLoop } + timer := time.NewTimer(retryInterval) select { case <-dispatcherCtx.Done(): + timer.Stop() return case <-streamLoopTimer.C: err = errs.ErrClientCreateTSOStream.FastGenByArgs(errs.RetryTimeoutErr) log.Error("[tso] create tso stream error", zap.String("dc-location", dc), errs.ZapError(err)) c.svcDiscovery.ScheduleCheckMemberChanged() c.finishRequest(tbc.getCollectedRequests(), 0, 0, 0, errors.WithStack(err)) + timer.Stop() continue tsoBatchLoop - case <-time.After(retryInterval): + case <-timer.C: + timer.Stop() continue streamChoosingLoop } } @@ -429,11 +461,7 @@ tsoBatchLoop: } } done := make(chan struct{}) - dl := deadline{ - timer: time.After(c.option.timeout), - done: done, - cancel: cancel, - } + dl := newTSDeadline(c.option.timeout, done, cancel) tsDeadlineCh, ok := c.tsDeadline.Load(dc) for !ok || tsDeadlineCh == nil { c.scheduleCheckTSDeadline() @@ -443,7 +471,7 @@ tsoBatchLoop: select { case <-dispatcherCtx.Done(): return - case tsDeadlineCh.(chan deadline) <- dl: + case tsDeadlineCh.(chan *deadline) <- dl: } opts = extractSpanReference(tbc, opts[:0]) err = c.processRequests(stream, dc, tbc, opts) @@ -558,6 +586,8 @@ func (c *tsoClient) tryConnectToTSO( } // retry several times before falling back to the follower when the network problem happens + ticker := time.NewTicker(retryInterval) + defer ticker.Stop() for i := 0; i < maxRetryTimes; i++ { c.svcDiscovery.ScheduleCheckMemberChanged() cc, url = c.GetTSOAllocatorClientConnByDCLocation(dc) @@ -587,7 +617,7 @@ func (c *tsoClient) tryConnectToTSO( select { case <-dispatcherCtx.Done(): return err - case <-time.After(retryInterval): + case <-ticker.C: } } diff --git a/client/tso_service_discovery.go b/client/tso_service_discovery.go index cee079634e9..6763e80976a 100644 --- a/client/tso_service_discovery.go +++ b/client/tso_service_discovery.go @@ -209,6 +209,8 @@ func (c *tsoServiceDiscovery) retry( maxRetryTimes int, retryInterval time.Duration, f func() error, ) error { var err error + ticker := time.NewTicker(retryInterval) + defer ticker.Stop() for i := 0; i < maxRetryTimes; i++ { if err = f(); err == nil { return nil @@ -216,7 +218,7 @@ func (c *tsoServiceDiscovery) retry( select { case <-c.ctx.Done(): return err - case <-time.After(retryInterval): + case <-ticker.C: } } return errors.WithStack(err) @@ -245,11 +247,13 @@ func (c *tsoServiceDiscovery) startCheckMemberLoop() { ctx, cancel := context.WithCancel(c.ctx) defer cancel() + ticker := time.NewTicker(memberUpdateInterval) + defer ticker.Stop() for { select { case <-c.checkMembershipCh: - case <-time.After(memberUpdateInterval): + case <-ticker.C: case <-ctx.Done(): log.Info("[tso] exit check member loop") return diff --git a/client/tso_stream.go b/client/tso_stream.go index 892512d8559..e3203818938 100644 --- a/client/tso_stream.go +++ b/client/tso_stream.go @@ -87,10 +87,12 @@ func (b *tsoTSOStreamBuilder) build( } func checkStreamTimeout(ctx context.Context, cancel context.CancelFunc, done chan struct{}, timeout time.Duration) { + timer := time.NewTimer(timeout) + defer timer.Stop() select { case <-done: return - case <-time.After(timeout): + case <-timer.C: cancel() case <-ctx.Done(): } diff --git a/pkg/election/lease.go b/pkg/election/lease.go index a0db045256f..1e3e66ddcce 100644 --- a/pkg/election/lease.go +++ b/pkg/election/lease.go @@ -109,6 +109,8 @@ func (l *lease) KeepAlive(ctx context.Context) { timeCh := l.keepAliveWorker(ctx, l.leaseTimeout/3) var maxExpire time.Time + timer := time.NewTimer(l.leaseTimeout) + defer timer.Stop() for { select { case t := <-timeCh: @@ -122,7 +124,17 @@ func (l *lease) KeepAlive(ctx context.Context) { l.expireTime.Store(t) } } - case <-time.After(l.leaseTimeout): + // Stop the timer if it's not stopped. + if !timer.Stop() { + select { + case <-timer.C: // try to drain from the channel + default: + } + } + // We need be careful here, see more details in the comments of Timer.Reset. + // https://pkg.go.dev/time@master#Timer.Reset + timer.Reset(l.leaseTimeout) + case <-timer.C: log.Info("lease timeout", zap.Time("expire", l.expireTime.Load().(time.Time)), zap.String("purpose", l.Purpose)) return case <-ctx.Done(): diff --git a/pkg/keyspace/keyspace.go b/pkg/keyspace/keyspace.go index df7eb653828..64dd1ba8622 100644 --- a/pkg/keyspace/keyspace.go +++ b/pkg/keyspace/keyspace.go @@ -334,6 +334,8 @@ func (manager *Manager) splitKeyspaceRegion(id uint32, waitRegionSplit bool) (er if region == nil || !bytes.Equal(region.GetStartKey(), txnRightBound) { continue } + // Note: we reset the ticker here to support updating configuration dynamically. + ticker.Reset(manager.config.GetCheckRegionSplitInterval()) case <-timer.C: log.Warn("[keyspace] wait region split timeout", zap.Uint32("keyspace-id", id), diff --git a/pkg/mcs/discovery/register.go b/pkg/mcs/discovery/register.go index 3e08d9b49cf..1e7800295de 100644 --- a/pkg/mcs/discovery/register.go +++ b/pkg/mcs/discovery/register.go @@ -86,6 +86,7 @@ func (sr *ServiceRegister) Register() error { select { case <-sr.ctx.Done(): log.Info("exit register process", zap.String("key", sr.key)) + t.Stop() return default: } @@ -94,11 +95,13 @@ func (sr *ServiceRegister) Register() error { resp, err := sr.cli.Grant(sr.ctx, sr.ttl) if err != nil { log.Error("grant lease failed", zap.String("key", sr.key), zap.Error(err)) + t.Stop() continue } if _, err := sr.cli.Put(sr.ctx, sr.key, sr.value, clientv3.WithLease(resp.ID)); err != nil { log.Error("put the key failed", zap.String("key", sr.key), zap.Error(err)) + t.Stop() continue } } diff --git a/pkg/mcs/resourcemanager/server/server.go b/pkg/mcs/resourcemanager/server/server.go index 6705c4b1da9..a0ccb52ed38 100644 --- a/pkg/mcs/resourcemanager/server/server.go +++ b/pkg/mcs/resourcemanager/server/server.go @@ -286,9 +286,11 @@ func (s *Server) startGRPCServer(l net.Listener) { gs.GracefulStop() close(done) }() + timer := time.NewTimer(utils.DefaultGRPCGracefulStopTimeout) + defer timer.Stop() select { case <-done: - case <-time.After(utils.DefaultGRPCGracefulStopTimeout): + case <-timer.C: log.Info("stopping grpc gracefully is taking longer than expected and force stopping now", zap.Duration("default", utils.DefaultGRPCGracefulStopTimeout)) gs.Stop() } diff --git a/pkg/mcs/tso/server/server.go b/pkg/mcs/tso/server/server.go index a0904f4dc7b..67e9c33f65c 100644 --- a/pkg/mcs/tso/server/server.go +++ b/pkg/mcs/tso/server/server.go @@ -595,6 +595,8 @@ func (s *Server) waitAPIServiceReady() error { ready bool err error ) + ticker := time.NewTicker(retryIntervalWaitAPIService) + defer ticker.Stop() for i := 0; i < maxRetryTimesWaitAPIService; i++ { ready, err = s.isAPIServiceReady() if err == nil && ready { @@ -604,7 +606,7 @@ func (s *Server) waitAPIServiceReady() error { select { case <-s.ctx.Done(): return errors.New("context canceled while waiting api server ready") - case <-time.After(retryIntervalWaitAPIService): + case <-ticker.C: } } if err != nil { diff --git a/pkg/mcs/utils/util.go b/pkg/mcs/utils/util.go index 1d66fc08c1c..a89c6c72b76 100644 --- a/pkg/mcs/utils/util.go +++ b/pkg/mcs/utils/util.go @@ -37,6 +37,8 @@ const ( // InitClusterID initializes the cluster ID. func InitClusterID(ctx context.Context, client *clientv3.Client) (id uint64, err error) { + ticker := time.NewTicker(retryInterval) + defer ticker.Stop() for i := 0; i < maxRetryTimes; i++ { if clusterID, err := etcdutil.GetClusterID(client, clusterIDPath); err == nil && clusterID != 0 { return clusterID, nil @@ -44,7 +46,7 @@ func InitClusterID(ctx context.Context, client *clientv3.Client) (id uint64, err select { case <-ctx.Done(): return 0, err - case <-time.After(retryInterval): + case <-ticker.C: } } return 0, errors.Errorf("failed to init cluster ID after retrying %d times", maxRetryTimes) diff --git a/pkg/replication/replication_mode.go b/pkg/replication/replication_mode.go index 03ed336fd9f..703dad03d26 100644 --- a/pkg/replication/replication_mode.go +++ b/pkg/replication/replication_mode.go @@ -375,14 +375,18 @@ const ( // Run starts the background job. func (m *ModeManager) Run(ctx context.Context) { // Wait for a while when just start, in case tikv do not connect in time. + timer := time.NewTimer(idleTimeout) + defer timer.Stop() select { - case <-time.After(idleTimeout): + case <-timer.C: case <-ctx.Done(): return } + ticker := time.NewTicker(tickInterval) + defer ticker.Stop() for { select { - case <-time.After(tickInterval): + case <-ticker.C: case <-ctx.Done(): return } diff --git a/pkg/schedule/coordinator.go b/pkg/schedule/coordinator.go index 419b6a7adae..a14a0ff556a 100644 --- a/pkg/schedule/coordinator.go +++ b/pkg/schedule/coordinator.go @@ -128,8 +128,8 @@ func (c *Coordinator) PatrolRegions() { defer logutil.LogPanic() defer c.wg.Done() - timer := time.NewTimer(c.cluster.GetOpts().GetPatrolRegionInterval()) - defer timer.Stop() + ticker := time.NewTicker(c.cluster.GetOpts().GetPatrolRegionInterval()) + defer ticker.Stop() log.Info("Coordinator starts patrol regions") start := time.Now() @@ -139,8 +139,9 @@ func (c *Coordinator) PatrolRegions() { ) for { select { - case <-timer.C: - timer.Reset(c.cluster.GetOpts().GetPatrolRegionInterval()) + case <-ticker.C: + // Note: we reset the ticker here to support updating configuration dynamically. + ticker.Reset(c.cluster.GetOpts().GetPatrolRegionInterval()) case <-c.ctx.Done(): log.Info("patrol regions has been stopped") return @@ -848,12 +849,11 @@ func (c *Coordinator) runScheduler(s *scheduleController) { defer c.wg.Done() defer s.Scheduler.Cleanup(c.cluster) - timer := time.NewTimer(s.GetInterval()) - defer timer.Stop() + ticker := time.NewTicker(s.GetInterval()) + defer ticker.Stop() for { select { - case <-timer.C: - timer.Reset(s.GetInterval()) + case <-ticker.C: diagnosable := s.diagnosticRecorder.isAllowed() if !s.AllowSchedule(diagnosable) { continue @@ -862,7 +862,8 @@ func (c *Coordinator) runScheduler(s *scheduleController) { added := c.opController.AddWaitingOperator(op...) log.Debug("add operator", zap.Int("added", added), zap.Int("total", len(op)), zap.String("scheduler", s.Scheduler.GetName())) } - + // Note: we reset the ticker here to support updating configuration dynamically. + ticker.Reset(s.GetInterval()) case <-s.Ctx().Done(): log.Info("scheduler has been stopped", zap.String("scheduler-name", s.Scheduler.GetName()), diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 0291bc5863d..8064c54bf39 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -568,6 +568,8 @@ func (kgm *KeyspaceGroupManager) primaryPriorityCheckLoop() { kgm.primaryPriorityCheckInterval = 200 * time.Millisecond }) + ticker := time.NewTicker(kgm.primaryPriorityCheckInterval) + defer ticker.Stop() ctx, cancel := context.WithCancel(kgm.ctx) defer cancel() groupID := 0 @@ -576,7 +578,7 @@ func (kgm *KeyspaceGroupManager) primaryPriorityCheckLoop() { case <-ctx.Done(): log.Info("exit primary priority check loop") return - case <-time.After(kgm.primaryPriorityCheckInterval): + case <-ticker.C: // Every primaryPriorityCheckInterval, we only reset the primary of one keyspace group member, kg, localPriority, nextGroupID := kgm.getNextPrimaryToReset(groupID, kgm.tsoServiceID.ServiceAddr) if member != nil { diff --git a/server/api/pprof.go b/server/api/pprof.go index b64278a21b7..900c48f8368 100644 --- a/server/api/pprof.go +++ b/server/api/pprof.go @@ -209,8 +209,10 @@ func (h *pprofHandler) PProfThreadcreate(w http.ResponseWriter, r *http.Request) } func sleepWithCtx(ctx context.Context, d time.Duration) { + timer := time.NewTimer(d) + defer timer.Stop() select { - case <-time.After(d): + case <-timer.C: case <-ctx.Done(): } } diff --git a/server/grpc_service.go b/server/grpc_service.go index f66bd37ed11..83ea0c35dfe 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -960,13 +960,15 @@ func (b *bucketHeartbeatServer) Send(bucket *pdpb.ReportBucketsResponse) error { defer logutil.LogPanic() done <- b.stream.SendAndClose(bucket) }() + timer := time.NewTimer(heartbeatSendTimeout) + defer timer.Stop() select { case err := <-done: if err != nil { atomic.StoreInt32(&b.closed, 1) } return err - case <-time.After(heartbeatSendTimeout): + case <-timer.C: atomic.StoreInt32(&b.closed, 1) return ErrSendHeartbeatTimeout } @@ -1000,13 +1002,15 @@ func (s *heartbeatServer) Send(m *pdpb.RegionHeartbeatResponse) error { defer logutil.LogPanic() done <- s.stream.Send(m) }() + timer := time.NewTimer(heartbeatSendTimeout) + defer timer.Stop() select { case err := <-done: if err != nil { atomic.StoreInt32(&s.closed, 1) } return errors.WithStack(err) - case <-time.After(heartbeatSendTimeout): + case <-timer.C: atomic.StoreInt32(&s.closed, 1) return ErrSendHeartbeatTimeout } diff --git a/server/server.go b/server/server.go index 08d6896a3ef..ae1b47d9b40 100644 --- a/server/server.go +++ b/server/server.go @@ -617,9 +617,11 @@ func (s *Server) serverMetricsLoop() { ctx, cancel := context.WithCancel(s.serverLoopCtx) defer cancel() + ticker := time.NewTicker(serverMetricsInterval) + defer ticker.Stop() for { select { - case <-time.After(serverMetricsInterval): + case <-ticker.C: s.collectEtcdStateMetrics() case <-ctx.Done(): log.Info("server is closed, exit metrics loop") @@ -1672,10 +1674,14 @@ func (s *Server) etcdLeaderLoop() { ctx, cancel := context.WithCancel(s.serverLoopCtx) defer cancel() + ticker := time.NewTicker(s.cfg.LeaderPriorityCheckInterval.Duration) + defer ticker.Stop() for { select { - case <-time.After(s.cfg.LeaderPriorityCheckInterval.Duration): + case <-ticker.C: s.member.CheckPriority(ctx) + // Note: we reset the ticker here to support updating configuration dynamically. + ticker.Reset(s.cfg.LeaderPriorityCheckInterval.Duration) case <-ctx.Done(): log.Info("server is closed, exit etcd leader loop") return @@ -1815,6 +1821,8 @@ func (s *Server) UnmarkSnapshotRecovering(ctx context.Context) error { // GetServicePrimaryAddr returns the primary address for a given service. // Note: This function will only return primary address without judging if it's alive. func (s *Server) GetServicePrimaryAddr(ctx context.Context, serviceName string) (string, bool) { + ticker := time.NewTicker(retryIntervalGetServicePrimary) + defer ticker.Stop() for i := 0; i < maxRetryTimesGetServicePrimary; i++ { if v, ok := s.servicePrimaryMap.Load(serviceName); ok { return v.(string), true @@ -1824,7 +1832,7 @@ func (s *Server) GetServicePrimaryAddr(ctx context.Context, serviceName string) return "", false case <-ctx.Done(): return "", false - case <-time.After(retryIntervalGetServicePrimary): + case <-ticker.C: } } return "", false diff --git a/tools/pd-simulator/simulator/client.go b/tools/pd-simulator/simulator/client.go index 36e0d27869b..8dd1ee1646e 100644 --- a/tools/pd-simulator/simulator/client.go +++ b/tools/pd-simulator/simulator/client.go @@ -149,21 +149,22 @@ func (c *client) createHeartbeatStream() (pdpb.PD_RegionHeartbeatClient, context cancel context.CancelFunc ctx context.Context ) + ticker := time.NewTicker(time.Second) + defer ticker.Stop() for { ctx, cancel = context.WithCancel(c.ctx) stream, err = c.pdClient().RegionHeartbeat(ctx) - if err != nil { - simutil.Logger.Error("create region heartbeat stream error", zap.String("tag", c.tag), zap.Error(err)) - cancel() - select { - case <-time.After(time.Second): - continue - case <-c.ctx.Done(): - simutil.Logger.Info("cancel create stream loop") - return nil, ctx, cancel - } + if err == nil { + break + } + simutil.Logger.Error("create region heartbeat stream error", zap.String("tag", c.tag), zap.Error(err)) + cancel() + select { + case <-c.ctx.Done(): + simutil.Logger.Info("cancel create stream loop") + return nil, ctx, cancel + case <-ticker.C: } - break } return stream, ctx, cancel } From 9a3af1191d97292b4ea57aaf55415262282a8a2c Mon Sep 17 00:00:00 2001 From: Hu# Date: Mon, 3 Jul 2023 16:35:12 +0800 Subject: [PATCH 04/15] operator: Merge operators canceled together (#6673) close tikv/pd#6607 cancel two merge operators together so that they won't occupy the merge-schedule-limit. Signed-off-by: husharp Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/schedule/operator/operator.go | 5 +++ pkg/schedule/operator/operator_controller.go | 16 ++++++++ server/cluster/cluster_test.go | 41 ++++++++++++++++++++ 3 files changed, 62 insertions(+) diff --git a/pkg/schedule/operator/operator.go b/pkg/schedule/operator/operator.go index 02a2f1a8a5f..f8df4a428dd 100644 --- a/pkg/schedule/operator/operator.go +++ b/pkg/schedule/operator/operator.go @@ -18,6 +18,7 @@ import ( "encoding/json" "fmt" "reflect" + "strconv" "strings" "sync/atomic" "time" @@ -63,6 +64,8 @@ var ( ExceedStoreLimit CancelReasonType = "exceed store limit" // ExceedWaitLimit is the cancel reason when the operator exceeds the waiting queue limit. ExceedWaitLimit CancelReasonType = "exceed wait limit" + // RelatedMergeRegion is the cancel reason when the operator is cancelled by related merge region. + RelatedMergeRegion CancelReasonType = "related merge region" // Unknown is the cancel reason when the operator is cancelled by an unknown reason. Unknown CancelReasonType = "unknown" ) @@ -117,6 +120,8 @@ func NewOperator(desc, brief string, regionID uint64, regionEpoch *metapb.Region // Sync some attribute with the given timeout. func (o *Operator) Sync(other *Operator) { o.timeout = other.timeout + o.AdditionalInfos[string(RelatedMergeRegion)] = strconv.FormatUint(other.RegionID(), 10) + other.AdditionalInfos[string(RelatedMergeRegion)] = strconv.FormatUint(o.RegionID(), 10) } func (o *Operator) String() string { diff --git a/pkg/schedule/operator/operator_controller.go b/pkg/schedule/operator/operator_controller.go index b1e40a35e58..4e1ff6e5de2 100644 --- a/pkg/schedule/operator/operator_controller.go +++ b/pkg/schedule/operator/operator_controller.go @@ -575,11 +575,27 @@ func (oc *Controller) removeOperatorLocked(op *Operator) bool { oc.updateCounts(oc.operators) operatorCounter.WithLabelValues(op.Desc(), "remove").Inc() oc.ack(op) + if op.Kind()&OpMerge != 0 { + oc.removeRelatedMergeOperator(op) + } return true } return false } +func (oc *Controller) removeRelatedMergeOperator(op *Operator) { + relatedID, _ := strconv.ParseUint(op.AdditionalInfos[string(RelatedMergeRegion)], 10, 64) + if relatedOp := oc.operators[relatedID]; relatedOp != nil && relatedOp.Status() != CANCELED { + log.Info("operator canceled related merge region", + zap.Uint64("region-id", relatedOp.RegionID()), + zap.String("additional-info", relatedOp.GetAdditionalInfo()), + zap.Duration("takes", relatedOp.RunningTime())) + oc.removeOperatorLocked(relatedOp) + relatedOp.Cancel(RelatedMergeRegion) + oc.buryOperator(relatedOp) + } +} + func (oc *Controller) buryOperator(op *Operator) { st := op.Status() diff --git a/server/cluster/cluster_test.go b/server/cluster/cluster_test.go index a722fed9bda..dea913f3d47 100644 --- a/server/cluster/cluster_test.go +++ b/server/cluster/cluster_test.go @@ -2570,6 +2570,47 @@ func TestCheckerIsBusy(t *testing.T) { checkRegionAndOperator(re, tc, co, num, 0) } +func TestMergeRegionCancelOneOperator(t *testing.T) { + re := require.New(t) + tc, co, cleanup := prepare(nil, nil, nil, re) + defer cleanup() + + source := core.NewRegionInfo( + &metapb.Region{ + Id: 1, + StartKey: []byte(""), + EndKey: []byte("a"), + }, + nil, + ) + target := core.NewRegionInfo( + &metapb.Region{ + Id: 2, + StartKey: []byte("a"), + EndKey: []byte("t"), + }, + nil, + ) + re.NoError(tc.putRegion(source)) + re.NoError(tc.putRegion(target)) + + // Cancel source region. + ops, err := operator.CreateMergeRegionOperator("merge-region", tc, source, target, operator.OpMerge) + re.NoError(err) + re.Len(ops, co.GetOperatorController().AddWaitingOperator(ops...)) + // Cancel source operator. + co.GetOperatorController().RemoveOperator(co.GetOperatorController().GetOperator(source.GetID())) + re.Len(co.GetOperatorController().GetOperators(), 0) + + // Cancel target region. + ops, err = operator.CreateMergeRegionOperator("merge-region", tc, source, target, operator.OpMerge) + re.NoError(err) + re.Len(ops, co.GetOperatorController().AddWaitingOperator(ops...)) + // Cancel target operator. + co.GetOperatorController().RemoveOperator(co.GetOperatorController().GetOperator(target.GetID())) + re.Len(co.GetOperatorController().GetOperators(), 0) +} + func TestReplica(t *testing.T) { re := require.New(t) From a7eedd2d644df4146a540b38f486a6e100df8f18 Mon Sep 17 00:00:00 2001 From: Hu# Date: Tue, 4 Jul 2023 09:59:14 +0800 Subject: [PATCH 05/15] resource_manage: fix gosigar import package (#6735) close tikv/pd#6734 In TiDB we introduced cloudfoundry/gosigar But in PD we introduced elastic/gosigar Some compatibility issues may be encountered in both components Signed-off-by: husharp Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/go.mod | 2 +- client/go.sum | 14 ++++++++---- client/resource_group/controller/model.go | 6 ++--- tests/integrations/mcs/go.mod | 2 +- tests/integrations/mcs/go.sum | 11 +++++---- tools/pd-tso-bench/go.mod | 1 + tools/pd-tso-bench/go.sum | 27 +++++++++++++++++++++-- 7 files changed, 46 insertions(+), 17 deletions(-) diff --git a/client/go.mod b/client/go.mod index 50faaa4b87e..2fc40b88c55 100644 --- a/client/go.mod +++ b/client/go.mod @@ -3,7 +3,7 @@ module github.com/tikv/pd/client go 1.20 require ( - github.com/elastic/gosigar v0.14.2 + github.com/cloudfoundry/gosigar v1.3.6 github.com/gogo/protobuf v1.3.2 github.com/opentracing/opentracing-go v1.2.0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c diff --git a/client/go.sum b/client/go.sum index eb02df1d654..9e512fc6d03 100644 --- a/client/go.sum +++ b/client/go.sum @@ -15,11 +15,12 @@ github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6r github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cloudfoundry/gosigar v1.3.6 h1:gIc08FbB3QPb+nAQhINIK/qhf5REKkY0FTGgRGXkcVc= +github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/elastic/gosigar v0.14.2 h1:Dg80n8cr90OZ7x+bAax/QjoW/XqTI11RmA79ZwIm9/4= -github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= +github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= @@ -74,6 +75,9 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= +github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= +github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= @@ -174,7 +178,6 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180810173357-98c5dad5d1a0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -210,6 +213,7 @@ golang.org/x/tools v0.6.0 h1:BOw41kyTf3PuCW1pVQf8+Cyg8pMlkYB1oo9iJ6D/lKM= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 h1:KpwkzHKEF7B9Zxg18WzOa7djJ+Ha5DzthMyZYQfEn2A= @@ -229,16 +233,18 @@ google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqw gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= gopkg.in/natefinch/lumberjack.v2 v2.2.1/go.mod h1:YD8tP3GAjkrDg1eZH7EGmyESg/lsYskCTPBJVb9jqSc= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0 h1:clyUAQHOM3G0M3f5vQj7LuJrETvjVot3Z5el9nffUtU= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/client/resource_group/controller/model.go b/client/resource_group/controller/model.go index 2d8101bfa1b..55a6f9ec939 100644 --- a/client/resource_group/controller/model.go +++ b/client/resource_group/controller/model.go @@ -18,11 +18,11 @@ import ( "os" "time" - "github.com/elastic/gosigar" - "github.com/pingcap/log" + "github.com/cloudfoundry/gosigar" "go.uber.org/zap" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" + "github.com/pingcap/log" ) // RequestUnit is the basic unit of the resource request management, which has two types: @@ -265,7 +265,7 @@ func getSQLProcessCPUTime(isSingleGroupByKeyspace bool) float64 { func getSysProcessCPUTime() float64 { pid := os.Getpid() - cpuTime := gosigar.ProcTime{} + cpuTime := sigar.ProcTime{} if err := cpuTime.Get(pid); err != nil { log.Error("getCPUTime get pid failed", zap.Error(err)) } diff --git a/tests/integrations/mcs/go.mod b/tests/integrations/mcs/go.mod index 70f9bcc3a9a..abcfe6f615e 100644 --- a/tests/integrations/mcs/go.mod +++ b/tests/integrations/mcs/go.mod @@ -52,13 +52,13 @@ require ( github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 // indirect github.com/cenkalti/backoff/v4 v4.0.2 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/cloudfoundry/gosigar v1.3.6 // indirect github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f // indirect github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/docker/go-units v0.4.0 // indirect github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4 // indirect - github.com/elastic/gosigar v0.14.2 // indirect github.com/elliotchance/pie/v2 v2.1.0 // indirect github.com/fogleman/gg v1.3.0 // indirect github.com/gin-contrib/cors v1.4.0 // indirect diff --git a/tests/integrations/mcs/go.sum b/tests/integrations/mcs/go.sum index 0fd5cbf2d6f..74bcbaff0da 100644 --- a/tests/integrations/mcs/go.sum +++ b/tests/integrations/mcs/go.sum @@ -80,6 +80,8 @@ github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWR github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cloudfoundry/gosigar v1.3.6 h1:gIc08FbB3QPb+nAQhINIK/qhf5REKkY0FTGgRGXkcVc= +github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= @@ -112,8 +114,6 @@ github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4 h1:qk/FSDDxo05w github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385 h1:clC1lXBpe2kTj2VHdaIu9ajZQe4kcEY9j0NsnDDBZ3o= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= -github.com/elastic/gosigar v0.14.2 h1:Dg80n8cr90OZ7x+bAax/QjoW/XqTI11RmA79ZwIm9/4= -github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= github.com/elliotchance/pie/v2 v2.1.0 h1:KEVAAzxYxTyFs4hvebFZVzBdEo3YeMzl2HYDWn+P3F4= github.com/elliotchance/pie/v2 v2.1.0/go.mod h1:18t0dgGFH006g4eVdDtWfgFZPQEgl10IoEO8YWEq3Og= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= @@ -124,6 +124,7 @@ github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5Kwzbycv github.com/fogleman/gg v1.3.0 h1:/7zJX8F6AaYQc57WQCyN9cAIz+4bCJGO9B+dyW29am8= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/cors v1.4.0 h1:oJ6gwtUl3lqV0WEIwM/LxPF1QZ5qe2lGWdY2+bz7y0g= github.com/gin-contrib/cors v1.4.0/go.mod h1:bs9pNM0x/UsmHPBWT2xZz9ROh8xYjYkiURUfmBoMlcs= @@ -252,7 +253,6 @@ github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFb github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69 h1:7xsUJsB2NrdcttQPa7JLEaGzvdbk7KvfrjgHZXOQRo0= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= -github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d h1:uGg2frlt3IcT7kbV6LEp5ONv4vmoO2FW4qSO+my/aoM= github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w= @@ -352,13 +352,14 @@ github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRW github.com/nfnt/resize v0.0.0-20160724205520-891127d8d1b5 h1:BvoENQQU+fZ9uukda/RzCAL/191HHwJA5b13R6diVlY= github.com/nfnt/resize v0.0.0-20160724205520-891127d8d1b5/go.mod h1:jpp1/29i3P1S/RLdc7JQKbRpFeM1dOBd8T9ki5s+AY8= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/oleiade/reflections v1.0.1 h1:D1XO3LVEYroYskEsoSiGItp9RUxG6jWnCVvrqH0HHQM= github.com/oleiade/reflections v1.0.1/go.mod h1:rdFxbxq4QXVZWj0F+e9jqjDkc7dbp97vkRixKo2JR60= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.20.1 h1:PA/3qinGoukvymdIDV8pii6tiZgC8kbmJO6Z5+b002Q= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= @@ -656,7 +657,6 @@ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0 h1:wsuoTGHzEhffawBOhz5CYhcrV4IdKZbEyZjBMuTp12o= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180810173357-98c5dad5d1a0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -779,7 +779,6 @@ gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntN gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= -gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= diff --git a/tools/pd-tso-bench/go.mod b/tools/pd-tso-bench/go.mod index cbfb4ad3c7e..6c10dbb5fce 100644 --- a/tools/pd-tso-bench/go.mod +++ b/tools/pd-tso-bench/go.mod @@ -9,6 +9,7 @@ require ( github.com/prometheus/client_golang v1.11.1 github.com/tikv/pd/client v0.0.0-00010101000000-000000000000 go.uber.org/zap v1.24.0 + google.golang.org/grpc v1.54.0 ) replace github.com/tikv/pd/client => ../../client diff --git a/tools/pd-tso-bench/go.sum b/tools/pd-tso-bench/go.sum index 4a22bcfc2c4..b7ca93b1272 100644 --- a/tools/pd-tso-bench/go.sum +++ b/tools/pd-tso-bench/go.sum @@ -633,6 +633,7 @@ github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWR github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cloudfoundry/gosigar v1.3.6/go.mod h1:lNWstu5g5gw59O09Y+wsMNFzBSnU8a0u+Sfx4dq360E= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= @@ -651,7 +652,6 @@ github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/elastic/gosigar v0.14.2/go.mod h1:iXRIGg2tLnu7LBdpqzyQfGDEidKCfWcCMS0WKyPWoMs= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= @@ -667,6 +667,9 @@ github.com/envoyproxy/protoc-gen-validate v0.6.7/go.mod h1:dyJXwwfPK2VSqiB9Klm1J github.com/envoyproxy/protoc-gen-validate v0.9.1/go.mod h1:OKNgG7TCp5pF4d6XftA0++PMirau2/yoOwVac3AbF2w= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= +github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= +github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-fonts/dejavu v0.1.0/go.mod h1:4Wt4I4OU2Nq9asgDCteaAaWZOV24E+0/Pwo0gppep4g= github.com/go-fonts/latin-modern v0.2.0/go.mod h1:rQVLdDMK+mK1xscDwsqM5J8U2jrRa3T0ecnM9pNujks= @@ -687,6 +690,7 @@ github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG github.com/go-pdf/fpdf v0.5.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/go-pdf/fpdf v0.6.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= @@ -794,6 +798,7 @@ github.com/grpc-ecosystem/grpc-gateway/v2 v2.7.0/go.mod h1:hgWBS7lorOAVIJEQMi4Zs github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.3/go.mod h1:o//XUCC/F+yRGJoPO/VU0GSB0f8Nhgmxx0VIRUvaC0w= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/iancoleman/strcase v0.2.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= @@ -840,6 +845,13 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= +github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= +github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= +github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= +github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= github.com/phpdave11/gofpdf v1.4.2/go.mod h1:zpO6xFn9yxo3YLyMvW8HcKWVdbNqgIfOOp2dXMnm1mY= @@ -1017,6 +1029,7 @@ golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1039,6 +1052,7 @@ golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/ golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= @@ -1118,9 +1132,9 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sys v0.0.0-20180810173357-98c5dad5d1a0/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1131,7 +1145,10 @@ golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1156,6 +1173,7 @@ golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1198,6 +1216,7 @@ golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220829200755-d48e67d00261/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.4.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -1285,6 +1304,7 @@ golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20201124115921-2c860bdd6e78/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210108195828-e2f9c7f1fc8e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -1297,6 +1317,7 @@ golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= +golang.org/x/tools v0.5.0/go.mod h1:N+Kgy78s5I24c24dU8OfWNEotWjutIs8SnJvn5IDq+k= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1579,9 +1600,11 @@ gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/natefinch/lumberjack.v2 v2.2.1 h1:bBRl1b0OH9s/DuPhuXpNl+VtCaJXFZ5/uEFST95x9zc= gopkg.in/natefinch/lumberjack.v2 v2.2.1/go.mod h1:YD8tP3GAjkrDg1eZH7EGmyESg/lsYskCTPBJVb9jqSc= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= From d0c03a3cb91aafc1058e4951060ac641b1626c9f Mon Sep 17 00:00:00 2001 From: JmPotato Date: Tue, 4 Jul 2023 14:17:13 +0800 Subject: [PATCH 06/15] tso: implement groupSplitPatroller to speed up the split process (#6736) ref tikv/pd#5895, close tikv/pd#6696 Implement `groupSplitPatroller` to speed up the split process. Signed-off-by: JmPotato Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- pkg/tso/keyspace_group_manager.go | 71 ++++++++++++++++++- .../mcs/tso/keyspace_group_manager_test.go | 20 +++--- 2 files changed, 77 insertions(+), 14 deletions(-) diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 8064c54bf39..529e9bcdd37 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -61,6 +61,7 @@ const ( // of the primaries on this TSO server/pod have changed. A goroutine will periodically check // do this check and re-distribute the primaries if necessary. defaultPrimaryPriorityCheckInterval = 10 * time.Second + groupPatrolInterval = time.Minute ) type state struct { @@ -74,13 +75,16 @@ type state struct { kgs [mcsutils.MaxKeyspaceGroupCountInUse]*endpoint.KeyspaceGroup // keyspaceLookupTable is a map from keyspace to the keyspace group to which it belongs. keyspaceLookupTable map[uint32]uint32 + // splittingGroups is the cache of splitting keyspace group related information. + splittingGroups map[uint32]struct{} } func (s *state) initialize() { s.keyspaceLookupTable = make(map[uint32]uint32) + s.splittingGroups = make(map[uint32]struct{}) } -func (s *state) deinitialize() { +func (s *state) deInitialize() { log.Info("closing all keyspace groups") s.Lock() @@ -398,8 +402,9 @@ func (kgm *KeyspaceGroupManager) Initialize() error { return errs.ErrLoadKeyspaceGroupsTerminated.Wrap(err) } - kgm.wg.Add(1) + kgm.wg.Add(2) go kgm.primaryPriorityCheckLoop() + go kgm.groupSplitPatroller() return nil } @@ -415,7 +420,7 @@ func (kgm *KeyspaceGroupManager) Close() { // added/initialized after that. kgm.cancel() kgm.wg.Wait() - kgm.state.deinitialize() + kgm.state.deInitialize() log.Info("keyspace group manager closed") } @@ -732,6 +737,10 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroup(group *endpoint.KeyspaceGro } kgm.kgs[group.ID] = group kgm.ams[group.ID] = am + // If the group is the split target, add it to the splitting group map. + if group.IsSplitTarget() { + kgm.splittingGroups[group.ID] = struct{}{} + } kgm.Unlock() } @@ -859,6 +868,7 @@ func (kgm *KeyspaceGroupManager) updateKeyspaceGroupMembership( // Check if the split is completed. if oldGroup != nil && oldGroup.IsSplitTarget() && !newGroup.IsSplitting() { kgm.ams[groupID].GetMember().(*member.Participant).SetCampaignChecker(nil) + delete(kgm.splittingGroups, groupID) } kgm.kgs[groupID] = newGroup } @@ -1322,3 +1332,58 @@ func (kgm *KeyspaceGroupManager) mergingChecker(ctx context.Context, mergeTarget return } } + +// groupSplitPatroller is used to patrol the groups that are in the on-going +// split state and to check if we could speed up the split process. +func (kgm *KeyspaceGroupManager) groupSplitPatroller() { + defer kgm.wg.Done() + patrolInterval := groupPatrolInterval + failpoint.Inject("fastGroupSplitPatroller", func() { + patrolInterval = 200 * time.Millisecond + }) + ticker := time.NewTicker(patrolInterval) + defer ticker.Stop() + log.Info("group split patroller is started", + zap.Duration("patrol-interval", patrolInterval)) + for { + select { + case <-kgm.ctx.Done(): + log.Info("group split patroller is exiting") + return + case <-ticker.C: + } + kgm.RLock() + if len(kgm.splittingGroups) == 0 { + kgm.RUnlock() + continue + } + var splittingGroups []uint32 + for id := range kgm.splittingGroups { + splittingGroups = append(splittingGroups, id) + } + kgm.RUnlock() + for _, groupID := range splittingGroups { + am, group := kgm.getKeyspaceGroupMeta(groupID) + if !am.IsLeader() { + continue + } + if len(group.Keyspaces) == 0 { + log.Warn("abnormal keyspace group with empty keyspace list", + zap.Uint32("keyspace-group-id", groupID)) + continue + } + log.Info("request tso for the splitting keyspace group", + zap.Uint32("keyspace-group-id", groupID), + zap.Uint32("keyspace-id", group.Keyspaces[0])) + // Request the TSO manually to speed up the split process. + _, _, err := kgm.HandleTSORequest(group.Keyspaces[0], groupID, GlobalDCLocation, 1) + if err != nil { + log.Warn("failed to request tso for the splitting keyspace group", + zap.Uint32("keyspace-group-id", groupID), + zap.Uint32("keyspace-id", group.Keyspaces[0]), + zap.Error(err)) + continue + } + } + } +} diff --git a/tests/integrations/mcs/tso/keyspace_group_manager_test.go b/tests/integrations/mcs/tso/keyspace_group_manager_test.go index 98c6b90ca28..a20eb33fb81 100644 --- a/tests/integrations/mcs/tso/keyspace_group_manager_test.go +++ b/tests/integrations/mcs/tso/keyspace_group_manager_test.go @@ -63,6 +63,7 @@ func TestTSOKeyspaceGroupManager(t *testing.T) { func (suite *tsoKeyspaceGroupManagerTestSuite) SetupSuite() { re := suite.Require() + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller", `return(true)`)) var err error suite.ctx, suite.cancel = context.WithCancel(context.Background()) @@ -81,6 +82,7 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TearDownSuite() { suite.cancel() suite.tsoCluster.Destroy() suite.cluster.Destroy() + suite.Require().NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller")) } func (suite *tsoKeyspaceGroupManagerTestSuite) TearDownTest() { @@ -276,17 +278,15 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplit() { NewID: 2, Keyspaces: []uint32{222, 333}, }) - kg2 := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 2) - re.Equal(uint32(2), kg2.ID) - re.Equal([]uint32{222, 333}, kg2.Keyspaces) - re.True(kg2.IsSplitTarget()) - // Check the split TSO from keyspace group 2. - var splitTS pdpb.Timestamp + // Wait for the split to complete automatically even there is no TSO request from the outside. testutil.Eventually(re, func() bool { - splitTS, err = suite.requestTSO(re, 222, 2) - return err == nil && tsoutil.CompareTimestamp(&splitTS, &pdpb.Timestamp{}) > 0 + kg2 := handlersutil.MustLoadKeyspaceGroupByID(re, suite.pdLeaderServer, 2) + re.Equal(uint32(2), kg2.ID) + re.Equal([]uint32{222, 333}, kg2.Keyspaces) + return !kg2.IsSplitting() }) - splitTS, err = suite.requestTSO(re, 222, 2) + // Check the split TSO from keyspace group 2 now. + splitTS, err := suite.requestTSO(re, 222, 2) re.NoError(err) re.Greater(tsoutil.CompareTimestamp(&splitTS, &ts), 0) } @@ -356,8 +356,6 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupSplitElection return len(member1.GetLeaderListenUrls()) > 0 && len(member2.GetLeaderListenUrls()) > 0 }) re.Equal(member1.GetLeaderListenUrls(), member2.GetLeaderListenUrls()) - // Finish the split. - handlersutil.MustFinishSplitKeyspaceGroup(re, suite.pdLeaderServer, 2) // Wait for the keyspace groups to finish the split. waitFinishSplit(re, suite.pdLeaderServer, 1, 2, []uint32{111}, []uint32{222, 333}) } From 69c6a344c72db4bad8063f33c70ebd42f299f22a Mon Sep 17 00:00:00 2001 From: Bin Shi <39923490+binshi-bing@users.noreply.github.com> Date: Tue, 4 Jul 2023 20:35:14 -0700 Subject: [PATCH 07/15] client: fix tso service discovery at the first time for NewClientWithAPIContext (#6749) close tikv/pd#6748 After NewClientWithAPIContextV2 returns, the keyspace group should be discovered by the passed keyspace name immediately Signed-off-by: Bin Shi --- client/client.go | 32 ++++++++++++++------------------ client/pd_service_discovery.go | 18 ++++++++++++++---- client/tso_service_discovery.go | 19 ++++++++++++------- 3 files changed, 40 insertions(+), 29 deletions(-) diff --git a/client/client.go b/client/client.go index 5fbc5f9eb23..006e303c1cd 100644 --- a/client/client.go +++ b/client/client.go @@ -277,14 +277,6 @@ type serviceModeKeeper struct { tsoSvcDiscovery ServiceDiscovery } -func (k *serviceModeKeeper) SetKeyspaceID(keyspaceID uint32) { - k.Lock() - defer k.Unlock() - if k.serviceMode == pdpb.ServiceMode_API_SVC_MODE { - k.tsoSvcDiscovery.SetKeyspaceID(keyspaceID) - } -} - func (k *serviceModeKeeper) close() { k.Lock() defer k.Unlock() @@ -392,7 +384,7 @@ func createClientWithKeyspace( c.pdSvcDiscovery = newPDServiceDiscovery( clientCtx, clientCancel, &c.wg, c.setServiceMode, - keyspaceID, c.svrUrls, c.tlsCfg, c.option) + nil, keyspaceID, c.svrUrls, c.tlsCfg, c.option) if err := c.setup(); err != nil { c.cancel() return nil, err @@ -504,23 +496,27 @@ func newClientWithKeyspaceName( opt(c) } + updateKeyspaceIDCb := func() error { + if err := c.initRetry(c.loadKeyspaceMeta, keyspaceName); err != nil { + return err + } + // c.keyspaceID is the source of truth for keyspace id. + c.pdSvcDiscovery.(*pdServiceDiscovery).SetKeyspaceID(c.keyspaceID) + return nil + } + // Create a PD service discovery with null keyspace id, then query the real id wth the keyspace name, // finally update the keyspace id to the PD service discovery for the following interactions. c.pdSvcDiscovery = newPDServiceDiscovery( - clientCtx, clientCancel, &c.wg, c.setServiceMode, nullKeyspaceID, c.svrUrls, c.tlsCfg, c.option) + clientCtx, clientCancel, &c.wg, c.setServiceMode, updateKeyspaceIDCb, nullKeyspaceID, c.svrUrls, c.tlsCfg, c.option) if err := c.setup(); err != nil { c.cancel() return nil, err } - if err := c.initRetry(c.loadKeyspaceMeta, keyspaceName); err != nil { - return nil, err - } - // We call "c.pdSvcDiscovery.SetKeyspaceID(c.keyspaceID)" after service mode already switching to API mode - // and tso service discovery already initialized, so here we need to set the tso_service_discovery's keyspace id too. - c.pdSvcDiscovery.SetKeyspaceID(c.keyspaceID) - c.serviceModeKeeper.SetKeyspaceID(c.keyspaceID) log.Info("[pd] create pd client with endpoints and keyspace", - zap.Strings("pd-address", svrAddrs), zap.String("keyspace-name", keyspaceName), zap.Uint32("keyspace-id", c.keyspaceID)) + zap.Strings("pd-address", svrAddrs), + zap.String("keyspace-name", keyspaceName), + zap.Uint32("keyspace-id", c.keyspaceID)) return c, nil } diff --git a/client/pd_service_discovery.go b/client/pd_service_discovery.go index bd7ed31209a..4499c9e17c0 100644 --- a/client/pd_service_discovery.go +++ b/client/pd_service_discovery.go @@ -59,8 +59,6 @@ type ServiceDiscovery interface { GetClusterID() uint64 // GetKeyspaceID returns the ID of the keyspace GetKeyspaceID() uint32 - // SetKeyspaceID sets the ID of the keyspace - SetKeyspaceID(keyspaceID uint32) // GetKeyspaceGroupID returns the ID of the keyspace group GetKeyspaceGroupID() uint32 // DiscoverServiceURLs discovers the microservice with the specified type and returns the server urls. @@ -99,6 +97,7 @@ type ServiceDiscovery interface { AddServiceAddrsSwitchedCallback(callbacks ...func()) } +type updateKeyspaceIDFunc func() error type tsoLocalServAddrsUpdatedFunc func(map[string]string) error type tsoGlobalServAddrUpdatedFunc func(string) error @@ -149,8 +148,9 @@ type pdServiceDiscovery struct { cancel context.CancelFunc closeOnce sync.Once - keyspaceID uint32 - tlsCfg *tlsutil.TLSConfig + updateKeyspaceIDCb updateKeyspaceIDFunc + keyspaceID uint32 + tlsCfg *tlsutil.TLSConfig // Client option. option *option } @@ -160,6 +160,7 @@ func newPDServiceDiscovery( ctx context.Context, cancel context.CancelFunc, wg *sync.WaitGroup, serviceModeUpdateCb func(pdpb.ServiceMode), + updateKeyspaceIDCb updateKeyspaceIDFunc, keyspaceID uint32, urls []string, tlsCfg *tlsutil.TLSConfig, option *option, ) *pdServiceDiscovery { @@ -169,6 +170,7 @@ func newPDServiceDiscovery( cancel: cancel, wg: wg, serviceModeUpdateCb: serviceModeUpdateCb, + updateKeyspaceIDCb: updateKeyspaceIDCb, keyspaceID: keyspaceID, tlsCfg: tlsCfg, option: option, @@ -192,6 +194,14 @@ func (c *pdServiceDiscovery) Init() error { } log.Info("[pd] init cluster id", zap.Uint64("cluster-id", c.clusterID)) + // We need to update the keyspace ID before we discover and update the service mode + // so that TSO in API mode can be initialized with the correct keyspace ID. + if c.updateKeyspaceIDCb != nil { + if err := c.updateKeyspaceIDCb(); err != nil { + return err + } + } + if err := c.checkServiceModeChanged(); err != nil { log.Warn("[pd] failed to check service mode and will check later", zap.Error(err)) } diff --git a/client/tso_service_discovery.go b/client/tso_service_discovery.go index 6763e80976a..c35809a347b 100644 --- a/client/tso_service_discovery.go +++ b/client/tso_service_discovery.go @@ -277,11 +277,6 @@ func (c *tsoServiceDiscovery) GetKeyspaceID() uint32 { return c.keyspaceID.Load() } -// SetKeyspaceID sets the ID of the keyspace -func (c *tsoServiceDiscovery) SetKeyspaceID(keyspaceID uint32) { - c.keyspaceID.Store(keyspaceID) -} - // GetKeyspaceGroupID returns the ID of the keyspace group. If the keyspace group is unknown, // it returns the default keyspace group ID. func (c *tsoServiceDiscovery) GetKeyspaceGroupID() uint32 { @@ -429,12 +424,16 @@ func (c *tsoServiceDiscovery) updateMember() error { return err } + keyspaceID := c.GetKeyspaceID() var keyspaceGroup *tsopb.KeyspaceGroup if len(tsoServerAddr) > 0 { - keyspaceGroup, err = c.findGroupByKeyspaceID(c.GetKeyspaceID(), tsoServerAddr, updateMemberTimeout) + keyspaceGroup, err = c.findGroupByKeyspaceID(keyspaceID, tsoServerAddr, updateMemberTimeout) if err != nil { if c.tsoServerDiscovery.countFailure() { - log.Error("[tso] failed to find the keyspace group", errs.ZapError(err)) + log.Error("[tso] failed to find the keyspace group", + zap.Uint32("keyspace-id-in-request", keyspaceID), + zap.String("tso-server-addr", tsoServerAddr), + errs.ZapError(err)) } return err } @@ -448,6 +447,8 @@ func (c *tsoServiceDiscovery) updateMember() error { c.printFallbackLogOnce.Do(func() { log.Warn("[tso] no tso server address found,"+ " fallback to the legacy path to discover from etcd directly", + zap.Uint32("keyspace-id-in-request", keyspaceID), + zap.String("tso-server-addr", tsoServerAddr), zap.String("discovery-key", c.defaultDiscoveryKey)) }) addrs, err := c.discoverWithLegacyPath() @@ -487,6 +488,8 @@ func (c *tsoServiceDiscovery) updateMember() error { if primarySwitched := !strings.EqualFold(primaryAddr, c.getPrimaryAddr()); primarySwitched { if _, err := c.GetOrCreateGRPCConn(primaryAddr); err != nil { log.Warn("[tso] failed to connect the next primary", + zap.Uint32("keyspace-id-in-request", keyspaceID), + zap.String("tso-server-addr", tsoServerAddr), zap.String("next-primary", primaryAddr), errs.ZapError(err)) return err } @@ -497,6 +500,8 @@ func (c *tsoServiceDiscovery) updateMember() error { c.keyspaceGroupSD.update(keyspaceGroup, primaryAddr, secondaryAddrs, addrs) if primarySwitched { log.Info("[tso] updated keyspace group service discovery info", + zap.Uint32("keyspace-id-in-request", keyspaceID), + zap.String("tso-server-addr", tsoServerAddr), zap.String("keyspace-group-service", keyspaceGroup.String())) if err := c.afterPrimarySwitched(oldPrimary, primaryAddr); err != nil { return err From dc180cad641495cf5ed8c401bfcd502e706d75bd Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Wed, 5 Jul 2023 14:38:13 +0800 Subject: [PATCH 08/15] *: add test for misusing keyspace ID when creating the client (#6754) ref tikv/pd#6747, ref tikv/pd#6748, ref tikv/pd#6749 Signed-off-by: Ryan Leung --- .gitignore | 1 + pkg/keyspace/tso_keyspace_group.go | 32 +++++ pkg/keyspace/util.go | 3 + .../mcs/tso/keyspace_group_manager_test.go | 113 +++++++++++------- 4 files changed, 108 insertions(+), 41 deletions(-) diff --git a/.gitignore b/.gitignore index 93e6189a687..748d24872b6 100644 --- a/.gitignore +++ b/.gitignore @@ -24,3 +24,4 @@ report.xml coverage.xml coverage *.txt +go.work* diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index dd9319e806f..d319798738b 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -17,6 +17,7 @@ package keyspace import ( "context" "encoding/json" + "fmt" "sort" "strconv" "strings" @@ -25,6 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/balancer" "github.com/tikv/pd/pkg/mcs/discovery" @@ -1010,3 +1012,33 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { zap.Reflect("merge-list", mergeList)) return nil } + +// GetKeyspaceGroupPrimaryByID returns the primary node of the keyspace group by ID. +func (m *GroupManager) GetKeyspaceGroupPrimaryByID(id uint32) (string, error) { + // check if the keyspace group exists + kg, err := m.GetKeyspaceGroupByID(id) + if err != nil { + return "", err + } + if kg == nil { + return "", ErrKeyspaceGroupNotExists(id) + } + + // default keyspace group: "/ms/{cluster_id}/tso/00000/primary". + // non-default keyspace group: "/ms/{cluster_id}/tso/keyspace_groups/election/{group}/primary". + path := fmt.Sprintf("/ms/%d/tso/00000/primary", m.clusterID) + if id != utils.DefaultKeyspaceGroupID { + path = fmt.Sprintf("/ms/%d/tso/keyspace_groups/election/%05d/primary", m.clusterID, id) + } + leader := &tsopb.Participant{} + ok, _, err := etcdutil.GetProtoMsgWithModRev(m.client, path, leader) + if err != nil { + return "", err + } + if !ok { + return "", ErrKeyspaceGroupPrimaryNotFound + } + // The format of leader name is address-groupID. + contents := strings.Split(leader.GetName(), "-") + return contents[0], err +} diff --git a/pkg/keyspace/util.go b/pkg/keyspace/util.go index 240306f8124..100b0eb6986 100644 --- a/pkg/keyspace/util.go +++ b/pkg/keyspace/util.go @@ -94,6 +94,9 @@ var ( } // Only keyspaces in the state specified by allowChangeConfig are allowed to change their config. allowChangeConfig = []keyspacepb.KeyspaceState{keyspacepb.KeyspaceState_ENABLED, keyspacepb.KeyspaceState_DISABLED} + + // ErrKeyspaceGroupPrimaryNotFound is used to indicate primary of target keyspace group does not exist. + ErrKeyspaceGroupPrimaryNotFound = errors.New("primary of keyspace group does not exist") ) // validateID check if keyspace falls within the acceptable range. diff --git a/tests/integrations/mcs/tso/keyspace_group_manager_test.go b/tests/integrations/mcs/tso/keyspace_group_manager_test.go index a20eb33fb81..52974e3155f 100644 --- a/tests/integrations/mcs/tso/keyspace_group_manager_test.go +++ b/tests/integrations/mcs/tso/keyspace_group_manager_test.go @@ -493,6 +493,7 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller", `return(true)`)) // Init api server config but not start. tc, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { @@ -503,21 +504,6 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { re.NoError(err) pdAddr := tc.GetConfig().GetClientURL() - // Start pd client and wait pd server start. - var clients sync.Map - go func() { - apiCtx := pd.NewAPIContextV2("keyspace_b") // its keyspace id is 2. - cli, err := pd.NewClientWithAPIContext(ctx, apiCtx, []string{pdAddr}, pd.SecurityOption{}) - re.NoError(err) - clients.Store("keyspace_b", cli) - }() - go func() { - apiCtx := pd.NewAPIContextV2("keyspace_a") // its keyspace id is 1. - cli, err := pd.NewClientWithAPIContext(ctx, apiCtx, []string{pdAddr}, pd.SecurityOption{}) - re.NoError(err) - clients.Store("keyspace_a", cli) - }() - // Start api server and tso server. err = tc.RunInitialServers() re.NoError(err) @@ -531,20 +517,6 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { defer tsoCluster.Destroy() tsoCluster.WaitForDefaultPrimaryServing(re) - // Wait pd clients are ready. - testutil.Eventually(re, func() bool { - count := 0 - clients.Range(func(_, _ interface{}) bool { - count++ - return true - }) - return count == 2 - }) - clientA, ok := clients.Load("keyspace_a") - re.True(ok) - clientB, ok := clients.Load("keyspace_b") - re.True(ok) - // First split keyspace group 0 to 1 with keyspace 2. kgm := leaderServer.GetServer().GetKeyspaceGroupManager() re.NotNil(kgm) @@ -553,13 +525,7 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { return err == nil }) - // Trigger checkTSOSplit to ensure the split is finished. - testutil.Eventually(re, func() bool { - _, _, err = clientB.(pd.Client).GetTS(ctx) - return err == nil - }) waitFinishSplit(re, leaderServer, 0, 1, []uint32{mcsutils.DefaultKeyspaceID, 1}, []uint32{2}) - clientB.(pd.Client).Close() // Then split keyspace group 0 to 2 with keyspace 1. testutil.Eventually(re, func() bool { @@ -567,13 +533,7 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { return err == nil }) - // Trigger checkTSOSplit to ensure the split is finished. - testutil.Eventually(re, func() bool { - _, _, err = clientA.(pd.Client).GetTS(ctx) - return err == nil - }) waitFinishSplit(re, leaderServer, 0, 2, []uint32{mcsutils.DefaultKeyspaceID}, []uint32{1}) - clientA.(pd.Client).Close() // Check the keyspace group 0 is split to 1 and 2. kg0 := handlersutil.MustLoadKeyspaceGroupByID(re, leaderServer, 0) @@ -586,6 +546,7 @@ func TestTwiceSplitKeyspaceGroup(t *testing.T) { re.False(kg1.IsSplitting()) re.False(kg2.IsSplitting()) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller")) re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) } @@ -724,3 +685,73 @@ func (suite *tsoKeyspaceGroupManagerTestSuite) TestTSOKeyspaceGroupMergeClient() cancel() wg.Wait() } + +// See https://github.com/tikv/pd/issues/6748 +func TestGetTSOImmediately(t *testing.T) { + re := require.New(t) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastPrimaryPriorityCheck", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes", `return(true)`)) + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller", `return(true)`)) + + // Init api server config but not start. + tc, err := tests.NewTestAPICluster(ctx, 1, func(conf *config.Config, _ string) { + conf.Keyspace.PreAlloc = []string{ + "keyspace_a", "keyspace_b", + } + }) + re.NoError(err) + pdAddr := tc.GetConfig().GetClientURL() + + // Start api server and tso server. + err = tc.RunInitialServers() + re.NoError(err) + defer tc.Destroy() + tc.WaitLeader() + leaderServer := tc.GetServer(tc.GetLeader()) + re.NoError(leaderServer.BootstrapCluster()) + + tsoCluster, err := mcs.NewTestTSOCluster(ctx, 2, pdAddr) + re.NoError(err) + defer tsoCluster.Destroy() + tsoCluster.WaitForDefaultPrimaryServing(re) + + // First split keyspace group 0 to 1 with keyspace 2. + kgm := leaderServer.GetServer().GetKeyspaceGroupManager() + re.NotNil(kgm) + testutil.Eventually(re, func() bool { + err = kgm.SplitKeyspaceGroupByID(0, 1, []uint32{2}) + return err == nil + }) + + waitFinishSplit(re, leaderServer, 0, 1, []uint32{mcsutils.DefaultKeyspaceID, 1}, []uint32{2}) + + kg0 := handlersutil.MustLoadKeyspaceGroupByID(re, leaderServer, 0) + kg1 := handlersutil.MustLoadKeyspaceGroupByID(re, leaderServer, 1) + re.Equal([]uint32{0, 1}, kg0.Keyspaces) + re.Equal([]uint32{2}, kg1.Keyspaces) + re.False(kg0.IsSplitting()) + re.False(kg1.IsSplitting()) + + // Let group 0 and group 1 have different primary node. + kgm.SetPriorityForKeyspaceGroup(0, kg0.Members[0].Address, 100) + kgm.SetPriorityForKeyspaceGroup(1, kg1.Members[1].Address, 100) + testutil.Eventually(re, func() bool { + p0, err := kgm.GetKeyspaceGroupPrimaryByID(0) + re.NoError(err) + p1, err := kgm.GetKeyspaceGroupPrimaryByID(1) + re.NoError(err) + return p0 == kg0.Members[0].Address && p1 == kg1.Members[1].Address + }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) + + apiCtx := pd.NewAPIContextV2("keyspace_b") // its keyspace id is 2. + cli, err := pd.NewClientWithAPIContext(ctx, apiCtx, []string{pdAddr}, pd.SecurityOption{}) + re.NoError(err) + _, _, err = cli.GetTS(ctx) + re.NoError(err) + cli.Close() + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastPrimaryPriorityCheck")) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/keyspace/acceleratedAllocNodes")) + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastGroupSplitPatroller")) +} From 693e172cf34cd15c96adbac596b91eb34925df20 Mon Sep 17 00:00:00 2001 From: Bin Shi <39923490+binshi-bing@users.noreply.github.com> Date: Wed, 5 Jul 2023 00:09:43 -0700 Subject: [PATCH 09/15] tso: support multi-keyspace, fault injection and keyspace-name in pd-tso-bench (#6608) ref tikv/pd#5895 support multi-keyspace, fault injection and keyspace-name in pd-tso-bench Signed-off-by: Bin Shi Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/tso_service_discovery.go | 8 ++ tools/pd-tso-bench/main.go | 157 ++++++++++++++++++++++---------- 2 files changed, 119 insertions(+), 46 deletions(-) diff --git a/client/tso_service_discovery.go b/client/tso_service_discovery.go index c35809a347b..2aeb49e1523 100644 --- a/client/tso_service_discovery.go +++ b/client/tso_service_discovery.go @@ -469,6 +469,14 @@ func (c *tsoServiceDiscovery) updateMember() error { } } + oldGroupID := c.GetKeyspaceGroupID() + if oldGroupID != keyspaceGroup.Id { + log.Info("[tso] the keyspace group changed", + zap.Uint32("keyspace-id", keyspaceGroup.Id), + zap.Uint32("new-keyspace-group-id", keyspaceGroup.Id), + zap.Uint32("old-keyspace-group-id", oldGroupID)) + } + // Initialize the serving addresses from the returned keyspace group info. primaryAddr := "" secondaryAddrs := make([]string, 0) diff --git a/tools/pd-tso-bench/main.go b/tools/pd-tso-bench/main.go index e82a9dcabba..236e78c7808 100644 --- a/tools/pd-tso-bench/main.go +++ b/tools/pd-tso-bench/main.go @@ -19,6 +19,7 @@ import ( "flag" "fmt" "io" + "math/rand" "net/http" "net/http/httptest" "os" @@ -43,20 +44,25 @@ const ( ) var ( - pdAddrs = flag.String("pd", "127.0.0.1:2379", "pd address") - clientNumber = flag.Int("client", 1, "the number of pd clients involved in each benchmark") - concurrency = flag.Int("c", 1000, "concurrency") - count = flag.Int("count", 1, "the count number that the test will run") - duration = flag.Duration("duration", 60*time.Second, "how many seconds the test will last") - dcLocation = flag.String("dc", "global", "which dc-location this bench will request") - verbose = flag.Bool("v", false, "output statistics info every interval and output metrics info at the end") - interval = flag.Duration("interval", time.Second, "interval to output the statistics") - caPath = flag.String("cacert", "", "path of file that contains list of trusted SSL CAs") - certPath = flag.String("cert", "", "path of file that contains X509 certificate in PEM format") - keyPath = flag.String("key", "", "path of file that contains X509 key in PEM format") - maxBatchWaitInterval = flag.Duration("batch-interval", 0, "the max batch wait interval") - enableTSOFollowerProxy = flag.Bool("enable-tso-follower-proxy", false, "whether enable the TSO Follower Proxy") - wg sync.WaitGroup + pdAddrs = flag.String("pd", "127.0.0.1:2379", "pd address") + clientNumber = flag.Int("client", 1, "the number of pd clients involved in each benchmark") + concurrency = flag.Int("c", 1000, "concurrency") + count = flag.Int("count", 1, "the count number that the test will run") + duration = flag.Duration("duration", 60*time.Second, "how many seconds the test will last") + dcLocation = flag.String("dc", "global", "which dc-location this bench will request") + verbose = flag.Bool("v", false, "output statistics info every interval and output metrics info at the end") + interval = flag.Duration("interval", time.Second, "interval to output the statistics") + caPath = flag.String("cacert", "", "path of file that contains list of trusted SSL CAs") + certPath = flag.String("cert", "", "path of file that contains X509 certificate in PEM format") + keyPath = flag.String("key", "", "path of file that contains X509 key in PEM format") + maxBatchWaitInterval = flag.Duration("batch-interval", 0, "the max batch wait interval") + enableTSOFollowerProxy = flag.Bool("enable-tso-follower-proxy", false, "whether enable the TSO Follower Proxy") + enableFaultInjection = flag.Bool("enable-fault-injection", false, "whether enable fault injection") + faultInjectionRate = flag.Float64("fault-injection-rate", 0.01, "the failure rate [0.0001, 1]. 0.01 means 1% failure rate") + maxTSOSendIntervalMilliseconds = flag.Int("max-send-interval-ms", 0, "max tso send interval in milliseconds, 60s by default") + keyspaceID = flag.Uint("keyspace-id", 0, "the id of the keyspace to access") + keyspaceName = flag.String("keyspace-name", "", "the name of the keyspace to access") + wg sync.WaitGroup ) var promServer *httptest.Server @@ -97,26 +103,7 @@ func bench(mainCtx context.Context) { fmt.Printf("Create %d client(s) for benchmark\n", *clientNumber) pdClients := make([]pd.Client, *clientNumber) for idx := range pdClients { - var ( - pdCli pd.Client - err error - ) - - opt := pd.WithGRPCDialOptions( - grpc.WithKeepaliveParams(keepalive.ClientParameters{ - Time: keepaliveTime, - Timeout: keepaliveTimeout, - }), - ) - - pdCli, err = pd.NewClientWithContext(mainCtx, []string{*pdAddrs}, pd.SecurityOption{ - CAPath: *caPath, - CertPath: *certPath, - KeyPath: *keyPath, - }, opt) - - pdCli.UpdateOption(pd.MaxTSOBatchWaitInterval, *maxBatchWaitInterval) - pdCli.UpdateOption(pd.EnableTSOFollowerProxy, *enableTSOFollowerProxy) + pdCli, err := createPDClient(mainCtx) if err != nil { log.Fatal(fmt.Sprintf("create pd client #%d failed: %v", idx, err)) } @@ -134,10 +121,18 @@ func bench(mainCtx context.Context) { durCh := make(chan time.Duration, 2*(*concurrency)*(*clientNumber)) - wg.Add((*concurrency) * (*clientNumber)) - for _, pdCli := range pdClients { - for i := 0; i < *concurrency; i++ { - go reqWorker(ctx, pdCli, durCh) + if *enableFaultInjection { + fmt.Printf("Enable fault injection, failure rate: %f\n", *faultInjectionRate) + wg.Add(*clientNumber) + for i := 0; i < *clientNumber; i++ { + go reqWorker(ctx, pdClients, i, durCh) + } + } else { + wg.Add((*concurrency) * (*clientNumber)) + for i := 0; i < *clientNumber; i++ { + for j := 0; j < *concurrency; j++ { + go reqWorker(ctx, pdClients, i, durCh) + } } } @@ -352,22 +347,51 @@ func (s *stats) calculate(count int) float64 { return float64(count) * 100 / float64(s.count) } -func reqWorker(ctx context.Context, pdCli pd.Client, durCh chan time.Duration) { +func reqWorker(ctx context.Context, pdClients []pd.Client, clientIdx int, durCh chan time.Duration) { defer wg.Done() reqCtx, cancel := context.WithCancel(ctx) defer cancel() + var ( + err error + maxRetryTime int = 120 + sleepIntervalOnFailure time.Duration = 1000 * time.Millisecond + totalSleepBeforeGetTS time.Duration + ) + pdCli := pdClients[clientIdx] for { + if pdCli == nil || (*enableFaultInjection && shouldInjectFault()) { + if pdCli != nil { + pdCli.Close() + } + pdCli, err = createPDClient(ctx) + if err != nil { + log.Error(fmt.Sprintf("re-create pd client #%d failed: %v", clientIdx, err)) + select { + case <-reqCtx.Done(): + case <-time.After(100 * time.Millisecond): + } + continue + } + pdClients[clientIdx] = pdCli + } + + totalSleepBeforeGetTS = 0 start := time.Now() - var ( - i int32 - err error - maxRetryTime int32 = 50 - sleepIntervalOnFailure time.Duration = 100 * time.Millisecond - ) + i := 0 for ; i < maxRetryTime; i++ { + if *maxTSOSendIntervalMilliseconds > 0 { + sleepBeforeGetTS := time.Duration(rand.Intn(*maxTSOSendIntervalMilliseconds)) * time.Millisecond + ticker := time.NewTicker(sleepBeforeGetTS) + defer ticker.Stop() + select { + case <-reqCtx.Done(): + case <-ticker.C: + totalSleepBeforeGetTS += sleepBeforeGetTS + } + } _, _, err = pdCli.GetLocalTS(reqCtx, *dcLocation) if errors.Cause(err) == context.Canceled { return @@ -381,7 +405,7 @@ func reqWorker(ctx context.Context, pdCli pd.Client, durCh chan time.Duration) { if err != nil { log.Fatal(fmt.Sprintf("%v", err)) } - dur := time.Since(start) - time.Duration(i)*sleepIntervalOnFailure + dur := time.Since(start) - time.Duration(i)*sleepIntervalOnFailure - totalSleepBeforeGetTS select { case <-reqCtx.Done(): @@ -390,3 +414,44 @@ func reqWorker(ctx context.Context, pdCli pd.Client, durCh chan time.Duration) { } } } + +func createPDClient(ctx context.Context) (pd.Client, error) { + var ( + pdCli pd.Client + err error + ) + + opts := make([]pd.ClientOption, 0) + opts = append(opts, pd.WithGRPCDialOptions( + grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: keepaliveTime, + Timeout: keepaliveTimeout, + }), + )) + + if len(*keyspaceName) > 0 { + apiCtx := pd.NewAPIContextV2(*keyspaceName) + pdCli, err = pd.NewClientWithAPIContext(ctx, apiCtx, []string{*pdAddrs}, pd.SecurityOption{ + CAPath: *caPath, + CertPath: *certPath, + KeyPath: *keyPath, + }, opts...) + } else { + pdCli, err = pd.NewClientWithKeyspace(ctx, uint32(*keyspaceID), []string{*pdAddrs}, pd.SecurityOption{ + CAPath: *caPath, + CertPath: *certPath, + KeyPath: *keyPath, + }, opts...) + } + if err != nil { + return nil, err + } + + pdCli.UpdateOption(pd.MaxTSOBatchWaitInterval, *maxBatchWaitInterval) + pdCli.UpdateOption(pd.EnableTSOFollowerProxy, *enableTSOFollowerProxy) + return pdCli, err +} + +func shouldInjectFault() bool { + return rand.Intn(10000) < int(*faultInjectionRate*10000) +} From 88aec38a08549b75831b3e7f534a62ae68c5d542 Mon Sep 17 00:00:00 2001 From: lhy1024 Date: Wed, 5 Jul 2023 15:54:43 +0800 Subject: [PATCH 10/15] *: remove full-width symbols (#6753) ref tikv/pd#4399 Signed-off-by: lhy1024 Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- Makefile | 2 +- client/resource_group/controller/controller.go | 2 +- client/resource_group/controller/limiter.go | 2 +- pkg/cgroup/cgroup.go | 4 ++-- pkg/core/rangetree/range_tree_test.go | 2 +- pkg/core/store_test.go | 2 +- pkg/schedule/checker/replica_checker_test.go | 2 +- pkg/schedule/labeler/rules.go | 2 +- pkg/schedule/operator/operator.go | 2 +- pkg/schedule/operator/operator_controller.go | 2 +- pkg/schedule/placement/fit_region_test.go | 4 ++-- pkg/statistics/buckets/hot_bucket_task_test.go | 6 +++--- server/config/store_config_test.go | 6 +++--- tests/server/storage/hot_region_storage_test.go | 4 ++-- 14 files changed, 21 insertions(+), 21 deletions(-) diff --git a/Makefile b/Makefile index f6879cf20d1..f412c998c69 100644 --- a/Makefile +++ b/Makefile @@ -242,7 +242,7 @@ test-tso-consistency: install-tools TASK_COUNT=1 TASK_ID=1 -# The command should be used in daily CI,it will split some tasks to run parallel. +# The command should be used in daily CI, it will split some tasks to run parallel. # It should retain report.xml,coverage,coverage.xml and package.list to analyze. test-with-cover-parallel: install-tools dashboard-ui split @$(FAILPOINT_ENABLE) diff --git a/client/resource_group/controller/controller.go b/client/resource_group/controller/controller.go index 873a9a33899..cc4595f7e42 100755 --- a/client/resource_group/controller/controller.go +++ b/client/resource_group/controller/controller.go @@ -60,7 +60,7 @@ type ResourceGroupKVInterceptor interface { OnResponse(resourceGroupName string, req RequestInfo, resp ResponseInfo) (*rmpb.Consumption, error) } -// ResourceGroupProvider provides some api to interact with resource manager server。 +// ResourceGroupProvider provides some api to interact with resource manager server. type ResourceGroupProvider interface { GetResourceGroup(ctx context.Context, resourceGroupName string) (*rmpb.ResourceGroup, error) AddResourceGroup(ctx context.Context, metaGroup *rmpb.ResourceGroup) (string, error) diff --git a/client/resource_group/controller/limiter.go b/client/resource_group/controller/limiter.go index 078894dbdec..a8038c6d8e5 100644 --- a/client/resource_group/controller/limiter.go +++ b/client/resource_group/controller/limiter.go @@ -185,7 +185,7 @@ func (r *Reservation) CancelAt(now time.Time) { // Reserve returns a Reservation that indicates how long the caller must wait before n events happen. // The Limiter takes this Reservation into account when allowing future events. -// The returned Reservation’s OK() method returns false if wait duration exceeds deadline. +// The returned Reservation's OK() method returns false if wait duration exceeds deadline. // Usage example: // // r := lim.Reserve(time.Now(), 1) diff --git a/pkg/cgroup/cgroup.go b/pkg/cgroup/cgroup.go index 1406cffed0a..2a99d2fcd3d 100644 --- a/pkg/cgroup/cgroup.go +++ b/pkg/cgroup/cgroup.go @@ -234,10 +234,10 @@ func getCgroupDetails(mountInfoPath string, cRoot string, controller string) (mo // It is possible that the controller mount and the cgroup path are not the same (both are relative to the NS root). // So start with the mount and construct the relative path of the cgroup. // To test: - // 1、start a docker to run unit test or tidb-server + // 1. start a docker to run unit test or tidb-server // > docker run -it --cpus=8 --memory=8g --name test --rm ubuntu:18.04 bash // - // 2、change the limit when the container is running + // 2. change the limit when the container is running // docker update --cpus=8 nsRelativePath := string(fields[3]) if !strings.Contains(nsRelativePath, "..") { diff --git a/pkg/core/rangetree/range_tree_test.go b/pkg/core/rangetree/range_tree_test.go index 6bbd6860989..29845cf0bca 100644 --- a/pkg/core/rangetree/range_tree_test.go +++ b/pkg/core/rangetree/range_tree_test.go @@ -100,7 +100,7 @@ func TestRingPutItem(t *testing.T) { re.Len(bucketTree.GetOverlaps(newSimpleBucketItem([]byte("010"), []byte("110"))), 2) re.Empty(bucketTree.GetOverlaps(newSimpleBucketItem([]byte("200"), []byte("300")))) - // test1: insert one key range, the old overlaps will retain like split buckets. + // test1: insert one key range, the old overlaps will retain like split buckets. // key range: [002,010],[010,090],[090,100],[100,200] bucketTree.Update(newSimpleBucketItem([]byte("010"), []byte("090"))) re.Equal(4, bucketTree.Len()) diff --git a/pkg/core/store_test.go b/pkg/core/store_test.go index f6f2518c241..be0fd0f9418 100644 --- a/pkg/core/store_test.go +++ b/pkg/core/store_test.go @@ -174,7 +174,7 @@ func TestLowSpaceScoreV2(t *testing.T) { bigger: newStoreInfoWithAvailable(1, 10*units.GiB, 100*units.GiB, 1.5), small: newStoreInfoWithAvailable(2, 10*units.GiB, 100*units.GiB, 1.4), }, { - // store1 and store2 has same capacity and regionSize(40g) + // store1 and store2 has same capacity and regionSize (40g) // but store1 has less available space size bigger: newStoreInfoWithAvailable(1, 60*units.GiB, 100*units.GiB, 1), small: newStoreInfoWithAvailable(2, 80*units.GiB, 100*units.GiB, 2), diff --git a/pkg/schedule/checker/replica_checker_test.go b/pkg/schedule/checker/replica_checker_test.go index 8975b8ec578..2d0961543e5 100644 --- a/pkg/schedule/checker/replica_checker_test.go +++ b/pkg/schedule/checker/replica_checker_test.go @@ -162,7 +162,7 @@ func (suite *replicaCheckerTestSuite) TestDownPeer() { suite.NotNil(op) suite.Equal("remove-extra-down-replica", op.Desc()) - // down a peer,the number of peers(except learner) is not enough. + // down a peer, the number of peers(except learner) is not enough. op = suite.downPeerAndCheck(metapb.PeerRole_Learner) suite.NotNil(op) suite.Equal("replace-down-replica", op.Desc()) diff --git a/pkg/schedule/labeler/rules.go b/pkg/schedule/labeler/rules.go index c902fff8f66..3b50779d659 100644 --- a/pkg/schedule/labeler/rules.go +++ b/pkg/schedule/labeler/rules.go @@ -171,7 +171,7 @@ func (rule *LabelRule) expireBefore(t time.Time) bool { return rule.minExpire.Before(t) } -// initKeyRangeRulesFromLabelRuleData init and adjust []KeyRangeRule from `LabelRule.Data“ +// initKeyRangeRulesFromLabelRuleData init and adjust []KeyRangeRule from `LabelRule.Data` func initKeyRangeRulesFromLabelRuleData(data interface{}) ([]*KeyRangeRule, error) { rules, ok := data.([]interface{}) if !ok { diff --git a/pkg/schedule/operator/operator.go b/pkg/schedule/operator/operator.go index f8df4a428dd..d841c785b59 100644 --- a/pkg/schedule/operator/operator.go +++ b/pkg/schedule/operator/operator.go @@ -129,7 +129,7 @@ func (o *Operator) String() string { for i := range o.steps { stepStrs[i] = o.steps[i].String() } - s := fmt.Sprintf("%s {%s} (kind:%s, region:%v(%v, %v), createAt:%s, startAt:%s, currentStep:%v, size:%d, steps:[%s],timeout:[%s])", + s := fmt.Sprintf("%s {%s} (kind:%s, region:%v(%v, %v), createAt:%s, startAt:%s, currentStep:%v, size:%d, steps:[%s], timeout:[%s])", o.desc, o.brief, o.kind, o.regionID, o.regionEpoch.GetVersion(), o.regionEpoch.GetConfVer(), o.GetCreateTime(), o.GetStartTime(), atomic.LoadInt32(&o.currentStep), o.ApproximateSize, strings.Join(stepStrs, ", "), o.timeout.String()) if o.CheckSuccess() { diff --git a/pkg/schedule/operator/operator_controller.go b/pkg/schedule/operator/operator_controller.go index 4e1ff6e5de2..c82b2228c9d 100644 --- a/pkg/schedule/operator/operator_controller.go +++ b/pkg/schedule/operator/operator_controller.go @@ -48,7 +48,7 @@ var ( fastNotifyInterval = 2 * time.Second // StoreBalanceBaseTime represents the base time of balance rate. StoreBalanceBaseTime float64 = 60 - // FastOperatorFinishTime min finish time, if finish duration less than it,op will be pushed to fast operator queue + // FastOperatorFinishTime min finish time, if finish duration less than it, op will be pushed to fast operator queue FastOperatorFinishTime = 10 * time.Second ) diff --git a/pkg/schedule/placement/fit_region_test.go b/pkg/schedule/placement/fit_region_test.go index 55b0e76a2d2..0ec67b2a2aa 100644 --- a/pkg/schedule/placement/fit_region_test.go +++ b/pkg/schedule/placement/fit_region_test.go @@ -326,7 +326,7 @@ func BenchmarkFitRegionWithMoreRulesAndStoreLabels(b *testing.B) { label := &metapb.StoreLabel{Key: "exclusive", Value: "exclusive"} labels = append(labels, label) // 5 peers in 5 different stores, - // split the stores(peers) to three zones,make the number of peers in each zone: 2:2:1 + // split the stores(peers) to three zones, make the number of peers in each zone: 2:2:1 for _, peer := range region.GetPeers() { storeID := peer.StoreId store := core.NewStoreInfo(&metapb.Store{Id: storeID}, core.SetLastHeartbeatTS(time.Now()), core.SetStoreLabels(labels)) @@ -380,7 +380,7 @@ func BenchmarkFitRegionWithLocationLabels(b *testing.B) { // create stores lists := make([]*core.StoreInfo, 0) // 10 peers in 10 different stores, - // split the stores(peers) to three zones,make the number of peers in each zone: 4:3:3 + // split the stores(peers) to three zones, make the number of peers in each zone: 4:3:3 for idx, peer := range region.GetPeers() { storeID := peer.StoreId zoneInfo := &metapb.StoreLabel{Key: "zone", Value: fmt.Sprintf("z_%02d", idx%3)} diff --git a/pkg/statistics/buckets/hot_bucket_task_test.go b/pkg/statistics/buckets/hot_bucket_task_test.go index f2f28ef3d02..49741fdc83f 100644 --- a/pkg/statistics/buckets/hot_bucket_task_test.go +++ b/pkg/statistics/buckets/hot_bucket_task_test.go @@ -68,7 +68,7 @@ func TestCheckBucketsTask(t *testing.T) { ctx, cancelFn := context.WithCancel(context.Background()) defer cancelFn() hotCache := NewBucketsCache(ctx) - // case1: add bucket successfully + // case1: add bucket successfully buckets := newTestBuckets(1, 1, [][]byte{[]byte("10"), []byte("20"), []byte("30")}, 0) task := NewCheckPeerTask(buckets) re.True(hotCache.CheckAsync(task)) @@ -93,7 +93,7 @@ func TestCheckBucketsTask(t *testing.T) { re.Len(item, 1) re.Equal(-2, item[0].HotDegree) - // case3:add bucket successful and the hot degree should inherit from the old one. + // case3: add bucket successful and the hot degree should inherit from the old one. buckets = newTestBuckets(1, 1, [][]byte{[]byte("10"), []byte("20")}, 0) task = NewCheckPeerTask(buckets) re.True(hotCache.CheckAsync(task)) @@ -109,7 +109,7 @@ func TestCollectBucketStatsTask(t *testing.T) { ctx, cancelFn := context.WithCancel(context.Background()) defer cancelFn() hotCache := NewBucketsCache(ctx) - // case1: add bucket successfully + // case1: add bucket successfully for i := uint64(0); i < 10; i++ { buckets := convertToBucketTreeItem(newTestBuckets(i, 1, [][]byte{[]byte(strconv.FormatUint(i*10, 10)), []byte(strconv.FormatUint((i+1)*10, 10))}, 0)) diff --git a/server/config/store_config_test.go b/server/config/store_config_test.go index 6342926b636..7cc9119f4d5 100644 --- a/server/config/store_config_test.go +++ b/server/config/store_config_test.go @@ -133,21 +133,21 @@ func TestMergeCheck(t *testing.T) { mergeKeys: 200000, pass: true, }, { - // case 2: the smallest region is 68MiB,it can't be merged again. + // case 2: the smallest region is 68MiB, it can't be merged again. size: 144 + 20, mergeSize: 20, keys: 1440000 + 200000, mergeKeys: 200000, pass: true, }, { - // case 3: the smallest region is 50MiB,it can be merged again. + // case 3: the smallest region is 50MiB, it can be merged again. size: 144 + 2, mergeSize: 50, keys: 1440000 + 20000, mergeKeys: 500000, pass: false, }, { - // case4: the smallest region is 51MiB,it can't be merged again. + // case4: the smallest region is 51MiB, it can't be merged again. size: 144 + 3, mergeSize: 50, keys: 1440000 + 30000, diff --git a/tests/server/storage/hot_region_storage_test.go b/tests/server/storage/hot_region_storage_test.go index 44b1cfd274a..fee4944826c 100644 --- a/tests/server/storage/hot_region_storage_test.go +++ b/tests/server/storage/hot_region_storage_test.go @@ -192,7 +192,7 @@ func TestHotRegionStorageReservedDayConfigChange(t *testing.T) { re.NoError(err) re.Nil(next) schedule := leaderServer.GetConfig().Schedule - // set reserved day to zero,close hot region storage + // set reserved day to zero, close hot region storage schedule.HotRegionsReservedDays = 0 leaderServer.GetServer().SetScheduleConfig(schedule) time.Sleep(3 * interval) @@ -210,7 +210,7 @@ func TestHotRegionStorageReservedDayConfigChange(t *testing.T) { next, err = iter.Next() re.NoError(err) re.Nil(next) - // set reserved day to one,open hot region storage + // set reserved day to one, open hot region storage schedule.HotRegionsReservedDays = 1 leaderServer.GetServer().SetScheduleConfig(schedule) time.Sleep(3 * interval) From ce5010df8969a9953346312c976ddbb9825568b4 Mon Sep 17 00:00:00 2001 From: Hu# Date: Wed, 5 Jul 2023 18:33:43 +0800 Subject: [PATCH 11/15] dashboard: update version to fix some bugs (#6759) ref tikv/pd#6415 Signed-off-by: husharp Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- tests/integrations/client/go.mod | 2 +- tests/integrations/client/go.sum | 4 ++-- tests/integrations/mcs/go.mod | 2 +- tests/integrations/mcs/go.sum | 4 ++-- tests/integrations/tso/go.mod | 2 +- tests/integrations/tso/go.sum | 4 ++-- 8 files changed, 12 insertions(+), 12 deletions(-) diff --git a/go.mod b/go.mod index 0dccd506aad..34b038b39f4 100644 --- a/go.mod +++ b/go.mod @@ -33,7 +33,7 @@ require ( github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 - github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 + github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27 github.com/prometheus/client_golang v1.11.1 github.com/prometheus/common v0.26.0 github.com/sasha-s/go-deadlock v0.2.0 diff --git a/go.sum b/go.sum index 2962b6ca897..90974271faa 100644 --- a/go.sum +++ b/go.sum @@ -431,8 +431,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 h1:/jqjj+ydlqjp144LAlHDfHtr7eWJyaNIIXX5viv0RZo= -github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= +github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27 h1:K9lZMYuDuAiR5kOjFESwJ8KfSb4ui5zX6vZGbUp58uk= +github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= diff --git a/tests/integrations/client/go.mod b/tests/integrations/client/go.mod index 00c6bd4daaf..32da542d3e5 100644 --- a/tests/integrations/client/go.mod +++ b/tests/integrations/client/go.mod @@ -119,7 +119,7 @@ require ( github.com/pingcap/errcode v0.3.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27 // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/tests/integrations/client/go.sum b/tests/integrations/client/go.sum index 0e51f7e6dec..b3e41e4e2c6 100644 --- a/tests/integrations/client/go.sum +++ b/tests/integrations/client/go.sum @@ -395,8 +395,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 h1:/jqjj+ydlqjp144LAlHDfHtr7eWJyaNIIXX5viv0RZo= -github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= +github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27 h1:K9lZMYuDuAiR5kOjFESwJ8KfSb4ui5zX6vZGbUp58uk= +github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= diff --git a/tests/integrations/mcs/go.mod b/tests/integrations/mcs/go.mod index abcfe6f615e..5520e15d5f0 100644 --- a/tests/integrations/mcs/go.mod +++ b/tests/integrations/mcs/go.mod @@ -119,7 +119,7 @@ require ( github.com/pingcap/errcode v0.3.0 // indirect github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27 // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/tests/integrations/mcs/go.sum b/tests/integrations/mcs/go.sum index 74bcbaff0da..da09322fc85 100644 --- a/tests/integrations/mcs/go.sum +++ b/tests/integrations/mcs/go.sum @@ -395,8 +395,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 h1:/jqjj+ydlqjp144LAlHDfHtr7eWJyaNIIXX5viv0RZo= -github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= +github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27 h1:K9lZMYuDuAiR5kOjFESwJ8KfSb4ui5zX6vZGbUp58uk= +github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= diff --git a/tests/integrations/tso/go.mod b/tests/integrations/tso/go.mod index 9b82706d7a4..8354a4ddd38 100644 --- a/tests/integrations/tso/go.mod +++ b/tests/integrations/tso/go.mod @@ -117,7 +117,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c // indirect github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 // indirect github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 // indirect - github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 // indirect + github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27 // indirect github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/tests/integrations/tso/go.sum b/tests/integrations/tso/go.sum index cdffbdfe8df..f5e6a47b40d 100644 --- a/tests/integrations/tso/go.sum +++ b/tests/integrations/tso/go.sum @@ -392,8 +392,8 @@ github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 h1:QV6jqlfOkh8hqvEAgwBZa+4bSgO0EeKC7s5c6Luam2I= github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21/go.mod h1:QYnjfA95ZaMefyl1NO8oPtKeb8pYUdnDVhQgf+qdpjM= -github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45 h1:/jqjj+ydlqjp144LAlHDfHtr7eWJyaNIIXX5viv0RZo= -github.com/pingcap/tidb-dashboard v0.0.0-20230626093106-fcc40851da45/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= +github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27 h1:K9lZMYuDuAiR5kOjFESwJ8KfSb4ui5zX6vZGbUp58uk= +github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27/go.mod h1:OUzFMMVjR1GKlf4LWLqza9QNKjCrYJ7stVn/3PN0djM= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e h1:FBaTXU8C3xgt/drM58VHxojHo/QoG1oPsgWTGvaSpO4= github.com/pingcap/tipb v0.0.0-20220718022156-3e2483c20a9e/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= From e9c6f33f5931cfe4c3e04c88bd127dc1a5a017b7 Mon Sep 17 00:00:00 2001 From: ShuNing Date: Wed, 5 Jul 2023 20:12:43 +0800 Subject: [PATCH 12/15] *: add background settings in group manager (#6742) close tikv/pd#6741 Signed-off-by: nolouch Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com> --- client/go.mod | 2 +- client/go.sum | 4 ++-- go.mod | 2 +- go.sum | 4 ++-- .../resourcemanager/server/resource_group.go | 20 +++++++++++-------- tests/integrations/client/go.mod | 2 +- tests/integrations/client/go.sum | 4 ++-- tests/integrations/mcs/go.mod | 2 +- tests/integrations/mcs/go.sum | 4 ++-- .../resourcemanager/resource_manager_test.go | 10 ++++++++-- tests/integrations/tso/go.mod | 2 +- tests/integrations/tso/go.sum | 4 ++-- tools/pd-tso-bench/go.sum | 4 ++-- 13 files changed, 37 insertions(+), 27 deletions(-) diff --git a/client/go.mod b/client/go.mod index 2fc40b88c55..9f935e03a33 100644 --- a/client/go.mod +++ b/client/go.mod @@ -8,7 +8,7 @@ require ( github.com/opentracing/opentracing-go v1.2.0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 + github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/prometheus/client_golang v1.11.1 github.com/stretchr/testify v1.8.2 diff --git a/client/go.sum b/client/go.sum index 9e512fc6d03..2aec7f63589 100644 --- a/client/go.sum +++ b/client/go.sum @@ -86,8 +86,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTm github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 h1:GBlml2UIrI9IR3DdBnUWNeXizK4PwJhYPO7eWgCNErg= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/go.mod b/go.mod index 34b038b39f4..5db9cd513ab 100644 --- a/go.mod +++ b/go.mod @@ -30,7 +30,7 @@ require ( github.com/pingcap/errcode v0.3.0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 + github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27 diff --git a/go.sum b/go.sum index 90974271faa..3f08eb5b07d 100644 --- a/go.sum +++ b/go.sum @@ -424,8 +424,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 h1:GBlml2UIrI9IR3DdBnUWNeXizK4PwJhYPO7eWgCNErg= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= diff --git a/pkg/mcs/resourcemanager/server/resource_group.go b/pkg/mcs/resourcemanager/server/resource_group.go index d20e8c6b717..8c743ae9ba4 100644 --- a/pkg/mcs/resourcemanager/server/resource_group.go +++ b/pkg/mcs/resourcemanager/server/resource_group.go @@ -33,9 +33,10 @@ type ResourceGroup struct { Name string `json:"name"` Mode rmpb.GroupMode `json:"mode"` // RU settings - RUSettings *RequestUnitSettings `json:"r_u_settings,omitempty"` - Priority uint32 `json:"priority"` - Runaway *rmpb.RunawaySettings `json:"runaway_settings,omitempty"` + RUSettings *RequestUnitSettings `json:"r_u_settings,omitempty"` + Priority uint32 `json:"priority"` + Runaway *rmpb.RunawaySettings `json:"runaway_settings,omitempty"` + Background *rmpb.BackgroundSettings `json:"background_settings,omitempty"` } // RequestUnitSettings is the definition of the RU settings. @@ -97,6 +98,7 @@ func (rg *ResourceGroup) PatchSettings(metaGroup *rmpb.ResourceGroup) error { } rg.Priority = metaGroup.Priority rg.Runaway = metaGroup.RunawaySettings + rg.Background = metaGroup.BackgroundSettings switch rg.Mode { case rmpb.GroupMode_RUMode: settings := metaGroup.GetRUSettings() @@ -115,10 +117,11 @@ func (rg *ResourceGroup) PatchSettings(metaGroup *rmpb.ResourceGroup) error { // FromProtoResourceGroup converts a rmpb.ResourceGroup to a ResourceGroup. func FromProtoResourceGroup(group *rmpb.ResourceGroup) *ResourceGroup { rg := &ResourceGroup{ - Name: group.Name, - Mode: group.Mode, - Priority: group.Priority, - Runaway: group.RunawaySettings, + Name: group.Name, + Mode: group.Mode, + Priority: group.Priority, + Runaway: group.RunawaySettings, + Background: group.BackgroundSettings, } switch group.GetMode() { case rmpb.GroupMode_RUMode: @@ -163,7 +166,8 @@ func (rg *ResourceGroup) IntoProtoResourceGroup() *rmpb.ResourceGroup { RUSettings: &rmpb.GroupRequestUnitSettings{ RU: rg.RUSettings.RU.GetTokenBucket(), }, - RunawaySettings: rg.Runaway, + RunawaySettings: rg.Runaway, + BackgroundSettings: rg.Background, } return group case rmpb.GroupMode_RawMode: // Raw mode diff --git a/tests/integrations/client/go.mod b/tests/integrations/client/go.mod index 32da542d3e5..6211297c389 100644 --- a/tests/integrations/client/go.mod +++ b/tests/integrations/client/go.mod @@ -13,7 +13,7 @@ replace google.golang.org/grpc v1.54.0 => google.golang.org/grpc v1.26.0 require ( github.com/docker/go-units v0.4.0 github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 + github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/stretchr/testify v1.8.2 github.com/tikv/pd v0.0.0-00010101000000-000000000000 diff --git a/tests/integrations/client/go.sum b/tests/integrations/client/go.sum index b3e41e4e2c6..208cbc31537 100644 --- a/tests/integrations/client/go.sum +++ b/tests/integrations/client/go.sum @@ -387,8 +387,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 h1:GBlml2UIrI9IR3DdBnUWNeXizK4PwJhYPO7eWgCNErg= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= diff --git a/tests/integrations/mcs/go.mod b/tests/integrations/mcs/go.mod index 5520e15d5f0..f421e682b6d 100644 --- a/tests/integrations/mcs/go.mod +++ b/tests/integrations/mcs/go.mod @@ -12,7 +12,7 @@ replace google.golang.org/grpc v1.54.0 => google.golang.org/grpc v1.26.0 require ( github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 + github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/stretchr/testify v1.8.2 github.com/tikv/pd v0.0.0-00010101000000-000000000000 diff --git a/tests/integrations/mcs/go.sum b/tests/integrations/mcs/go.sum index da09322fc85..d95617229b7 100644 --- a/tests/integrations/mcs/go.sum +++ b/tests/integrations/mcs/go.sum @@ -388,8 +388,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 h1:GBlml2UIrI9IR3DdBnUWNeXizK4PwJhYPO7eWgCNErg= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= diff --git a/tests/integrations/mcs/resourcemanager/resource_manager_test.go b/tests/integrations/mcs/resourcemanager/resource_manager_test.go index 5cbed81d57e..064d0bfe80e 100644 --- a/tests/integrations/mcs/resourcemanager/resource_manager_test.go +++ b/tests/integrations/mcs/resourcemanager/resource_manager_test.go @@ -730,7 +730,7 @@ func (suite *resourceManagerClientTestSuite) TestBasicResourceGroupCURD() { }, {"test2", rmpb.GroupMode_RUMode, true, true, - `{"name":"test2","mode":1,"r_u_settings":{"r_u":{"settings":{"fill_rate":20000},"state":{"initialized":false}}},"priority":0,"runaway_settings":{"rule":{"exec_elapsed_time_ms":10000},"action":1}}`, + `{"name":"test2","mode":1,"r_u_settings":{"r_u":{"settings":{"fill_rate":20000},"state":{"initialized":false}}},"priority":0,"runaway_settings":{"rule":{"exec_elapsed_time_ms":10000},"action":1},"background_settings":{"job_types":["test"]}}`, func(gs *rmpb.ResourceGroup) { gs.RUSettings = &rmpb.GroupRequestUnitSettings{ RU: &rmpb.TokenBucket{ @@ -745,10 +745,13 @@ func (suite *resourceManagerClientTestSuite) TestBasicResourceGroupCURD() { }, Action: rmpb.RunawayAction_CoolDown, } + gs.BackgroundSettings = &rmpb.BackgroundSettings{ + JobTypes: []string{"test"}, + } }, }, {"test2", rmpb.GroupMode_RUMode, false, true, - `{"name":"test2","mode":1,"r_u_settings":{"r_u":{"settings":{"fill_rate":30000,"burst_limit":-1},"state":{"initialized":false}}},"priority":0,"runaway_settings":{"rule":{"exec_elapsed_time_ms":1000},"action":2,"watch":{"lasting_duration_ms":100000,"type":1}}}`, + `{"name":"test2","mode":1,"r_u_settings":{"r_u":{"settings":{"fill_rate":30000,"burst_limit":-1},"state":{"initialized":false}}},"priority":0,"runaway_settings":{"rule":{"exec_elapsed_time_ms":1000},"action":2,"watch":{"lasting_duration_ms":100000,"type":1}},"background_settings":{"job_types":["br","lightning"]}}`, func(gs *rmpb.ResourceGroup) { gs.RUSettings = &rmpb.GroupRequestUnitSettings{ RU: &rmpb.TokenBucket{ @@ -768,6 +771,9 @@ func (suite *resourceManagerClientTestSuite) TestBasicResourceGroupCURD() { LastingDurationMs: 100000, }, } + gs.BackgroundSettings = &rmpb.BackgroundSettings{ + JobTypes: []string{"br", "lightning"}, + } }, }, } diff --git a/tests/integrations/tso/go.mod b/tests/integrations/tso/go.mod index 8354a4ddd38..95035400c43 100644 --- a/tests/integrations/tso/go.mod +++ b/tests/integrations/tso/go.mod @@ -13,7 +13,7 @@ replace google.golang.org/grpc v1.54.0 => google.golang.org/grpc v1.26.0 require ( github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c - github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 + github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 github.com/stretchr/testify v1.8.4 github.com/tikv/pd v0.0.0-00010101000000-000000000000 github.com/tikv/pd/client v0.0.0-00010101000000-000000000000 diff --git a/tests/integrations/tso/go.sum b/tests/integrations/tso/go.sum index f5e6a47b40d..449141101bb 100644 --- a/tests/integrations/tso/go.sum +++ b/tests/integrations/tso/go.sum @@ -385,8 +385,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 h1:GBlml2UIrI9IR3DdBnUWNeXizK4PwJhYPO7eWgCNErg= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= diff --git a/tools/pd-tso-bench/go.sum b/tools/pd-tso-bench/go.sum index b7ca93b1272..6a6696c3133 100644 --- a/tools/pd-tso-bench/go.sum +++ b/tools/pd-tso-bench/go.sum @@ -864,8 +864,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTm github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46 h1:GBlml2UIrI9IR3DdBnUWNeXizK4PwJhYPO7eWgCNErg= -github.com/pingcap/kvproto v0.0.0-20230530111525-e4919c190b46/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= +github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= From b8190c5bc7381140a1b6a611b49c86c7fa7f6f2b Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Thu, 6 Jul 2023 10:23:14 +0800 Subject: [PATCH 13/15] *: make test great again (#6767) close tikv/pd#6761 Signed-off-by: Ryan Leung --- tests/integrations/mcs/tso/keyspace_group_manager_test.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integrations/mcs/tso/keyspace_group_manager_test.go b/tests/integrations/mcs/tso/keyspace_group_manager_test.go index 52974e3155f..0f7c916555d 100644 --- a/tests/integrations/mcs/tso/keyspace_group_manager_test.go +++ b/tests/integrations/mcs/tso/keyspace_group_manager_test.go @@ -738,10 +738,8 @@ func TestGetTSOImmediately(t *testing.T) { kgm.SetPriorityForKeyspaceGroup(0, kg0.Members[0].Address, 100) kgm.SetPriorityForKeyspaceGroup(1, kg1.Members[1].Address, 100) testutil.Eventually(re, func() bool { - p0, err := kgm.GetKeyspaceGroupPrimaryByID(0) - re.NoError(err) - p1, err := kgm.GetKeyspaceGroupPrimaryByID(1) - re.NoError(err) + p0, _ := kgm.GetKeyspaceGroupPrimaryByID(0) + p1, _ := kgm.GetKeyspaceGroupPrimaryByID(1) return p0 == kg0.Members[0].Address && p1 == kg1.Members[1].Address }, testutil.WithWaitFor(5*time.Second), testutil.WithTickInterval(50*time.Millisecond)) From b9adc835b0568afb793f1257cc72dc3c5bd90a0d Mon Sep 17 00:00:00 2001 From: JmPotato Date: Thu, 6 Jul 2023 14:02:14 +0800 Subject: [PATCH 14/15] tso: implement deletedGroupCleaner to clean up the legacy TSO key (#6745) close tikv/pd#6589 - Implement `deletedGroupCleaner` to clean up the legacy TSO key. - Extract the timestamp key path constructor. Signed-off-by: JmPotato --- pkg/storage/endpoint/key_path.go | 23 ++++- pkg/storage/endpoint/tso.go | 10 ++- pkg/storage/storage_tso_test.go | 17 ++-- pkg/tso/allocator_manager.go | 26 ++---- pkg/tso/global_allocator.go | 3 +- pkg/tso/keyspace_group_manager.go | 103 ++++++++++++++++++---- pkg/tso/keyspace_group_manager_test.go | 53 +++++++++++ pkg/tso/tso.go | 4 +- tests/integrations/mcs/tso/server_test.go | 2 +- tools/pd-backup/pdbackup/backup.go | 4 +- tools/pd-backup/pdbackup/backup_test.go | 4 +- 11 files changed, 192 insertions(+), 57 deletions(-) diff --git a/pkg/storage/endpoint/key_path.go b/pkg/storage/endpoint/key_path.go index 2eca0e6bf17..86022b273f7 100644 --- a/pkg/storage/endpoint/key_path.go +++ b/pkg/storage/endpoint/key_path.go @@ -53,8 +53,10 @@ const ( resourceGroupStatesPath = "states" controllerConfigPath = "controller" // tso storage endpoint has prefix `tso` - tsoServiceKey = utils.TSOServiceName - timestampKey = "timestamp" + tsoServiceKey = utils.TSOServiceName + globalTSOAllocatorEtcdPrefix = "gta" + // TimestampKey is the key of timestamp oracle used for the suffix. + TimestampKey = "timestamp" tsoKeyspaceGroupPrefix = tsoServiceKey + "/" + utils.KeyspaceGroupsKey keyspaceGroupMembershipKey = "membership" @@ -261,3 +263,20 @@ func buildPath(withSuffix bool, str ...string) string { } return sb.String() } + +// GetKeyspaceGroupTSPath constructs the timestampOracle path prefix, which is: +// 1. for the default keyspace group: +// "" in /pd/{cluster_id}/timestamp +// 2. for the non-default keyspace groups: +// {group}/gta in /ms/{cluster_id}/tso/{group}/gta/timestamp +func GetKeyspaceGroupTSPath(groupID uint32) string { + if groupID == utils.DefaultKeyspaceGroupID { + return "" + } + return path.Join(fmt.Sprintf("%05d", groupID), globalTSOAllocatorEtcdPrefix) +} + +// GetTimestampPath returns the timestamp path for the given timestamp oracle path prefix. +func GetTimestampPath(tsPath string) string { + return path.Join(tsPath, TimestampKey) +} diff --git a/pkg/storage/endpoint/tso.go b/pkg/storage/endpoint/tso.go index e67f5d33f4f..f0aa9d8cde0 100644 --- a/pkg/storage/endpoint/tso.go +++ b/pkg/storage/endpoint/tso.go @@ -31,6 +31,7 @@ import ( type TSOStorage interface { LoadTimestamp(prefix string) (time.Time, error) SaveTimestamp(key string, ts time.Time) error + DeleteTimestamp(key string) error } var _ TSOStorage = (*StorageEndpoint)(nil) @@ -51,7 +52,7 @@ func (se *StorageEndpoint) LoadTimestamp(prefix string) (time.Time, error) { maxTSWindow := typeutil.ZeroTime for i, key := range keys { key := strings.TrimSpace(key) - if !strings.HasSuffix(key, timestampKey) { + if !strings.HasSuffix(key, TimestampKey) { continue } tsWindow, err := typeutil.ParseTimestamp([]byte(values[i])) @@ -89,3 +90,10 @@ func (se *StorageEndpoint) SaveTimestamp(key string, ts time.Time) error { return txn.Save(key, string(data)) }) } + +// DeleteTimestamp deletes the timestamp from the storage. +func (se *StorageEndpoint) DeleteTimestamp(key string) error { + return se.RunInTxn(context.Background(), func(txn kv.Txn) error { + return txn.Remove(key) + }) +} diff --git a/pkg/storage/storage_tso_test.go b/pkg/storage/storage_tso_test.go index b8f31cc377c..1dbba289512 100644 --- a/pkg/storage/storage_tso_test.go +++ b/pkg/storage/storage_tso_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/stretchr/testify/require" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" @@ -42,9 +43,8 @@ func TestSaveLoadTimestamp(t *testing.T) { rootPath := path.Join("/pd", strconv.FormatUint(100, 10)) storage := NewStorageWithEtcdBackend(client, rootPath) - key := "timestamp" expectedTS := time.Now().Round(0) - err = storage.SaveTimestamp(key, expectedTS) + err = storage.SaveTimestamp(endpoint.TimestampKey, expectedTS) re.NoError(err) ts, err := storage.LoadTimestamp("") re.NoError(err) @@ -68,16 +68,15 @@ func TestGlobalLocalTimestamp(t *testing.T) { storage := NewStorageWithEtcdBackend(client, rootPath) ltaKey := "lta" - timestampKey := "timestamp" dc1LocationKey, dc2LocationKey := "dc1", "dc2" localTS1 := time.Now().Round(0) - l1 := path.Join(ltaKey, dc1LocationKey, timestampKey) - l2 := path.Join(ltaKey, dc2LocationKey, timestampKey) + l1 := path.Join(ltaKey, dc1LocationKey, endpoint.TimestampKey) + l2 := path.Join(ltaKey, dc2LocationKey, endpoint.TimestampKey) err = storage.SaveTimestamp(l1, localTS1) re.NoError(err) globalTS := time.Now().Round(0) - err = storage.SaveTimestamp(timestampKey, globalTS) + err = storage.SaveTimestamp(endpoint.TimestampKey, globalTS) re.NoError(err) localTS2 := time.Now().Round(0) err = storage.SaveTimestamp(l2, localTS2) @@ -108,14 +107,12 @@ func TestTimestampTxn(t *testing.T) { rootPath := path.Join("/pd", strconv.FormatUint(100, 10)) storage := NewStorageWithEtcdBackend(client, rootPath) - timestampKey := "timestamp" - globalTS1 := time.Now().Round(0) - err = storage.SaveTimestamp(timestampKey, globalTS1) + err = storage.SaveTimestamp(endpoint.TimestampKey, globalTS1) re.NoError(err) globalTS2 := globalTS1.Add(-time.Millisecond).Round(0) - err = storage.SaveTimestamp(timestampKey, globalTS2) + err = storage.SaveTimestamp(endpoint.TimestampKey, globalTS2) re.Error(err) ts, err := storage.LoadTimestamp("") diff --git a/pkg/tso/allocator_manager.go b/pkg/tso/allocator_manager.go index 6e67ccf6951..d7a8a9eb81d 100644 --- a/pkg/tso/allocator_manager.go +++ b/pkg/tso/allocator_manager.go @@ -46,13 +46,12 @@ import ( const ( // GlobalDCLocation is the Global TSO Allocator's DC location label. - GlobalDCLocation = "global" - checkStep = time.Minute - patrolStep = time.Second - defaultAllocatorLeaderLease = 3 - globalTSOAllocatorEtcdPrefix = "gta" - localTSOAllocatorEtcdPrefix = "lta" - localTSOSuffixEtcdPrefix = "lts" + GlobalDCLocation = "global" + checkStep = time.Minute + patrolStep = time.Second + defaultAllocatorLeaderLease = 3 + localTSOAllocatorEtcdPrefix = "lta" + localTSOSuffixEtcdPrefix = "lts" ) var ( @@ -1406,16 +1405,3 @@ func (am *AllocatorManager) GetLeaderAddr() string { } return leaderAddrs[0] } - -// Construct the timestampOracle path prefix, which is: -// 1. for the default keyspace group: -// "" in /pd/{cluster_id}/timestamp -// 2. for the non-default keyspace groups: -// {group}/gta in /ms/{cluster_id}/tso/{group}/gta/timestamp -func (am *AllocatorManager) getKeyspaceGroupTSPath(groupID uint32) string { - tsPath := "" - if am.kgID != mcsutils.DefaultKeyspaceGroupID { - tsPath = path.Join(fmt.Sprintf("%05d", groupID), globalTSOAllocatorEtcdPrefix) - } - return tsPath -} diff --git a/pkg/tso/global_allocator.go b/pkg/tso/global_allocator.go index 2c715d0cc7c..07eec5490c8 100644 --- a/pkg/tso/global_allocator.go +++ b/pkg/tso/global_allocator.go @@ -28,6 +28,7 @@ import ( "github.com/tikv/pd/pkg/errs" mcsutils "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/slice" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/logutil" "github.com/tikv/pd/pkg/utils/tsoutil" "github.com/tikv/pd/pkg/utils/typeutil" @@ -96,7 +97,7 @@ func NewGlobalTSOAllocator( member: am.member, timestampOracle: ×tampOracle{ client: am.member.GetLeadership().GetClient(), - tsPath: am.getKeyspaceGroupTSPath(am.kgID), + tsPath: endpoint.GetKeyspaceGroupTSPath(am.kgID), storage: am.storage, saveInterval: am.saveInterval, updatePhysicalInterval: am.updatePhysicalInterval, diff --git a/pkg/tso/keyspace_group_manager.go b/pkg/tso/keyspace_group_manager.go index 529e9bcdd37..ec47089405c 100644 --- a/pkg/tso/keyspace_group_manager.go +++ b/pkg/tso/keyspace_group_manager.go @@ -77,11 +77,14 @@ type state struct { keyspaceLookupTable map[uint32]uint32 // splittingGroups is the cache of splitting keyspace group related information. splittingGroups map[uint32]struct{} + // deletedGroups is the cache of deleted keyspace group related information. + deletedGroups map[uint32]struct{} } func (s *state) initialize() { s.keyspaceLookupTable = make(map[uint32]uint32) s.splittingGroups = make(map[uint32]struct{}) + s.deletedGroups = make(map[uint32]struct{}) } func (s *state) deInitialize() { @@ -116,6 +119,28 @@ func (s *state) getKeyspaceGroupMeta( return s.ams[groupID], s.kgs[groupID] } +// getSplittingGroups returns the IDs of the splitting keyspace groups. +func (s *state) getSplittingGroups() []uint32 { + s.RLock() + defer s.RUnlock() + groups := make([]uint32, 0, len(s.splittingGroups)) + for groupID := range s.splittingGroups { + groups = append(groups, groupID) + } + return groups +} + +// getDeletedGroups returns the IDs of the deleted keyspace groups. +func (s *state) getDeletedGroups() []uint32 { + s.RLock() + defer s.RUnlock() + groups := make([]uint32, 0, len(s.deletedGroups)) + for groupID := range s.deletedGroups { + groups = append(groups, groupID) + } + return groups +} + func (s *state) checkTSOSplit( targetGroupID uint32, ) (splitTargetAM, splitSourceAM *AllocatorManager, err error) { @@ -402,9 +427,10 @@ func (kgm *KeyspaceGroupManager) Initialize() error { return errs.ErrLoadKeyspaceGroupsTerminated.Wrap(err) } - kgm.wg.Add(2) + kgm.wg.Add(3) go kgm.primaryPriorityCheckLoop() go kgm.groupSplitPatroller() + go kgm.deletedGroupCleaner() return nil } @@ -915,10 +941,12 @@ func (kgm *KeyspaceGroupManager) deleteKeyspaceGroup(groupID uint32) { am.close() kgm.ams[groupID] = nil } + + kgm.deletedGroups[groupID] = struct{}{} } // exitElectionMembership exits the election membership of the given keyspace group by -// deinitializing the allocator manager, but still keeps the keyspace group info. +// de-initializing the allocator manager, but still keeps the keyspace group info. func (kgm *KeyspaceGroupManager) exitElectionMembership(group *endpoint.KeyspaceGroup) { log.Info("resign election membership", zap.Uint32("keyspace-group-id", group.ID)) @@ -1272,7 +1300,7 @@ func (kgm *KeyspaceGroupManager) mergingChecker(ctx context.Context, mergeTarget // update the newly merged TSO to make sure it is greater than the original ones. var mergedTS time.Time for _, id := range mergeList { - ts, err := kgm.tsoSvcStorage.LoadTimestamp(am.getKeyspaceGroupTSPath(id)) + ts, err := kgm.tsoSvcStorage.LoadTimestamp(endpoint.GetKeyspaceGroupTSPath(id)) if err != nil || ts == typeutil.ZeroTime { log.Error("failed to load the keyspace group TSO", zap.String("member", kgm.tsoServiceID.ServiceAddr), @@ -1336,6 +1364,7 @@ func (kgm *KeyspaceGroupManager) mergingChecker(ctx context.Context, mergeTarget // groupSplitPatroller is used to patrol the groups that are in the on-going // split state and to check if we could speed up the split process. func (kgm *KeyspaceGroupManager) groupSplitPatroller() { + defer logutil.LogPanic() defer kgm.wg.Done() patrolInterval := groupPatrolInterval failpoint.Inject("fastGroupSplitPatroller", func() { @@ -1348,21 +1377,11 @@ func (kgm *KeyspaceGroupManager) groupSplitPatroller() { for { select { case <-kgm.ctx.Done(): - log.Info("group split patroller is exiting") + log.Info("group split patroller exited") return case <-ticker.C: } - kgm.RLock() - if len(kgm.splittingGroups) == 0 { - kgm.RUnlock() - continue - } - var splittingGroups []uint32 - for id := range kgm.splittingGroups { - splittingGroups = append(splittingGroups, id) - } - kgm.RUnlock() - for _, groupID := range splittingGroups { + for _, groupID := range kgm.getSplittingGroups() { am, group := kgm.getKeyspaceGroupMeta(groupID) if !am.IsLeader() { continue @@ -1387,3 +1406,57 @@ func (kgm *KeyspaceGroupManager) groupSplitPatroller() { } } } + +// deletedGroupCleaner is used to clean the deleted keyspace groups related data. +// For example, the TSO keys of the merged keyspace groups remain in the storage. +func (kgm *KeyspaceGroupManager) deletedGroupCleaner() { + defer logutil.LogPanic() + defer kgm.wg.Done() + patrolInterval := groupPatrolInterval + failpoint.Inject("fastDeletedGroupCleaner", func() { + patrolInterval = 200 * time.Millisecond + }) + ticker := time.NewTicker(patrolInterval) + defer ticker.Stop() + log.Info("deleted group cleaner is started", + zap.Duration("patrol-interval", patrolInterval)) + for { + select { + case <-kgm.ctx.Done(): + log.Info("deleted group cleaner exited") + return + case <-ticker.C: + } + for _, groupID := range kgm.getDeletedGroups() { + // Do not clean the default keyspace group data. + if groupID == mcsutils.DefaultKeyspaceGroupID { + continue + } + // Make sure the allocator and group meta are not in use anymore. + am, _ := kgm.getKeyspaceGroupMeta(groupID) + if am != nil { + log.Info("the keyspace group tso allocator has not been closed yet", + zap.Uint32("keyspace-group-id", groupID)) + continue + } + log.Info("delete the keyspace group tso key", + zap.Uint32("keyspace-group-id", groupID)) + // Clean up the remaining TSO keys. + // TODO: support the Local TSO Allocator clean up. + err := kgm.tsoSvcStorage.DeleteTimestamp( + endpoint.GetTimestampPath( + endpoint.GetKeyspaceGroupTSPath(groupID), + ), + ) + if err != nil { + log.Warn("failed to delete the keyspace group tso key", + zap.Uint32("keyspace-group-id", groupID), + zap.Error(err)) + continue + } + kgm.Lock() + delete(kgm.deletedGroups, groupID) + kgm.Unlock() + } + } +} diff --git a/pkg/tso/keyspace_group_manager_test.go b/pkg/tso/keyspace_group_manager_test.go index 2e03418bae7..09baac9a592 100644 --- a/pkg/tso/keyspace_group_manager_test.go +++ b/pkg/tso/keyspace_group_manager_test.go @@ -38,6 +38,7 @@ import ( "github.com/tikv/pd/pkg/utils/tempurl" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/tsoutil" + "github.com/tikv/pd/pkg/utils/typeutil" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/goleak" @@ -91,6 +92,58 @@ func (suite *keyspaceGroupManagerTestSuite) createConfig() *TestServiceConfig { } } +func (suite *keyspaceGroupManagerTestSuite) TestDeletedGroupCleanup() { + re := suite.Require() + re.NoError(failpoint.Enable("github.com/tikv/pd/pkg/tso/fastDeletedGroupCleaner", "return(true)")) + + // Start with the empty keyspace group assignment. + mgr := suite.newUniqueKeyspaceGroupManager(0) + re.NotNil(mgr) + defer mgr.Close() + err := mgr.Initialize() + re.NoError(err) + + rootPath := mgr.legacySvcRootPath + svcAddr := mgr.tsoServiceID.ServiceAddr + + // Add keyspace group 1. + suite.applyEtcdEvents(re, rootPath, []*etcdEvent{generateKeyspaceGroupPutEvent(1, []uint32{1}, []string{svcAddr})}) + // Check if the TSO key is created. + testutil.Eventually(re, func() bool { + ts, err := mgr.tsoSvcStorage.LoadTimestamp(endpoint.GetKeyspaceGroupTSPath(1)) + re.NoError(err) + return ts != typeutil.ZeroTime + }) + // Delete keyspace group 1. + suite.applyEtcdEvents(re, rootPath, []*etcdEvent{generateKeyspaceGroupDeleteEvent(1)}) + // Check if the TSO key is deleted. + testutil.Eventually(re, func() bool { + ts, err := mgr.tsoSvcStorage.LoadTimestamp(endpoint.GetKeyspaceGroupTSPath(1)) + re.NoError(err) + return ts == typeutil.ZeroTime + }) + // Check if the keyspace group is deleted completely. + mgr.RLock() + re.Nil(mgr.ams[1]) + re.Nil(mgr.kgs[1]) + re.NotContains(mgr.deletedGroups, 1) + mgr.RUnlock() + // Try to delete the default keyspace group. + suite.applyEtcdEvents(re, rootPath, []*etcdEvent{generateKeyspaceGroupDeleteEvent(mcsutils.DefaultKeyspaceGroupID)}) + // Default keyspace group should NOT be deleted. + mgr.RLock() + re.NotNil(mgr.ams[mcsutils.DefaultKeyspaceGroupID]) + re.NotNil(mgr.kgs[mcsutils.DefaultKeyspaceGroupID]) + re.NotContains(mgr.deletedGroups, mcsutils.DefaultKeyspaceGroupID) + mgr.RUnlock() + // Default keyspace group TSO key should NOT be deleted. + ts, err := mgr.legacySvcStorage.LoadTimestamp(endpoint.GetKeyspaceGroupTSPath(mcsutils.DefaultKeyspaceGroupID)) + re.NoError(err) + re.NotEmpty(ts) + + re.NoError(failpoint.Disable("github.com/tikv/pd/pkg/tso/fastDeletedGroupCleaner")) +} + // TestNewKeyspaceGroupManager tests the initialization of KeyspaceGroupManager. // It should initialize the allocator manager with the desired configurations and parameters. func (suite *keyspaceGroupManagerTestSuite) TestNewKeyspaceGroupManager() { diff --git a/pkg/tso/tso.go b/pkg/tso/tso.go index 54f0cb927be..1ce039a762c 100644 --- a/pkg/tso/tso.go +++ b/pkg/tso/tso.go @@ -16,7 +16,6 @@ package tso import ( "fmt" - "path" "sync/atomic" "time" @@ -34,7 +33,6 @@ import ( ) const ( - timestampKey = "timestamp" // UpdateTimestampGuard is the min timestamp interval. UpdateTimestampGuard = time.Millisecond // maxLogical is the max upper limit for logical time. @@ -142,7 +140,7 @@ func (t *timestampOracle) calibrateLogical(rawLogical int64, suffixBits int) int // GetTimestampPath returns the timestamp path in etcd. func (t *timestampOracle) GetTimestampPath() string { - return path.Join(t.tsPath, timestampKey) + return endpoint.GetTimestampPath(t.tsPath) } // SyncTimestamp is used to synchronize the timestamp. diff --git a/tests/integrations/mcs/tso/server_test.go b/tests/integrations/mcs/tso/server_test.go index 94e76d3ea03..6de703741ad 100644 --- a/tests/integrations/mcs/tso/server_test.go +++ b/tests/integrations/mcs/tso/server_test.go @@ -204,7 +204,7 @@ func getEtcdTimestampKeyNum(re *require.Assertions, client *clientv3.Client) int var count int for _, kv := range resp.Kvs { key := strings.TrimSpace(string(kv.Key)) - if !strings.HasSuffix(key, "timestamp") { + if !strings.HasSuffix(key, endpoint.TimestampKey) { continue } count++ diff --git a/tools/pd-backup/pdbackup/backup.go b/tools/pd-backup/pdbackup/backup.go index a645138b68b..eeb5ee3cf30 100644 --- a/tools/pd-backup/pdbackup/backup.go +++ b/tools/pd-backup/pdbackup/backup.go @@ -25,6 +25,7 @@ import ( "path" "strconv" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/typeutil" "github.com/tikv/pd/server/config" @@ -74,8 +75,7 @@ func GetBackupInfo(client *clientv3.Client, pdAddr string) (*BackupInfo, error) backInfo.AllocIDMax = allocIDMax - timestampPath := path.Join(rootPath, "timestamp") - resp, err = etcdutil.EtcdKVGet(client, timestampPath) + resp, err = etcdutil.EtcdKVGet(client, endpoint.GetTimestampPath(rootPath)) if err != nil { return nil, err } diff --git a/tools/pd-backup/pdbackup/backup_test.go b/tools/pd-backup/pdbackup/backup_test.go index c747ace21de..8e3ca1eaaac 100644 --- a/tools/pd-backup/pdbackup/backup_test.go +++ b/tools/pd-backup/pdbackup/backup_test.go @@ -16,6 +16,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/testutil" "github.com/tikv/pd/pkg/utils/typeutil" @@ -133,10 +134,9 @@ func (s *backupTestSuite) BeforeTest(suiteName, testName string) { var ( rootPath = path.Join(pdRootPath, strconv.FormatUint(clusterID, 10)) - timestampPath = path.Join(rootPath, "timestamp") allocTimestampMaxBytes = typeutil.Uint64ToBytes(allocTimestampMax) ) - _, err = s.etcdClient.Put(ctx, timestampPath, string(allocTimestampMaxBytes)) + _, err = s.etcdClient.Put(ctx, endpoint.GetTimestampPath(rootPath), string(allocTimestampMaxBytes)) s.NoError(err) var ( From fbedf3ab82fe395e01a73d4e37b7a30ba29e7554 Mon Sep 17 00:00:00 2001 From: ystaticy Date: Thu, 6 Jul 2023 15:56:16 +0800 Subject: [PATCH 15/15] load all gc sp v2 (#6750) ref tikv/pd#6752 Signed-off-by: ystaticy --- client/go.mod | 2 +- client/go.sum | 4 +- go.mod | 2 +- go.sum | 4 +- server/gc_service.go | 69 ++++++++++++++++++++++++++++++++ tests/integrations/client/go.mod | 2 +- tests/integrations/client/go.sum | 4 +- tests/integrations/mcs/go.mod | 2 +- tests/integrations/mcs/go.sum | 4 +- tests/integrations/tso/go.mod | 2 +- tests/integrations/tso/go.sum | 4 +- tools/pd-tso-bench/go.sum | 4 +- 12 files changed, 86 insertions(+), 17 deletions(-) diff --git a/client/go.mod b/client/go.mod index 9f935e03a33..b323ed33d5c 100644 --- a/client/go.mod +++ b/client/go.mod @@ -8,7 +8,7 @@ require ( github.com/opentracing/opentracing-go v1.2.0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 + github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/prometheus/client_golang v1.11.1 github.com/stretchr/testify v1.8.2 diff --git a/client/go.sum b/client/go.sum index 2aec7f63589..0c7ea93ae95 100644 --- a/client/go.sum +++ b/client/go.sum @@ -86,8 +86,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTm github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 h1:VM6INL8StTPYMKufyHRX2hPUMP7isHnkYvtRMA7Sdsc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/go.mod b/go.mod index 5db9cd513ab..61720604c3c 100644 --- a/go.mod +++ b/go.mod @@ -30,7 +30,7 @@ require ( github.com/pingcap/errcode v0.3.0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 + github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/pingcap/sysutil v1.0.1-0.20230407040306-fb007c5aff21 github.com/pingcap/tidb-dashboard v0.0.0-20230705095454-5e220f970f27 diff --git a/go.sum b/go.sum index 3f08eb5b07d..0e9c275d18b 100644 --- a/go.sum +++ b/go.sum @@ -424,8 +424,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 h1:VM6INL8StTPYMKufyHRX2hPUMP7isHnkYvtRMA7Sdsc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= diff --git a/server/gc_service.go b/server/gc_service.go index 0a4f8f4a6c9..fcb0550c15f 100644 --- a/server/gc_service.go +++ b/server/gc_service.go @@ -20,11 +20,14 @@ import ( "fmt" "math" "path" + "strings" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/tso" + "github.com/tikv/pd/pkg/utils/etcdutil" "github.com/tikv/pd/pkg/utils/tsoutil" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" @@ -197,3 +200,69 @@ func (s *GrpcServer) WatchGCSafePointV2(request *pdpb.WatchGCSafePointV2Request, } } } + +// GetAllGCSafePointV2 return all gc safe point v2. +func (s *GrpcServer) GetAllGCSafePointV2(ctx context.Context, request *pdpb.GetAllGCSafePointV2Request) (*pdpb.GetAllGCSafePointV2Response, error) { + fn := func(ctx context.Context, client *grpc.ClientConn) (interface{}, error) { + return pdpb.NewPDClient(client).GetAllGCSafePointV2(ctx, request) + } + if rsp, err := s.unaryMiddleware(ctx, request, fn); err != nil { + return nil, err + } else if rsp != nil { + return rsp.(*pdpb.GetAllGCSafePointV2Response), err + } + + startkey := endpoint.GCSafePointV2Prefix() + endkey := clientv3.GetPrefixRangeEnd(startkey) + _, values, revision, err := s.loadRangeFromETCD(startkey, endkey) + + gcSafePoints := make([]*pdpb.GCSafePointV2, 0, len(values)) + for _, value := range values { + jsonGcSafePoint := &endpoint.GCSafePointV2{} + if err = json.Unmarshal([]byte(value), jsonGcSafePoint); err != nil { + return nil, errs.ErrJSONUnmarshal.Wrap(err).GenWithStackByCause() + } + gcSafePoint := &pdpb.GCSafePointV2{ + KeyspaceId: jsonGcSafePoint.KeyspaceID, + GcSafePoint: jsonGcSafePoint.SafePoint, + } + log.Debug("get all gc safe point v2", + zap.Uint32("keyspace-id", jsonGcSafePoint.KeyspaceID), + zap.Uint64("gc-safe-point", jsonGcSafePoint.SafePoint)) + gcSafePoints = append(gcSafePoints, gcSafePoint) + } + + if err != nil { + return &pdpb.GetAllGCSafePointV2Response{ + Header: s.wrapErrorToHeader(pdpb.ErrorType_UNKNOWN, err.Error()), + }, err + } + + return &pdpb.GetAllGCSafePointV2Response{ + Header: s.header(), + GcSafePoints: gcSafePoints, + Revision: revision, + }, nil +} + +func (s *GrpcServer) loadRangeFromETCD(startKey, endKey string) ([]string, []string, int64, error) { + startKey = strings.Join([]string{s.rootPath, startKey}, "/") + var opOption []clientv3.OpOption + if endKey == "\x00" { + opOption = append(opOption, clientv3.WithPrefix()) + } else { + endKey = strings.Join([]string{s.rootPath, endKey}, "/") + opOption = append(opOption, clientv3.WithRange(endKey)) + } + resp, err := etcdutil.EtcdKVGet(s.client, startKey, opOption...) + if err != nil { + return nil, nil, 0, err + } + keys := make([]string, 0, len(resp.Kvs)) + values := make([]string, 0, len(resp.Kvs)) + for _, item := range resp.Kvs { + keys = append(keys, strings.TrimPrefix(strings.TrimPrefix(string(item.Key), s.rootPath), "/")) + values = append(values, string(item.Value)) + } + return keys, values, resp.Header.Revision, nil +} diff --git a/tests/integrations/client/go.mod b/tests/integrations/client/go.mod index 6211297c389..a5ebdaa3a8f 100644 --- a/tests/integrations/client/go.mod +++ b/tests/integrations/client/go.mod @@ -13,7 +13,7 @@ replace google.golang.org/grpc v1.54.0 => google.golang.org/grpc v1.26.0 require ( github.com/docker/go-units v0.4.0 github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 + github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/stretchr/testify v1.8.2 github.com/tikv/pd v0.0.0-00010101000000-000000000000 diff --git a/tests/integrations/client/go.sum b/tests/integrations/client/go.sum index 208cbc31537..e4566aa4c6c 100644 --- a/tests/integrations/client/go.sum +++ b/tests/integrations/client/go.sum @@ -387,8 +387,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 h1:VM6INL8StTPYMKufyHRX2hPUMP7isHnkYvtRMA7Sdsc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= diff --git a/tests/integrations/mcs/go.mod b/tests/integrations/mcs/go.mod index f421e682b6d..9ae99f0aba6 100644 --- a/tests/integrations/mcs/go.mod +++ b/tests/integrations/mcs/go.mod @@ -12,7 +12,7 @@ replace google.golang.org/grpc v1.54.0 => google.golang.org/grpc v1.26.0 require ( github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 - github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 + github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 github.com/stretchr/testify v1.8.2 github.com/tikv/pd v0.0.0-00010101000000-000000000000 diff --git a/tests/integrations/mcs/go.sum b/tests/integrations/mcs/go.sum index d95617229b7..7b876f7a63e 100644 --- a/tests/integrations/mcs/go.sum +++ b/tests/integrations/mcs/go.sum @@ -388,8 +388,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 h1:VM6INL8StTPYMKufyHRX2hPUMP7isHnkYvtRMA7Sdsc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= diff --git a/tests/integrations/tso/go.mod b/tests/integrations/tso/go.mod index 95035400c43..bd193317f63 100644 --- a/tests/integrations/tso/go.mod +++ b/tests/integrations/tso/go.mod @@ -13,7 +13,7 @@ replace google.golang.org/grpc v1.54.0 => google.golang.org/grpc v1.26.0 require ( github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c - github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 + github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 github.com/stretchr/testify v1.8.4 github.com/tikv/pd v0.0.0-00010101000000-000000000000 github.com/tikv/pd/client v0.0.0-00010101000000-000000000000 diff --git a/tests/integrations/tso/go.sum b/tests/integrations/tso/go.sum index 449141101bb..b95a53d0a1f 100644 --- a/tests/integrations/tso/go.sum +++ b/tests/integrations/tso/go.sum @@ -385,8 +385,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ue github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c h1:CgbKAHto5CQgWM9fSBIvaxsJHuGP0uM74HXtv3MyyGQ= github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 h1:VM6INL8StTPYMKufyHRX2hPUMP7isHnkYvtRMA7Sdsc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= diff --git a/tools/pd-tso-bench/go.sum b/tools/pd-tso-bench/go.sum index 6a6696c3133..7e8996db186 100644 --- a/tools/pd-tso-bench/go.sum +++ b/tools/pd-tso-bench/go.sum @@ -864,8 +864,8 @@ github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTm github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3 h1:TN9FcS+r19rKyrsPJDPfcXWkztVHfbpZ9Xkic6kE+v0= -github.com/pingcap/kvproto v0.0.0-20230703085931-3788ab4ee6b3/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2 h1:VM6INL8StTPYMKufyHRX2hPUMP7isHnkYvtRMA7Sdsc= +github.com/pingcap/kvproto v0.0.0-20230705080816-a48f4fe282a2/go.mod h1:r0q/CFcwvyeRhKtoqzmWMBebrtpIziQQ9vR+JKh1knc= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3 h1:HR/ylkkLmGdSSDaD8IDP+SZrdhV1Kibl9KrHxJ9eciw= github.com/pingcap/log v1.1.1-0.20221110025148-ca232912c9f3/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA=